topics) throws IOException {
log.info("--- Start draining backlog ---");
+ if (workload.consumerOnly) {
+ log.info("Consume only test. Pausing producers while backlog is drained");
+ worker.pauseProducers();
+ }
worker.resumeConsumers();
final long minBacklog = 1000;
@@ -367,6 +383,11 @@ private TestResult printAndCollectStats(long testDurations, TimeUnit unit) throw
TestResult result = new TestResult();
result.workload = workload.name;
result.driver = driverName;
+ result.topics = workload.topics;
+ result.partitions = workload.partitionsPerTopic;
+ result.messageSize = workload.messageSize;
+ result.producersPerTopic = workload.producersPerTopic;
+ result.consumersPerTopic = workload.consumerPerSubscription;
while (true) {
try {
@@ -390,7 +411,7 @@ private TestResult printAndCollectStats(long testDurations, TimeUnit unit) throw
- stats.totalMessagesReceived;
log.info(
- "Pub rate {} msg/s / {} Mb/s | Cons rate {} msg/s / {} Mb/s | Backlog: {} K | Pub Latency (ms) avg: {} - 50%: {} - 99%: {} - 99.9%: {} - Max: {}",
+ "Pub rate {} msg/s / {} MB/s | Cons rate {} msg/s / {} MB/s | Backlog: {} K | Pub Latency (ms) avg: {} - 50%: {} - 99%: {} - 99.9%: {} - Max: {} | Pub Delay Latency (us) avg: {} - 50%: {} - 99%: {} - 99.9%: {} - Max: {}",
rateFormat.format(publishRate), throughputFormat.format(publishThroughput),
rateFormat.format(consumeRate), throughputFormat.format(consumeThroughput),
dec.format(currentBacklog / 1000.0), //
@@ -398,7 +419,19 @@ private TestResult printAndCollectStats(long testDurations, TimeUnit unit) throw
dec.format(microsToMillis(stats.publishLatency.getValueAtPercentile(50))),
dec.format(microsToMillis(stats.publishLatency.getValueAtPercentile(99))),
dec.format(microsToMillis(stats.publishLatency.getValueAtPercentile(99.9))),
- throughputFormat.format(microsToMillis(stats.publishLatency.getMaxValue())));
+ throughputFormat.format(microsToMillis(stats.publishLatency.getMaxValue())),
+ dec.format(stats.publishDelayLatency.getMean()),
+ dec.format(stats.publishDelayLatency.getValueAtPercentile(50)),
+ dec.format(stats.publishDelayLatency.getValueAtPercentile(99)),
+ dec.format(stats.publishDelayLatency.getValueAtPercentile(99.9)),
+ throughputFormat.format(stats.publishDelayLatency.getMaxValue()));
+
+ log.info("E2E Latency (ms) avg: {} - 50%: {} - 99%: {} - 99.9%: {} - Max: {}",
+ dec.format(microsToMillis(stats.endToEndLatency.getMean())),
+ dec.format(microsToMillis(stats.endToEndLatency.getValueAtPercentile(50))),
+ dec.format(microsToMillis(stats.endToEndLatency.getValueAtPercentile(99))),
+ dec.format(microsToMillis(stats.endToEndLatency.getValueAtPercentile(99.9))),
+ throughputFormat.format(microsToMillis(stats.endToEndLatency.getMaxValue())));
result.publishRate.add(publishRate);
result.consumeRate.add(consumeRate);
@@ -412,6 +445,16 @@ private TestResult printAndCollectStats(long testDurations, TimeUnit unit) throw
result.publishLatency9999pct.add(microsToMillis(stats.publishLatency.getValueAtPercentile(99.99)));
result.publishLatencyMax.add(microsToMillis(stats.publishLatency.getMaxValue()));
+ result.publishDelayLatencyAvg.add(stats.publishDelayLatency.getMean());
+ result.publishDelayLatency50pct.add(stats.publishDelayLatency.getValueAtPercentile(50));
+ result.publishDelayLatency75pct.add(stats.publishDelayLatency.getValueAtPercentile(75));
+ result.publishDelayLatency95pct.add(stats.publishDelayLatency.getValueAtPercentile(95));
+ result.publishDelayLatency99pct.add(stats.publishDelayLatency.getValueAtPercentile(99));
+ result.publishDelayLatency999pct.add(stats.publishDelayLatency.getValueAtPercentile(99.9));
+ result.publishDelayLatency9999pct.add(stats.publishDelayLatency.getValueAtPercentile(99.99));
+ result.publishDelayLatencyMax.add(stats.publishDelayLatency.getMaxValue());
+
+
result.endToEndLatencyAvg.add(microsToMillis(stats.endToEndLatency.getMean()));
result.endToEndLatency50pct.add(microsToMillis(stats.endToEndLatency.getValueAtPercentile(50)));
result.endToEndLatency75pct.add(microsToMillis(stats.endToEndLatency.getValueAtPercentile(75)));
@@ -422,38 +465,76 @@ private TestResult printAndCollectStats(long testDurations, TimeUnit unit) throw
result.endToEndLatencyMax.add(microsToMillis(stats.endToEndLatency.getMaxValue()));
if (now >= testEndTime && !needToWaitForBacklogDraining) {
- CumulativeLatencies agg = worker.getCumulativeLatencies();
+ boolean complete = false;
+ int retry = 0;
+ CumulativeLatencies agg = null;
+ do {
+ try {
+ agg = worker.getCumulativeLatencies();
+ } catch (Exception e) {
+ log.info("Retrying");
+ retry++;
+ continue;
+ }
+ complete = true;
+ } while (!complete && retry < 10);
+
+ if (!complete) {
+ throw new RuntimeException("Failed to collect aggregate latencies");
+ }
+
log.info(
- "----- Aggregated Pub Latency (ms) avg: {} - 50%: {} - 95%: {} - 99%: {} - 99.9%: {} - 99.99%: {} - Max: {}",
+ "----- Aggregated Pub Latency (ms) avg: {} - 50%: {} - 95%: {} - 99%: {} - 99.9%: {} - 99.99%: {} - Max: {} | Pub Delay (us) avg: {} - 50%: {} - 95%: {} - 99%: {} - 99.9%: {} - 99.99%: {} - Max: {}",
dec.format(agg.publishLatency.getMean() / 1000.0),
dec.format(agg.publishLatency.getValueAtPercentile(50) / 1000.0),
dec.format(agg.publishLatency.getValueAtPercentile(95) / 1000.0),
dec.format(agg.publishLatency.getValueAtPercentile(99) / 1000.0),
dec.format(agg.publishLatency.getValueAtPercentile(99.9) / 1000.0),
dec.format(agg.publishLatency.getValueAtPercentile(99.99) / 1000.0),
- throughputFormat.format(agg.publishLatency.getMaxValue() / 1000.0));
-
- result.aggregatedPublishLatencyAvg = agg.publishLatency.getMean() / 1000.0;
- result.aggregatedPublishLatency50pct = agg.publishLatency.getValueAtPercentile(50) / 1000.0;
- result.aggregatedPublishLatency75pct = agg.publishLatency.getValueAtPercentile(75) / 1000.0;
- result.aggregatedPublishLatency95pct = agg.publishLatency.getValueAtPercentile(95) / 1000.0;
- result.aggregatedPublishLatency99pct = agg.publishLatency.getValueAtPercentile(99) / 1000.0;
- result.aggregatedPublishLatency999pct = agg.publishLatency.getValueAtPercentile(99.9) / 1000.0;
- result.aggregatedPublishLatency9999pct = agg.publishLatency.getValueAtPercentile(99.99) / 1000.0;
- result.aggregatedPublishLatencyMax = agg.publishLatency.getMaxValue() / 1000.0;
-
- result.aggregatedEndToEndLatencyAvg = agg.endToEndLatency.getMean() / 1000.0;
- result.aggregatedEndToEndLatency50pct = agg.endToEndLatency.getValueAtPercentile(50) / 1000.0;
- result.aggregatedEndToEndLatency75pct = agg.endToEndLatency.getValueAtPercentile(75) / 1000.0;
- result.aggregatedEndToEndLatency95pct = agg.endToEndLatency.getValueAtPercentile(95) / 1000.0;
- result.aggregatedEndToEndLatency99pct = agg.endToEndLatency.getValueAtPercentile(99) / 1000.0;
- result.aggregatedEndToEndLatency999pct = agg.endToEndLatency.getValueAtPercentile(99.9) / 1000.0;
- result.aggregatedEndToEndLatency9999pct = agg.endToEndLatency.getValueAtPercentile(99.99) / 1000.0;
- result.aggregatedEndToEndLatencyMax = agg.endToEndLatency.getMaxValue() / 1000.0;
+ throughputFormat.format(agg.publishLatency.getMaxValue() / 1000.0),
+ dec.format(agg.publishDelayLatency.getMean()),
+ dec.format(agg.publishDelayLatency.getValueAtPercentile(50)),
+ dec.format(agg.publishDelayLatency.getValueAtPercentile(95)),
+ dec.format(agg.publishDelayLatency.getValueAtPercentile(99)),
+ dec.format(agg.publishDelayLatency.getValueAtPercentile(99.9)),
+ dec.format(agg.publishDelayLatency.getValueAtPercentile(99.99)),
+ throughputFormat.format(agg.publishDelayLatency.getMaxValue()));
+
+ result.aggregatedPublishLatencyAvg = microsToMillis(agg.publishLatency.getMean());
+ result.aggregatedPublishLatency50pct = microsToMillis(agg.publishLatency.getValueAtPercentile(50));
+ result.aggregatedPublishLatency75pct = microsToMillis(agg.publishLatency.getValueAtPercentile(75));
+ result.aggregatedPublishLatency95pct = microsToMillis(agg.publishLatency.getValueAtPercentile(95));
+ result.aggregatedPublishLatency99pct = microsToMillis(agg.publishLatency.getValueAtPercentile(99));
+ result.aggregatedPublishLatency999pct = microsToMillis(agg.publishLatency.getValueAtPercentile(99.9));
+ result.aggregatedPublishLatency9999pct = microsToMillis(agg.publishLatency.getValueAtPercentile(99.99));
+ result.aggregatedPublishLatencyMax = microsToMillis(agg.publishLatency.getMaxValue());
+
+ result.aggregatedEndToEndLatencyAvg = microsToMillis(agg.endToEndLatency.getMean());
+ result.aggregatedEndToEndLatency50pct = microsToMillis(agg.endToEndLatency.getValueAtPercentile(50));
+ result.aggregatedEndToEndLatency75pct = microsToMillis(agg.endToEndLatency.getValueAtPercentile(75));
+ result.aggregatedEndToEndLatency95pct = microsToMillis(agg.endToEndLatency.getValueAtPercentile(95));
+ result.aggregatedEndToEndLatency99pct = microsToMillis(agg.endToEndLatency.getValueAtPercentile(99));
+ result.aggregatedEndToEndLatency999pct = microsToMillis(agg.endToEndLatency.getValueAtPercentile(99.9));
+ result.aggregatedEndToEndLatency9999pct = microsToMillis(
+ agg.endToEndLatency.getValueAtPercentile(99.99));
+ result.aggregatedEndToEndLatencyMax = microsToMillis(agg.endToEndLatency.getMaxValue());
+ result.aggregatedPublishDelayLatencyAvg = agg.publishDelayLatency.getMean();
+ result.aggregatedPublishDelayLatency50pct = agg.publishDelayLatency.getValueAtPercentile(50);
+ result.aggregatedPublishDelayLatency75pct = agg.publishDelayLatency.getValueAtPercentile(75);
+ result.aggregatedPublishDelayLatency95pct = agg.publishDelayLatency.getValueAtPercentile(95);
+ result.aggregatedPublishDelayLatency99pct = agg.publishDelayLatency.getValueAtPercentile(99);
+ result.aggregatedPublishDelayLatency999pct = agg.publishDelayLatency.getValueAtPercentile(99.9);
+ result.aggregatedPublishDelayLatency9999pct = agg.publishDelayLatency.getValueAtPercentile(99.99);
+ result.aggregatedPublishDelayLatencyMax = agg.publishDelayLatency.getMaxValue();
agg.publishLatency.percentiles(100).forEach(value -> {
result.aggregatedPublishLatencyQuantiles.put(value.getPercentile(),
- value.getValueIteratedTo() / 1000.0);
+ microsToMillis(value.getValueIteratedTo()));
+ });
+
+ agg.publishDelayLatency.percentiles(100).forEach(value -> {
+ result.aggregatedPublishDelayLatencyQuantiles.put(value.getPercentile(),
+ value.getValueIteratedTo());
});
agg.endToEndLatency.percentiles(100).forEach(value -> {
@@ -470,17 +551,17 @@ private TestResult printAndCollectStats(long testDurations, TimeUnit unit) throw
return result;
}
- private static final DecimalFormat rateFormat = new PaddingDecimalFormat("0.0", 7);
- private static final DecimalFormat throughputFormat = new PaddingDecimalFormat("0.0", 4);
- private static final DecimalFormat dec = new PaddingDecimalFormat("0.0", 4);
-
- private static double microsToMillis(double timeInMicros) {
- return timeInMicros / 1000.0;
+ private static double microsToMillis(double microTime) {
+ return microTime / (1000);
}
- private static double microsToMillis(long timeInMicros) {
- return timeInMicros / 1000.0;
+ private static double microsToMillis(long microTime) {
+ return microTime / (1000.0);
}
+ private static final DecimalFormat rateFormat = new PaddingDecimalFormat("0.000", 7);
+ private static final DecimalFormat throughputFormat = new PaddingDecimalFormat("0.000", 4);
+ private static final DecimalFormat dec = new PaddingDecimalFormat("0.000", 4);
+
private static final Logger log = LoggerFactory.getLogger(WorkloadGenerator.class);
}
diff --git a/benchmark-framework/src/main/java/io/openmessaging/benchmark/utils/RateLimiter.java b/benchmark-framework/src/main/java/io/openmessaging/benchmark/utils/RateLimiter.java
new file mode 100644
index 00000000..10f27eac
--- /dev/null
+++ b/benchmark-framework/src/main/java/io/openmessaging/benchmark/utils/RateLimiter.java
@@ -0,0 +1,388 @@
+package io.openmessaging.benchmark.utils;
+import static com.google.common.base.Preconditions.checkArgument;
+import static com.google.common.base.Preconditions.checkNotNull;
+import static com.google.common.util.concurrent.Uninterruptibles.sleepUninterruptibly;
+import static java.lang.Math.max;
+import static java.util.concurrent.TimeUnit.NANOSECONDS;
+import static java.util.concurrent.TimeUnit.SECONDS;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Stopwatch;
+import com.google.errorprone.annotations.CanIgnoreReturnValue;
+import io.openmessaging.benchmark.utils.SmoothRateLimiter.SmoothBursty;
+import io.openmessaging.benchmark.utils.SmoothRateLimiter.SmoothWarmingUp;
+import java.util.Locale;
+import java.util.concurrent.TimeUnit;
+import org.checkerframework.checker.nullness.qual.Nullable;
+
+public abstract class RateLimiter {
+ /**
+ * Creates a {@code RateLimiter} with the specified stable throughput, given
+ * as "permits per second" (commonly referred to as QPS , queries per
+ * second).
+ *
+ * The returned {@code RateLimiter} ensures that on average no more than
+ * {@code permitsPerSecond} are issued during any given second, with sustained
+ * requests being smoothly spread over each second. When the incoming request
+ * rate exceeds {@code permitsPerSecond} the rate limiter will release one
+ * permit every {@code (1.0 / permitsPerSecond)} seconds. When the rate
+ * limiter is unused, bursts of up to {@code permitsPerSecond} permits will be
+ * allowed, with subsequent requests being smoothly limited at the stable rate
+ * of {@code permitsPerSecond}.
+ *
+ * @param permitsPerSecond the rate of the returned {@code RateLimiter},
+ * measured in how many permits become available per second
+ * @throws IllegalArgumentException if {@code permitsPerSecond} is negative or
+ * zero
+ */
+ // TODO(user): "This is equivalent to
+ // {@code createWithCapacity(permitsPerSecond, 1, TimeUnit.SECONDS)}".
+ public static RateLimiter create(double permitsPerSecond) {
+ /*
+ * The default RateLimiter configuration can save the unused permits of up
+ * to one second. This is to avoid unnecessary stalls in situations like
+ * this: A RateLimiter of 1qps, and 4 threads, all calling acquire() at
+ * these moments:
+ *
+ * T0 at 0 seconds
+ * T1 at 1.05 seconds
+ * T2 at 2 seconds
+ * T3 at 3 seconds
+ *
+ * Due to the slight delay of T1, T2 would have to sleep till 2.05 seconds,
+ * and T3 would also have to sleep till 3.05 seconds.
+ */
+ return create(permitsPerSecond, SleepingStopwatch.createFromSystemTimer());
+ }
+
+ static RateLimiter create(double permitsPerSecond,
+ SleepingStopwatch stopwatch) {
+ RateLimiter rateLimiter =
+ new SmoothBursty(stopwatch, 1.0 /* maxBurstSeconds */);
+ rateLimiter.setRate(permitsPerSecond);
+ return rateLimiter;
+ }
+
+ /**
+ * Creates a {@code RateLimiter} with the specified stable throughput, given
+ * as "permits per second" (commonly referred to as QPS , queries per
+ * second), and a warmup period , during which the {@code RateLimiter}
+ * smoothly ramps up its rate, until it reaches its maximum rate at the end of
+ * the period (as long as there are enough requests to saturate it).
+ * Similarly, if the {@code RateLimiter} is left unused for a duration
+ * of {@code warmupPeriod}, it will gradually return to its "cold" state, i.e.
+ * it will go through the same warming up process as when it was first
+ * created.
+ *
+ *
The returned {@code RateLimiter} is intended for cases where the
+ * resource that actually fulfills the requests (e.g., a remote server) needs
+ * "warmup" time, rather than being immediately accessed at the stable
+ * (maximum) rate.
+ *
+ *
The returned {@code RateLimiter} starts in a "cold" state (i.e. the
+ * warmup period will follow), and if it is left unused for long enough, it
+ * will return to that state.
+ *
+ * @param permitsPerSecond the rate of the returned {@code RateLimiter},
+ * measured in how many permits become available per second
+ * @param warmupPeriod the duration of the period where the {@code
+ * RateLimiter} ramps up its rate, before reaching its stable (maximum)
+ * rate
+ * @param unit the time unit of the warmupPeriod argument
+ * @throws IllegalArgumentException if {@code permitsPerSecond} is negative or
+ * zero or {@code warmupPeriod} is negative
+ */
+ @SuppressWarnings("GoodTime") // should accept a java.time.Duration
+ public static RateLimiter create(double permitsPerSecond, long warmupPeriod,
+ TimeUnit unit) {
+ checkArgument(warmupPeriod >= 0, "warmupPeriod must not be negative: %s",
+ warmupPeriod);
+ return create(permitsPerSecond, warmupPeriod, unit, 3.0,
+ SleepingStopwatch.createFromSystemTimer());
+ }
+
+ @VisibleForTesting
+ static RateLimiter create(double permitsPerSecond, long warmupPeriod,
+ TimeUnit unit, double coldFactor,
+ SleepingStopwatch stopwatch) {
+ RateLimiter rateLimiter =
+ new SmoothWarmingUp(stopwatch, warmupPeriod, unit, coldFactor);
+ rateLimiter.setRate(permitsPerSecond);
+ return rateLimiter;
+ }
+
+ /**
+ * The underlying timer; used both to measure elapsed time and sleep as
+ * necessary. A separate object to facilitate testing.
+ */
+ private final SleepingStopwatch stopwatch;
+
+ // Can't be initialized in the constructor because mocks don't call the
+ // constructor.
+ private volatile @Nullable Object mutexDoNotUseDirectly;
+
+ private Object mutex() {
+ Object mutex = mutexDoNotUseDirectly;
+ if (mutex == null) {
+ synchronized (this) {
+ mutex = mutexDoNotUseDirectly;
+ if (mutex == null) {
+ mutexDoNotUseDirectly = mutex = new Object();
+ }
+ }
+ }
+ return mutex;
+ }
+
+ RateLimiter(SleepingStopwatch stopwatch) {
+ this.stopwatch = checkNotNull(stopwatch);
+ }
+
+ /**
+ * Updates the stable rate of this {@code RateLimiter}, that is, the {@code
+ * permitsPerSecond} argument provided in the factory method that constructed
+ * the {@code RateLimiter}. Currently throttled threads will not be
+ * awakened as a result of this invocation, thus they do not observe the new
+ * rate; only subsequent requests will.
+ *
+ *
Note though that, since each request repays (by waiting, if necessary)
+ * the cost of the previous request, this means that the very next
+ * request after an invocation to {@code setRate} will not be affected by the
+ * new rate; it will pay the cost of the previous request, which is in terms
+ * of the previous rate.
+ *
+ *
The behavior of the {@code RateLimiter} is not modified in any other
+ * way, e.g. if the {@code RateLimiter} was configured with a warmup period of
+ * 20 seconds, it still has a warmup period of 20 seconds after this method
+ * invocation.
+ *
+ * @param permitsPerSecond the new stable rate of this {@code RateLimiter}
+ * @throws IllegalArgumentException if {@code permitsPerSecond} is negative or
+ * zero
+ */
+ public final void setRate(double permitsPerSecond) {
+ checkArgument(permitsPerSecond > 0.0 && !Double.isNaN(permitsPerSecond),
+ "rate must be positive");
+ synchronized (mutex()) {
+ doSetRate(permitsPerSecond, stopwatch.readNanos());
+ }
+ }
+
+ abstract void doSetRate(double permitsPerSecond, long nowNanos);
+
+ /**
+ * Returns the stable rate (as {@code permits per seconds}) with which this
+ * {@code RateLimiter} is configured with. The initial value of this is the
+ * same as the {@code permitsPerSecond} argument passed in the factory method
+ * that produced this {@code RateLimiter}, and it is only updated after
+ * invocations to {@linkplain #setRate}.
+ */
+ public final double getRate() {
+ synchronized (mutex()) { return doGetRate(); }
+ }
+
+ abstract double doGetRate();
+
+ /**
+ * Acquires a single permit from this {@code RateLimiter}, blocking until the
+ * request can be granted. Tells the amount of time slept, if any.
+ *
+ *
This method is equivalent to {@code acquire(1)}.
+ *
+ * @return time spent sleeping to enforce rate, in seconds; 0.0 if not
+ * rate-limited
+ * @since 16.0 (present in 13.0 with {@code void} return type})
+ */
+ @CanIgnoreReturnValue
+ public double acquire() {
+ return acquire(1);
+ }
+
+ /**
+ * Acquires the given number of permits from this {@code RateLimiter},
+ * blocking until the request can be granted. Tells the amount of time slept,
+ * if any.
+ *
+ * @param permits the number of permits to acquire
+ * @return time spent sleeping to enforce rate, in seconds; 0.0 if not
+ * rate-limited
+ * @throws IllegalArgumentException if the requested number of permits is
+ * negative or zero
+ * @since 16.0 (present in 13.0 with {@code void} return type})
+ */
+ @CanIgnoreReturnValue
+ public double acquire(int permits) {
+ long nanosToWait = reserve(permits);
+ stopwatch.sleepNanosUninterruptibly(nanosToWait);
+ return 1.0 * nanosToWait / SECONDS.toNanos(1L);
+ }
+
+ /**
+ * Reserves the given number of permits from this {@code RateLimiter} for
+ * future use, returning the number of NANOSECONDS until the reservation can
+ * be consumed.
+ *
+ * @return time in NANOSECONDS to wait until the resource can be acquired,
+ * never negative
+ */
+ final long reserve(int permits) {
+ checkPermits(permits);
+ synchronized (mutex()) {
+ return reserveAndGetWaitLength(permits, stopwatch.readNanos());
+ }
+ }
+
+ /**
+ * Acquires a permit from this {@code RateLimiter} if it can be obtained
+ * without exceeding the specified {@code timeout}, or returns {@code false}
+ * immediately (without waiting) if the permit would not have been granted
+ * before the timeout expired.
+ *
+ *
This method is equivalent to {@code tryAcquire(1, timeout, unit)}.
+ *
+ * @param timeout the maximum time to wait for the permit. Negative values are
+ * treated as zero.
+ * @param unit the time unit of the timeout argument
+ * @return {@code true} if the permit was acquired, {@code false} otherwise
+ * @throws IllegalArgumentException if the requested number of permits is
+ * negative or zero
+ */
+ @SuppressWarnings("GoodTime") // should accept a java.time.Duration
+ public boolean tryAcquire(long timeout, TimeUnit unit) {
+ return tryAcquire(1, timeout, unit);
+ }
+
+ /**
+ * Acquires permits from this {@link RateLimiter} if it can be acquired
+ * immediately without delay.
+ *
+ *
This method is equivalent to {@code tryAcquire(permits, 0, anyUnit)}.
+ *
+ * @param permits the number of permits to acquire
+ * @return {@code true} if the permits were acquired, {@code false} otherwise
+ * @throws IllegalArgumentException if the requested number of permits is
+ * negative or zero
+ * @since 14.0
+ */
+ public boolean tryAcquire(int permits) {
+ return tryAcquire(permits, 0, NANOSECONDS);
+ }
+
+ /**
+ * Acquires a permit from this {@link RateLimiter} if it can be acquired
+ * immediately without delay.
+ *
+ *
This method is equivalent to {@code tryAcquire(1)}.
+ *
+ * @return {@code true} if the permit was acquired, {@code false} otherwise
+ * @since 14.0
+ */
+ public boolean tryAcquire() { return tryAcquire(1, 0, NANOSECONDS); }
+
+ /**
+ * Acquires the given number of permits from this {@code RateLimiter} if it
+ * can be obtained without exceeding the specified {@code timeout}, or returns
+ * {@code false} immediately (without waiting) if the permits would not have
+ * been granted before the timeout expired.
+ *
+ * @param permits the number of permits to acquire
+ * @param timeout the maximum time to wait for the permits. Negative values
+ * are treated as zero.
+ * @param unit the time unit of the timeout argument
+ * @return {@code true} if the permits were acquired, {@code false} otherwise
+ * @throws IllegalArgumentException if the requested number of permits is
+ * negative or zero
+ */
+ @SuppressWarnings("GoodTime") // should accept a java.time.Duration
+ public boolean tryAcquire(int permits, long timeout, TimeUnit unit) {
+ long timeoutNanos = max(unit.toNanos(timeout), 0);
+ checkPermits(permits);
+ long nanosToWait;
+ synchronized (mutex()) {
+ long nowNanos = stopwatch.readNanos();
+ if (!canAcquire(nowNanos, timeoutNanos)) {
+ return false;
+ } else {
+ nanosToWait = reserveAndGetWaitLength(permits, nowNanos);
+ }
+ }
+ stopwatch.sleepNanosUninterruptibly(nanosToWait);
+ return true;
+ }
+
+ private boolean canAcquire(long nowNanos, long timeoutNanos) {
+ return queryEarliestAvailable(nowNanos) - timeoutNanos <= nowNanos;
+ }
+
+ /**
+ * Reserves next ticket and returns the wait time that the caller must wait
+ * for.
+ *
+ * @return the required wait time, never negative
+ */
+ final long reserveAndGetWaitLength(int permits, long nowNanos) {
+ long momentAvailable = reserveEarliestAvailable(permits, nowNanos);
+ return max(momentAvailable - nowNanos, 0);
+ }
+
+ /**
+ * Returns the earliest time that permits are available (with one caveat).
+ *
+ * @return the time that permits are available, or, if permits are available
+ * immediately, an arbitrary past or present time
+ */
+ abstract long queryEarliestAvailable(long nowNanos);
+
+ /**
+ * Reserves the requested number of permits and returns the time that those
+ * permits can be used (with one caveat).
+ *
+ * @return the time that the permits may be used, or, if the permits may be
+ * used immediately, an arbitrary past or present time
+ */
+ abstract long reserveEarliestAvailable(int permits, long nowNanos);
+
+ @Override
+ public String toString() {
+ return String.format(Locale.ROOT, "RateLimiter[stableRate=%3.1fqps]",
+ getRate());
+ }
+
+ abstract static class SleepingStopwatch {
+ /** Constructor for use by subclasses. */
+ protected SleepingStopwatch() {}
+
+ /*
+ * We always hold the mutex when calling this. TODO(cpovirk): Is that
+ * important? Perhaps we need to guarantee that each call to
+ * reserveEarliestAvailable, etc. sees a value >= the previous? Also, is it
+ * OK that we don't hold the mutex when sleeping?
+ */
+ protected abstract long readNanos();
+
+ protected abstract void sleepNanosUninterruptibly(long nanos);
+
+ public static SleepingStopwatch createFromSystemTimer() {
+ return new SleepingStopwatch() {
+ final Stopwatch stopwatch = Stopwatch.createStarted();
+
+ @Override
+ protected long readNanos() {
+ return stopwatch.elapsed(NANOSECONDS);
+ }
+
+ @Override
+ protected void sleepNanosUninterruptibly(long nanos) {
+ if (nanos > 0) {
+ sleepUninterruptibly(nanos, NANOSECONDS);
+ }
+ }
+ };
+ }
+ }
+
+ private static void checkPermits(int permits) {
+ checkArgument(permits > 0, "Requested permits (%s) must be positive",
+ permits);
+ }
+}
diff --git a/benchmark-framework/src/main/java/io/openmessaging/benchmark/utils/SmoothRateLimiter.java b/benchmark-framework/src/main/java/io/openmessaging/benchmark/utils/SmoothRateLimiter.java
new file mode 100644
index 00000000..6ce2453c
--- /dev/null
+++ b/benchmark-framework/src/main/java/io/openmessaging/benchmark/utils/SmoothRateLimiter.java
@@ -0,0 +1,397 @@
+/*
+ * Copyright (C) 2012 The Guava Authors
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except
+ * in compliance with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License
+ * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
+ * or implied. See the License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package io.openmessaging.benchmark.utils;
+
+import static java.lang.Math.min;
+import static java.util.concurrent.TimeUnit.SECONDS;
+
+import com.google.common.annotations.GwtIncompatible;
+import com.google.common.math.LongMath;
+import java.util.concurrent.TimeUnit;
+
+@GwtIncompatible
+abstract class SmoothRateLimiter extends RateLimiter {
+ /*
+ * How is the RateLimiter designed, and why?
+ *
+ * The primary feature of a RateLimiter is its "stable rate", the maximum rate that is should
+ * allow at normal conditions. This is enforced by "throttling" incoming requests as needed, i.e.
+ * compute, for an incoming request, the appropriate throttle time, and make the calling thread
+ * wait as much.
+ *
+ * The simplest way to maintain a rate of QPS is to keep the timestamp of the last granted
+ * request, and ensure that (1/QPS) seconds have elapsed since then. For example, for a rate of
+ * QPS=5 (5 tokens per second), if we ensure that a request isn't granted earlier than 200ms after
+ * the last one, then we achieve the intended rate. If a request comes and the last request was
+ * granted only 100ms ago, then we wait for another 100ms. At this rate, serving 15 fresh permits
+ * (i.e. for an acquire(15) request) naturally takes 3 seconds.
+ *
+ * It is important to realize that such a RateLimiter has a very superficial memory of the past:
+ * it only remembers the last request. What if the RateLimiter was unused for a long period of
+ * time, then a request arrived and was immediately granted? This RateLimiter would immediately
+ * forget about that past underutilization. This may result in either underutilization or
+ * overflow, depending on the real world consequences of not using the expected rate.
+ *
+ * Past underutilization could mean that excess resources are available. Then, the RateLimiter
+ * should speed up for a while, to take advantage of these resources. This is important when the
+ * rate is applied to networking (limiting bandwidth), where past underutilization typically
+ * translates to "almost empty buffers", which can be filled immediately.
+ *
+ * On the other hand, past underutilization could mean that "the server responsible for handling
+ * the request has become less ready for future requests", i.e. its caches become stale, and
+ * requests become more likely to trigger expensive operations (a more extreme case of this
+ * example is when a server has just booted, and it is mostly busy with getting itself up to
+ * speed).
+ *
+ * To deal with such scenarios, we add an extra dimension, that of "past underutilization",
+ * modeled by "storedPermits" variable. This variable is zero when there is no underutilization,
+ * and it can grow up to maxStoredPermits, for sufficiently large underutilization. So, the
+ * requested permits, by an invocation acquire(permits), are served from:
+ *
+ * - stored permits (if available)
+ *
+ * - fresh permits (for any remaining permits)
+ *
+ * How this works is best explained with an example:
+ *
+ * For a RateLimiter that produces 1 token per second, every second that goes by with the
+ * RateLimiter being unused, we increase storedPermits by 1. Say we leave the RateLimiter unused
+ * for 10 seconds (i.e., we expected a request at time X, but we are at time X + 10 seconds before
+ * a request actually arrives; this is also related to the point made in the last paragraph), thus
+ * storedPermits becomes 10.0 (assuming maxStoredPermits >= 10.0). At that point, a request of
+ * acquire(3) arrives. We serve this request out of storedPermits, and reduce that to 7.0 (how
+ * this is translated to throttling time is discussed later). Immediately after, assume that an
+ * acquire(10) request arriving. We serve the request partly from storedPermits, using all the
+ * remaining 7.0 permits, and the remaining 3.0, we serve them by fresh permits produced by the
+ * rate limiter.
+ *
+ * We already know how much time it takes to serve 3 fresh permits: if the rate is
+ * "1 token per second", then this will take 3 seconds. But what does it mean to serve 7 stored
+ * permits? As explained above, there is no unique answer. If we are primarily interested to deal
+ * with underutilization, then we want stored permits to be given out /faster/ than fresh ones,
+ * because underutilization = free resources for the taking. If we are primarily interested to
+ * deal with overflow, then stored permits could be given out /slower/ than fresh ones. Thus, we
+ * require a (different in each case) function that translates storedPermits to throtting time.
+ *
+ * This role is played by storedPermitsToWaitTime(double storedPermits, double permitsToTake). The
+ * underlying model is a continuous function mapping storedPermits (from 0.0 to maxStoredPermits)
+ * onto the 1/rate (i.e. intervals) that is effective at the given storedPermits. "storedPermits"
+ * essentially measure unused time; we spend unused time buying/storing permits. Rate is
+ * "permits / time", thus "1 / rate = time / permits". Thus, "1/rate" (time / permits) times
+ * "permits" gives time, i.e., integrals on this function (which is what storedPermitsToWaitTime()
+ * computes) correspond to minimum intervals between subsequent requests, for the specified number
+ * of requested permits.
+ *
+ * Here is an example of storedPermitsToWaitTime: If storedPermits == 10.0, and we want 3 permits,
+ * we take them from storedPermits, reducing them to 7.0, and compute the throttling for these as
+ * a call to storedPermitsToWaitTime(storedPermits = 10.0, permitsToTake = 3.0), which will
+ * evaluate the integral of the function from 7.0 to 10.0.
+ *
+ * Using integrals guarantees that the effect of a single acquire(3) is equivalent to {
+ * acquire(1); acquire(1); acquire(1); }, or { acquire(2); acquire(1); }, etc, since the integral
+ * of the function in [7.0, 10.0] is equivalent to the sum of the integrals of [7.0, 8.0], [8.0,
+ * 9.0], [9.0, 10.0] (and so on), no matter what the function is. This guarantees that we handle
+ * correctly requests of varying weight (permits), /no matter/ what the actual function is - so we
+ * can tweak the latter freely. (The only requirement, obviously, is that we can compute its
+ * integrals).
+ *
+ * Note well that if, for this function, we chose a horizontal line, at height of exactly (1/QPS),
+ * then the effect of the function is non-existent: we serve storedPermits at exactly the same
+ * cost as fresh ones (1/QPS is the cost for each). We use this trick later.
+ *
+ * If we pick a function that goes /below/ that horizontal line, it means that we reduce the area
+ * of the function, thus time. Thus, the RateLimiter becomes /faster/ after a period of
+ * underutilization. If, on the other hand, we pick a function that goes /above/ that horizontal
+ * line, then it means that the area (time) is increased, thus storedPermits are more costly than
+ * fresh permits, thus the RateLimiter becomes /slower/ after a period of underutilization.
+ *
+ * Last, but not least: consider a RateLimiter with rate of 1 permit per second, currently
+ * completely unused, and an expensive acquire(100) request comes. It would be nonsensical to just
+ * wait for 100 seconds, and /then/ start the actual task. Why wait without doing anything? A much
+ * better approach is to /allow/ the request right away (as if it was an acquire(1) request
+ * instead), and postpone /subsequent/ requests as needed. In this version, we allow starting the
+ * task immediately, and postpone by 100 seconds future requests, thus we allow for work to get
+ * done in the meantime instead of waiting idly.
+ *
+ * This has important consequences: it means that the RateLimiter doesn't remember the time of the
+ * _last_ request, but it remembers the (expected) time of the _next_ request. This also enables
+ * us to tell immediately (see tryAcquire(timeout)) whether a particular timeout is enough to get
+ * us to the point of the next scheduling time, since we always maintain that. And what we mean by
+ * "an unused RateLimiter" is also defined by that notion: when we observe that the
+ * "expected arrival time of the next request" is actually in the past, then the difference (now -
+ * past) is the amount of time that the RateLimiter was formally unused, and it is that amount of
+ * time which we translate to storedPermits. (We increase storedPermits with the amount of permits
+ * that would have been produced in that idle time). So, if rate == 1 permit per second, and
+ * arrivals come exactly one second after the previous, then storedPermits is _never_ increased --
+ * we would only increase it for arrivals _later_ than the expected one second.
+ */
+
+ /**
+ * This implements the following function where coldInterval = coldFactor * stableInterval.
+ *
+ *
+ * ^ throttling
+ * |
+ * cold + /
+ * interval | /.
+ * | / .
+ * | / . ← "warmup period" is the area of the trapezoid between
+ * | / . thresholdPermits and maxPermits
+ * | / .
+ * | / .
+ * | / .
+ * stable +----------/ WARM .
+ * interval | . UP .
+ * | . PERIOD.
+ * | . .
+ * 0 +----------+-------+--------------→ storedPermits
+ * 0 thresholdPermits maxPermits
+ *
+ *
+ * Before going into the details of this particular function, let's keep in mind the basics:
+ *
+ *
+ * The state of the RateLimiter (storedPermits) is a vertical line in this figure.
+ * When the RateLimiter is not used, this goes right (up to maxPermits)
+ * When the RateLimiter is used, this goes left (down to zero), since if we have
+ * storedPermits, we serve from those first
+ * When _unused_, we go right at a constant rate! The rate at which we move to the right is
+ * chosen as maxPermits / warmupPeriod. This ensures that the time it takes to go from 0 to
+ * maxPermits is equal to warmupPeriod.
+ * When _used_, the time it takes, as explained in the introductory class note, is equal to
+ * the integral of our function, between X permits and X-K permits, assuming we want to
+ * spend K saved permits.
+ *
+ *
+ * In summary, the time it takes to move to the left (spend K permits), is equal to the area of
+ * the function of width == K.
+ *
+ *
Assuming we have saturated demand, the time to go from maxPermits to thresholdPermits is
+ * equal to warmupPeriod. And the time to go from thresholdPermits to 0 is warmupPeriod/2. (The
+ * reason that this is warmupPeriod/2 is to maintain the behavior of the original implementation
+ * where coldFactor was hard coded as 3.)
+ *
+ *
It remains to calculate thresholdsPermits and maxPermits.
+ *
+ *
+ * The time to go from thresholdPermits to 0 is equal to the integral of the function
+ * between 0 and thresholdPermits. This is thresholdPermits * stableIntervals. By (5) it is
+ * also equal to warmupPeriod/2. Therefore
+ *
+ * thresholdPermits = 0.5 * warmupPeriod / stableInterval
+ *
+ *
+ * The time to go from maxPermits to thresholdPermits is equal to the integral of the
+ * function between thresholdPermits and maxPermits. This is the area of the pictured
+ * trapezoid, and it is equal to 0.5 * (stableInterval + coldInterval) * (maxPermits -
+ * thresholdPermits). It is also equal to warmupPeriod, so
+ *
+ * maxPermits = thresholdPermits + 2 * warmupPeriod / (stableInterval + coldInterval)
+ *
+ *
+ *
+ */
+ static final class SmoothWarmingUp extends SmoothRateLimiter {
+ private final long warmupPeriodNanos;
+ /**
+ * The slope of the line from the stable interval (when permits == 0), to the cold interval
+ * (when permits == maxPermits)
+ */
+ private double slope;
+ private double thresholdPermits;
+ private double coldFactor;
+
+ SmoothWarmingUp(
+ SleepingStopwatch stopwatch, long warmupPeriod, TimeUnit timeUnit, double coldFactor) {
+ super(stopwatch);
+ this.warmupPeriodNanos = timeUnit.toNanos(warmupPeriod);
+ this.coldFactor = coldFactor;
+ }
+
+ @Override
+ void doSetRate(double permitsPerSecond, double stableIntervalNanos) {
+ double oldMaxPermits = maxPermits;
+ double coldIntervalNanos = stableIntervalNanos * coldFactor;
+ thresholdPermits = 0.5 * warmupPeriodNanos / stableIntervalNanos;
+ maxPermits =
+ thresholdPermits + 2.0 * warmupPeriodNanos / (stableIntervalNanos + coldIntervalNanos);
+ slope = (coldIntervalNanos - stableIntervalNanos) / (maxPermits - thresholdPermits);
+ if (oldMaxPermits == Double.POSITIVE_INFINITY) {
+ // if we don't special-case this, we would get storedPermits == NaN, below
+ storedPermits = 0.0;
+ } else {
+ storedPermits =
+ (oldMaxPermits == 0.0)
+ ? maxPermits // initial state is cold
+ : storedPermits * maxPermits / oldMaxPermits;
+ }
+ }
+
+ @Override
+ long storedPermitsToWaitTime(double storedPermits, double permitsToTake) {
+ double availablePermitsAboveThreshold = storedPermits - thresholdPermits;
+ long nanos = 0;
+ // measuring the integral on the right part of the function (the climbing line)
+ if (availablePermitsAboveThreshold > 0.0) {
+ double permitsAboveThresholdToTake = min(availablePermitsAboveThreshold, permitsToTake);
+ // TODO(cpovirk): Figure out a good name for this variable.
+ double length = permitsToTime(availablePermitsAboveThreshold)
+ + permitsToTime(availablePermitsAboveThreshold - permitsAboveThresholdToTake);
+ nanos = (long) (permitsAboveThresholdToTake * length / 2.0);
+ permitsToTake -= permitsAboveThresholdToTake;
+ }
+ // measuring the integral on the left part of the function (the horizontal line)
+ nanos += (long) (stableIntervalNanos * permitsToTake);
+ return nanos;
+ }
+
+ private double permitsToTime(double permits) {
+ return stableIntervalNanos + permits * slope;
+ }
+
+ @Override
+ double coolDownIntervalNanos() {
+ return warmupPeriodNanos / maxPermits;
+ }
+ }
+
+ /**
+ * This implements a "bursty" RateLimiter, where storedPermits are translated to zero throttling.
+ * The maximum number of permits that can be saved (when the RateLimiter is unused) is defined in
+ * terms of time, in this sense: if a RateLimiter is 2qps, and this time is specified as 10
+ * seconds, we can save up to 2 * 10 = 20 permits.
+ */
+ static final class SmoothBursty extends SmoothRateLimiter {
+ /** The work (permits) of how many seconds can be saved up if this RateLimiter is unused? */
+ final double maxBurstSeconds;
+
+ SmoothBursty(SleepingStopwatch stopwatch, double maxBurstSeconds) {
+ super(stopwatch);
+ this.maxBurstSeconds = maxBurstSeconds;
+ }
+
+ @Override
+ void doSetRate(double permitsPerSecond, double stableIntervalNanos) {
+ double oldMaxPermits = this.maxPermits;
+ maxPermits = maxBurstSeconds * permitsPerSecond;
+ if (oldMaxPermits == Double.POSITIVE_INFINITY) {
+ // if we don't special-case this, we would get storedPermits == NaN, below
+ storedPermits = maxPermits;
+ } else {
+ storedPermits =
+ (oldMaxPermits == 0.0)
+ ? 0.0 // initial state
+ : storedPermits * maxPermits / oldMaxPermits;
+ }
+ }
+
+ @Override
+ long storedPermitsToWaitTime(double storedPermits, double permitsToTake) {
+ return 0L;
+ }
+
+ @Override
+ double coolDownIntervalNanos() {
+ return stableIntervalNanos;
+ }
+ }
+
+ /**
+ * The currently stored permits.
+ */
+ double storedPermits;
+
+ /**
+ * The maximum number of stored permits.
+ */
+ double maxPermits;
+
+ /**
+ * The interval between two unit requests, at our stable rate. E.g., a stable rate of 5 permits
+ * per second has a stable interval of 200ms.
+ */
+ double stableIntervalNanos;
+
+ /**
+ * The time when the next request (no matter its size) will be granted. After granting a request,
+ * this is pushed further in the future. Large requests push this further than small requests.
+ */
+ private long nextFreeTicketNanos = 0L; // could be either in the past or future
+
+ private SmoothRateLimiter(SleepingStopwatch stopwatch) {
+ super(stopwatch);
+ }
+
+ @Override
+ final void doSetRate(double permitsPerSecond, long nowNanos) {
+ resync(nowNanos);
+ double stableIntervalNanos = SECONDS.toNanos(1L) / permitsPerSecond;
+ this.stableIntervalNanos = stableIntervalNanos;
+ doSetRate(permitsPerSecond, stableIntervalNanos);
+ }
+
+ abstract void doSetRate(double permitsPerSecond, double stableIntervalNanos);
+
+ @Override
+ final double doGetRate() {
+ return SECONDS.toNanos(1L) / stableIntervalNanos;
+ }
+
+ @Override
+ final long queryEarliestAvailable(long nowNanos) {
+ return nextFreeTicketNanos;
+ }
+
+ @Override
+ final long reserveEarliestAvailable(int requiredPermits, long nowNanos) {
+ resync(nowNanos);
+ long returnValue = nextFreeTicketNanos;
+ double storedPermitsToSpend = min(requiredPermits, this.storedPermits);
+ double freshPermits = requiredPermits - storedPermitsToSpend;
+ long waitNanos =
+ storedPermitsToWaitTime(this.storedPermits, storedPermitsToSpend)
+ + (long) (freshPermits * stableIntervalNanos);
+
+ this.nextFreeTicketNanos = LongMath.saturatedAdd(nextFreeTicketNanos, waitNanos);
+ this.storedPermits -= storedPermitsToSpend;
+ return returnValue;
+ }
+
+ /**
+ * Translates a specified portion of our currently stored permits which we want to spend/acquire,
+ * into a throttling time. Conceptually, this evaluates the integral of the underlying function we
+ * use, for the range of [(storedPermits - permitsToTake), storedPermits].
+ *
+ * This always holds: {@code 0 <= permitsToTake <= storedPermits}
+ */
+ abstract long storedPermitsToWaitTime(double storedPermits, double permitsToTake);
+
+ /**
+ * Returns the number of nanoseconds during cool down that we have to wait to get a new permit.
+ */
+ abstract double coolDownIntervalNanos();
+
+ /**
+ * Updates {@code storedPermits} and {@code nextFreeTicketNanos} based on the current time.
+ */
+ void resync(long nowNanos) {
+ // if nextFreeTicket is in the past, resync to now
+ if (nowNanos > nextFreeTicketNanos) {
+ double newPermits = (nowNanos - nextFreeTicketNanos) / coolDownIntervalNanos();
+ storedPermits = min(maxPermits, storedPermits + newPermits);
+ nextFreeTicketNanos = nowNanos;
+ }
+ }
+}
diff --git a/benchmark-framework/src/main/java/io/openmessaging/benchmark/utils/UniformRateLimiter.java b/benchmark-framework/src/main/java/io/openmessaging/benchmark/utils/UniformRateLimiter.java
new file mode 100644
index 00000000..2c31cb1c
--- /dev/null
+++ b/benchmark-framework/src/main/java/io/openmessaging/benchmark/utils/UniformRateLimiter.java
@@ -0,0 +1,79 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package io.openmessaging.benchmark.utils;
+
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLongFieldUpdater;
+import java.util.concurrent.locks.LockSupport;
+
+/**
+ * Provides a next operation time for rate limited operation streams.
+ * The rate limiter is thread safe and can be shared by all threads.
+ */
+public final class UniformRateLimiter {
+
+ private static final AtomicLongFieldUpdater V_TIME_UPDATER =
+ AtomicLongFieldUpdater.newUpdater(UniformRateLimiter.class, "virtualTime");
+ private static final AtomicLongFieldUpdater START_UPDATER =
+ AtomicLongFieldUpdater.newUpdater(UniformRateLimiter.class, "start");
+ private static final double ONE_SEC_IN_NS = TimeUnit.SECONDS.toNanos(1);
+ private volatile long start = Long.MIN_VALUE;
+ private volatile long virtualTime;
+ private final double opsPerSec;
+ private final long intervalNs;
+
+ public UniformRateLimiter(final double opsPerSec) {
+ if (Double.isNaN(opsPerSec) || Double.isInfinite(opsPerSec)) {
+ throw new IllegalArgumentException("opsPerSec cannot be Nan or Infinite");
+ }
+ if (opsPerSec <= 0) {
+ throw new IllegalArgumentException("opsPerSec must be greater then 0");
+ }
+ this.opsPerSec = opsPerSec;
+ intervalNs = Math.round(ONE_SEC_IN_NS / opsPerSec);
+ }
+
+ public double getOpsPerSec() {
+ return opsPerSec;
+ }
+
+ public long getIntervalNs() {
+ return intervalNs;
+ }
+
+ public long acquire() {
+ final long currOpIndex = V_TIME_UPDATER.getAndIncrement(this);
+ long start = this.start;
+ if (start == Long.MIN_VALUE) {
+ start = System.nanoTime();
+ if (!START_UPDATER.compareAndSet(this, Long.MIN_VALUE, start)) {
+ start = this.start;
+ assert start != Long.MIN_VALUE;
+ }
+ }
+ return start + currOpIndex * intervalNs;
+ }
+
+ public static void uninterruptibleSleepNs(final long intendedTime) {
+ long sleepNs;
+ while ((sleepNs = (intendedTime - System.nanoTime())) > 0) {
+ LockSupport.parkNanos(sleepNs);
+ }
+ }
+}
diff --git a/benchmark-framework/src/main/java/io/openmessaging/benchmark/worker/DistributedWorkersEnsemble.java b/benchmark-framework/src/main/java/io/openmessaging/benchmark/worker/DistributedWorkersEnsemble.java
index f10b0cf4..a2c4958e 100644
--- a/benchmark-framework/src/main/java/io/openmessaging/benchmark/worker/DistributedWorkersEnsemble.java
+++ b/benchmark-framework/src/main/java/io/openmessaging/benchmark/worker/DistributedWorkersEnsemble.java
@@ -19,7 +19,6 @@
package io.openmessaging.benchmark.worker;
import static java.util.stream.Collectors.toList;
-import static org.asynchttpclient.Dsl.asyncHttpClient;
import java.io.File;
import java.io.IOException;
@@ -27,11 +26,15 @@
import java.nio.file.Files;
import java.nio.file.Paths;
import java.util.List;
+import java.util.ArrayList;
import java.util.Map;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.TimeUnit;
import java.util.zip.DataFormatException;
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.core.type.TypeReference;
+
import org.HdrHistogram.Histogram;
import org.apache.pulsar.common.util.FutureUtil;
import org.asynchttpclient.AsyncHttpClient;
@@ -55,19 +58,22 @@
import io.openmessaging.benchmark.worker.commands.ProducerWorkAssignment;
import io.openmessaging.benchmark.worker.commands.TopicSubscription;
import io.openmessaging.benchmark.worker.commands.TopicsInfo;
+import static org.asynchttpclient.Dsl.*;
public class DistributedWorkersEnsemble implements Worker {
+ private final static int REQUEST_TIMEOUT_MS = 300_000;
+ private final static int READ_TIMEOUT_MS = 300_000;
private final List workers;
private final List producerWorkers;
private final List consumerWorkers;
- private final AsyncHttpClient httpClient = asyncHttpClient();
+ private final AsyncHttpClient httpClient;
private int numberOfUsedProducerWorkers;
public DistributedWorkersEnsemble(List workers) {
- Preconditions.checkArgument(workers.size() > 1);
+ Preconditions.checkArgument(workers.size() > 1, "Workers must be > 1");
this.workers = workers;
List> partitions = Lists.partition(workers, workers.size() / 2);
@@ -76,6 +82,8 @@ public DistributedWorkersEnsemble(List workers) {
log.info("Workers list - producers: {}", producerWorkers);
log.info("Workers list - consumers: {}", consumerWorkers);
+
+ httpClient = asyncHttpClient(config().setRequestTimeout(REQUEST_TIMEOUT_MS).setReadTimeout(READ_TIMEOUT_MS));
}
@Override
@@ -85,17 +93,45 @@ public void initializeDriver(File configurationFile) throws IOException {
}
@Override
- @SuppressWarnings("unchecked")
- public List createTopics(TopicsInfo topicsInfo) throws IOException {
+ public List createTopics(TopicsInfo topicsInfo) throws IOException {
// Create all topics from a single worker node
- return (List) post(workers.get(0), "/create-topics", writer.writeValueAsBytes(topicsInfo), List.class)
- .join();
+ return (List) post(workers.get(0), "/create-topics", writer.writeValueAsBytes(topicsInfo),
+ new TypeReference>() {
+ }).join();
+ }
+
+ @Override
+ public void notifyTopicCreation(List topics) throws IOException {
+ List> futures = workers.stream().map(worker -> {
+ try {
+ return sendPost(worker, "/notify-topic-creation", writer.writeValueAsBytes(topics));
+ } catch (JsonProcessingException e) {
+ CompletableFuture future = new CompletableFuture<>();
+ future.completeExceptionally(e);
+ return future;
+ }
+ }).collect(toList());
+
+ FutureUtil.waitForAll(futures).join();
}
@Override
public void createProducers(List topics) {
- List> topicsPerProducer = ListPartition.partitionList(topics,
- producerWorkers.size());
+ // topics is a normalized list i.e. it accounts for duplicated entries in case
+ // of m topics and n producers where m < n. In this case, map the topics as is
+ // to honor the number of producers per topic configured for the workload
+ List> topicsPerProducer;
+ if (topics.size() <= producerWorkers.size()) {
+ topicsPerProducer = new ArrayList<>();
+ for (String topic : topics) {
+ List topicList = new ArrayList<>();
+ topicList.add(topic);
+ topicsPerProducer.add(topicList);
+ }
+ } else {
+ topicsPerProducer = ListPartition.partitionList(topics, producerWorkers.size());
+ }
+
Map> topicsPerProducerMap = Maps.newHashMap();
int i = 0;
for (List assignedTopics : topicsPerProducer) {
@@ -105,6 +141,8 @@ public void createProducers(List topics) {
// Number of actually used workers might be less than available workers
numberOfUsedProducerWorkers = i;
+ log.info("Number of producers configured for the topic: " + numberOfUsedProducerWorkers);
+
List> futures = topicsPerProducerMap.keySet().stream().map(producer -> {
try {
return sendPost(producer, "/create-producers",
@@ -155,9 +193,8 @@ public void resumeConsumers() throws IOException {
@Override
public void createConsumers(ConsumerAssignment overallConsumerAssignment) {
- List> subscriptionsPerConsumer = ListPartition.partitionList(
- overallConsumerAssignment.topicsSubscriptions,
- consumerWorkers.size());
+ List> subscriptionsPerConsumer = ListPartition
+ .partitionList(overallConsumerAssignment.topicsSubscriptions, consumerWorkers.size());
Map topicsPerWorkerMap = Maps.newHashMap();
int i = 0;
for (List tsl : subscriptionsPerConsumer) {
@@ -196,6 +233,9 @@ public PeriodStats getPeriodStats() {
stats.publishLatency.add(Histogram.decodeFromCompressedByteBuffer(
ByteBuffer.wrap(is.publishLatencyBytes), TimeUnit.SECONDS.toMicros(30)));
+ stats.publishDelayLatency.add(Histogram.decodeFromCompressedByteBuffer(
+ ByteBuffer.wrap(is.publishDelayLatencyBytes), TimeUnit.SECONDS.toMicros(30)));
+
stats.endToEndLatency.add(Histogram.decodeFromCompressedByteBuffer(
ByteBuffer.wrap(is.endToEndLatencyBytes), TimeUnit.HOURS.toMicros(12)));
} catch (ArrayIndexOutOfBoundsException | DataFormatException e) {
@@ -216,8 +256,16 @@ public CumulativeLatencies getCumulativeLatencies() {
stats.publishLatency.add(Histogram.decodeFromCompressedByteBuffer(
ByteBuffer.wrap(is.publishLatencyBytes), TimeUnit.SECONDS.toMicros(30)));
} catch (Exception e) {
- log.error("Failed to decode publish latency: {}",
- ByteBufUtil.prettyHexDump(Unpooled.wrappedBuffer(is.publishLatencyBytes)));
+ log.error("Failed to decode publish latency");
+ throw new RuntimeException(e);
+ }
+
+ try {
+ stats.publishDelayLatency.add(Histogram.decodeFromCompressedByteBuffer(
+ ByteBuffer.wrap(is.publishDelayLatencyBytes), TimeUnit.SECONDS.toMicros(30)));
+ } catch (Exception e) {
+ log.error("Failed to decode publish delay latency: {}",
+ ByteBufUtil.prettyHexDump(Unpooled.wrappedBuffer(is.publishDelayLatencyBytes)));
throw new RuntimeException(e);
}
@@ -263,9 +311,10 @@ private void sendPost(List hosts, String path, byte[] body) {
private CompletableFuture sendPost(String host, String path, byte[] body) {
return httpClient.preparePost(host + path).setBody(body).execute().toCompletableFuture().thenApply(x -> {
if (x.getStatusCode() != 200) {
- log.error("Failed to do HTTP post request to {}{} -- code: {}", host, path, x.getStatusCode());
+ log.error("Failed to do HTTP post request to {}{} -- code: {} error: {}", host, path, x.getStatusCode(),
+ x.getResponseBody());
}
- Preconditions.checkArgument(x.getStatusCode() == 200);
+ Preconditions.checkArgument(x.getStatusCode() == 200, "Status should be 200");
return (Void) null;
});
}
@@ -288,9 +337,10 @@ private CompletableFuture get(String host, String path, Class clazz) {
return httpClient.prepareGet(host + path).execute().toCompletableFuture().thenApply(response -> {
try {
if (response.getStatusCode() != 200) {
- log.error("Failed to do HTTP get request to {}{} -- code: {}", host, path, response.getStatusCode());
+ log.error("Failed to do HTTP get request to {}{} -- code: {}", host, path,
+ response.getStatusCode());
}
- Preconditions.checkArgument(response.getStatusCode() == 200);
+ Preconditions.checkArgument(response.getStatusCode() == 200, "Status should be 200");
return mapper.readValue(response.getResponseBody(), clazz);
} catch (IOException e) {
throw new RuntimeException(e);
@@ -298,14 +348,15 @@ private CompletableFuture get(String host, String path, Class clazz) {
});
}
- private CompletableFuture post(String host, String path, byte[] body, Class clazz) {
+ private CompletableFuture post(String host, String path, byte[] body, TypeReference type) {
return httpClient.preparePost(host + path).setBody(body).execute().toCompletableFuture().thenApply(response -> {
try {
if (response.getStatusCode() != 200) {
- log.error("Failed to do HTTP post request to {}{} -- code: {}", host, path, response.getStatusCode());
+ log.error("Failed to do HTTP post request to {}{} -- code: {}", host, path,
+ response.getStatusCode());
}
- Preconditions.checkArgument(response.getStatusCode() == 200);
- return mapper.readValue(response.getResponseBody(), clazz);
+ Preconditions.checkArgument(response.getStatusCode() == 200, "Status should be 200");
+ return mapper.readValue(response.getResponseBody(), type);
} catch (IOException e) {
throw new RuntimeException(e);
}
@@ -328,4 +379,14 @@ public void close() throws Exception {
private static final Logger log = LoggerFactory.getLogger(DistributedWorkersEnsemble.class);
+ @Override
+ public void pauseProducers() throws IOException {
+ sendPost(producerWorkers, "/pause-producers", new byte[0]);
+ }
+
+ @Override
+ public void resumeProducers() throws IOException {
+ sendPost(producerWorkers, "/resume-producers", new byte[0]);
+ }
+
}
diff --git a/benchmark-framework/src/main/java/io/openmessaging/benchmark/worker/LocalWorker.java b/benchmark-framework/src/main/java/io/openmessaging/benchmark/worker/LocalWorker.java
index a4de0364..b2d0afe4 100644
--- a/benchmark-framework/src/main/java/io/openmessaging/benchmark/worker/LocalWorker.java
+++ b/benchmark-framework/src/main/java/io/openmessaging/benchmark/worker/LocalWorker.java
@@ -18,10 +18,13 @@
*/
package io.openmessaging.benchmark.worker;
+import static io.openmessaging.benchmark.utils.UniformRateLimiter.*;
import static java.util.stream.Collectors.toList;
import java.io.File;
import java.io.IOException;
+import java.time.Instant;
+import java.time.temporal.ChronoUnit;
import java.util.ArrayList;
import java.util.List;
import java.util.Map;
@@ -30,10 +33,18 @@
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
+import java.util.concurrent.ThreadLocalRandom;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.LongAdder;
-import java.util.function.Function;
+import com.fasterxml.jackson.databind.DeserializationFeature;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.ObjectWriter;
+import com.fasterxml.jackson.dataformat.yaml.YAMLFactory;
+import com.google.common.base.Preconditions;
+import io.openmessaging.benchmark.utils.RateLimiter;
+
+import io.openmessaging.benchmark.utils.UniformRateLimiter;
import org.HdrHistogram.Recorder;
import org.apache.bookkeeper.stats.Counter;
import org.apache.bookkeeper.stats.NullStatsLogger;
@@ -42,13 +53,6 @@
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-import com.fasterxml.jackson.databind.DeserializationFeature;
-import com.fasterxml.jackson.databind.ObjectMapper;
-import com.fasterxml.jackson.databind.ObjectWriter;
-import com.fasterxml.jackson.dataformat.yaml.YAMLFactory;
-import com.google.common.base.Preconditions;
-import com.google.common.util.concurrent.RateLimiter;
-
import io.netty.util.concurrent.DefaultThreadFactory;
import io.openmessaging.benchmark.DriverConfiguration;
import io.openmessaging.benchmark.driver.BenchmarkConsumer;
@@ -72,7 +76,7 @@ public class LocalWorker implements Worker, ConsumerCallback {
private List producers = new ArrayList<>();
private List consumers = new ArrayList<>();
- private final RateLimiter rateLimiter = RateLimiter.create(1.0);
+ private volatile UniformRateLimiter rateLimiter = new UniformRateLimiter(1.0);
private final ExecutorService executor = Executors.newCachedThreadPool(new DefaultThreadFactory("local-worker"));
@@ -97,6 +101,10 @@ public class LocalWorker implements Worker, ConsumerCallback {
private final Recorder cumulativePublishLatencyRecorder = new Recorder(TimeUnit.SECONDS.toMicros(60), 5);
private final OpStatsLogger publishLatencyStats;
+ private final Recorder publishDelayLatencyRecorder = new Recorder(TimeUnit.SECONDS.toMicros(60), 5);
+ private final Recorder cumulativePublishDelayLatencyRecorder = new Recorder(TimeUnit.SECONDS.toMicros(60), 5);
+ private final OpStatsLogger publishDelayLatencyStats;
+
private final Recorder endToEndLatencyRecorder = new Recorder(TimeUnit.HOURS.toMicros(12), 5);
private final Recorder endToEndCumulativeLatencyRecorder = new Recorder(TimeUnit.HOURS.toMicros(12), 5);
private final OpStatsLogger endToEndLatencyStats;
@@ -105,6 +113,8 @@ public class LocalWorker implements Worker, ConsumerCallback {
private boolean consumersArePaused = false;
+ private boolean producersArePaused = false;
+
public LocalWorker() {
this(NullStatsLogger.INSTANCE);
}
@@ -115,6 +125,7 @@ public LocalWorker(StatsLogger statsLogger) {
StatsLogger producerStatsLogger = statsLogger.scope("producer");
this.messagesSentCounter = producerStatsLogger.getCounter("messages_sent");
this.bytesSentCounter = producerStatsLogger.getCounter("bytes_sent");
+ this.publishDelayLatencyStats = producerStatsLogger.getOpStatsLogger("producer_delay_latency");
this.publishLatencyStats = producerStatsLogger.getOpStatsLogger("produce_latency");
StatsLogger consumerStatsLogger = statsLogger.scope("consumer");
@@ -141,18 +152,19 @@ public void initializeDriver(File driverConfigFile) throws IOException {
}
@Override
- public List createTopics(TopicsInfo topicsInfo) {
+ public List createTopics(TopicsInfo topicsInfo) {
List> futures = new ArrayList<>();
Timer timer = new Timer();
String topicPrefix = benchmarkDriver.getTopicNamePrefix();
- List topics = new ArrayList<>();
+ List topics = new ArrayList<>();
for (int i = 0; i < topicsInfo.numberOfTopics; i++) {
- String topic = String.format("%s-%s-%04d", topicPrefix, RandomGenerator.getRandomString(), i);
+ Topic topic = new Topic(String.format("%s-%s-%04d", topicPrefix, RandomGenerator.getRandomString(), i),
+ topicsInfo.numberOfPartitionsPerTopic);
topics.add(topic);
- futures.add(benchmarkDriver.createTopic(topic, topicsInfo.numberOfPartitionsPerTopic));
+ futures.add(benchmarkDriver.createTopic(topic.name, topic.partitions));
}
futures.forEach(CompletableFuture::join);
@@ -161,6 +173,17 @@ public List createTopics(TopicsInfo topicsInfo) {
return topics;
}
+ @Override
+ public void notifyTopicCreation(List topics) {
+ List> futures = new ArrayList<>();
+
+ for (Topic topic : topics) {
+ futures.add(benchmarkDriver.notifyTopicCreation(topic.name, topic.partitions));
+ }
+
+ futures.forEach(CompletableFuture::join);
+ }
+
@Override
public void createProducers(List topics) {
Timer timer = new Timer();
@@ -177,7 +200,8 @@ public void createConsumers(ConsumerAssignment consumerAssignment) {
Timer timer = new Timer();
List> futures = consumerAssignment.topicsSubscriptions.stream()
- .map(ts -> benchmarkDriver.createConsumer(ts.topic, ts.subscription, this)).collect(toList());
+ .map(ts -> benchmarkDriver.createConsumer(ts.topic, ts.subscription, Optional.of(ts.partition), this))
+ .collect(toList());
futures.forEach(f -> consumers.add(f.join()));
log.info("Created {} consumers in {} ms", consumers.size(), timer.elapsedMillis());
@@ -187,7 +211,7 @@ public void createConsumers(ConsumerAssignment consumerAssignment) {
public void startLoad(ProducerWorkAssignment producerWorkAssignment) {
int processors = Runtime.getRuntime().availableProcessors();
- rateLimiter.setRate(producerWorkAssignment.publishRate);
+ rateLimiter = new UniformRateLimiter(producerWorkAssignment.publishRate);
Map> processorAssignemnt = new TreeMap<>();
@@ -211,13 +235,22 @@ public void probeProducers() throws IOException {
private void submitProducersToExecutor(List producers, KeyDistributor keyDistributor,
byte[] payloadData) {
executor.submit(() -> {
+ ThreadLocalRandom r = ThreadLocalRandom.current();
try {
while (!testCompleted) {
+ while (producersArePaused) {
+ try {
+ Thread.sleep(1000);
+ } catch (InterruptedException e) {
+ e.printStackTrace();
+ }
+ }
+
producers.forEach(producer -> {
- rateLimiter.acquire();
+ final long intendedSendTime = rateLimiter.acquire();
+ uninterruptibleSleepNs(intendedSendTime);
final long sendTime = System.nanoTime();
- producer.sendAsync(Optional.ofNullable(keyDistributor.next()), payloadData)
- .thenRun(() -> {
+ producer.sendAsync(Optional.ofNullable(keyDistributor.next()), payloadData).thenRun(() -> {
messagesSent.increment();
totalMessagesSent.increment();
messagesSentCounter.inc();
@@ -228,6 +261,11 @@ private void submitProducersToExecutor(List producers, KeyDis
publishLatencyRecorder.recordValue(latencyMicros);
cumulativePublishLatencyRecorder.recordValue(latencyMicros);
publishLatencyStats.registerSuccessfulEvent(latencyMicros, TimeUnit.MICROSECONDS);
+
+ final long sendDelayMicros = TimeUnit.NANOSECONDS.toMicros(sendTime - intendedSendTime);
+ publishDelayLatencyRecorder.recordValue(sendDelayMicros);
+ cumulativePublishDelayLatencyRecorder.recordValue(sendDelayMicros);
+ publishDelayLatencyStats.registerSuccessfulEvent(sendDelayMicros, TimeUnit.MICROSECONDS);
}).exceptionally(ex -> {
log.warn("Write error on message", ex);
return null;
@@ -243,10 +281,10 @@ private void submitProducersToExecutor(List producers, KeyDis
@Override
public void adjustPublishRate(double publishRate) {
if(publishRate < 1.0) {
- rateLimiter.setRate(1.0);
+ rateLimiter = new UniformRateLimiter(1.0);
return;
}
- rateLimiter.setRate(publishRate);
+ rateLimiter = new UniformRateLimiter(publishRate);
}
@Override
@@ -263,6 +301,7 @@ public PeriodStats getPeriodStats() {
stats.totalMessagesReceived = totalMessagesReceived.sum();
stats.publishLatency = publishLatencyRecorder.getIntervalHistogram();
+ stats.publishDelayLatency = publishDelayLatencyRecorder.getIntervalHistogram();
stats.endToEndLatency = endToEndLatencyRecorder.getIntervalHistogram();
return stats;
}
@@ -271,6 +310,7 @@ public PeriodStats getPeriodStats() {
public CumulativeLatencies getCumulativeLatencies() {
CumulativeLatencies latencies = new CumulativeLatencies();
latencies.publishLatency = cumulativePublishLatencyRecorder.getIntervalHistogram();
+ latencies.publishDelayLatency = cumulativePublishDelayLatencyRecorder.getIntervalHistogram();
latencies.endToEndLatency = endToEndCumulativeLatencyRecorder.getIntervalHistogram();
return latencies;
}
@@ -291,8 +331,11 @@ public void messageReceived(byte[] data, long publishTimestamp) {
bytesReceived.add(data.length);
bytesReceivedCounter.add(data.length);
- long now = System.currentTimeMillis();
- long endToEndLatencyMicros = TimeUnit.MILLISECONDS.toMicros(now - publishTimestamp);
+ // NOTE: PublishTimestamp is expected to be using the wall-clock time across
+ // machines
+ Instant currentTime = Instant.now();
+ long currentTimeNanos = TimeUnit.SECONDS.toNanos(currentTime.getEpochSecond()) + currentTime.getNano();
+ long endToEndLatencyMicros = TimeUnit.NANOSECONDS.toMicros(currentTimeNanos - publishTimestamp);
if (endToEndLatencyMicros > 0) {
endToEndCumulativeLatencyRecorder.recordValue(endToEndLatencyMicros);
endToEndLatencyRecorder.recordValue(endToEndLatencyMicros);
@@ -324,6 +367,8 @@ public void resumeConsumers() throws IOException {
public void resetStats() throws IOException {
publishLatencyRecorder.reset();
cumulativePublishLatencyRecorder.reset();
+ publishDelayLatencyRecorder.reset();
+ cumulativePublishDelayLatencyRecorder.reset();
endToEndLatencyRecorder.reset();
endToEndCumulativeLatencyRecorder.reset();
}
@@ -332,9 +377,12 @@ public void resetStats() throws IOException {
public void stopAll() throws IOException {
testCompleted = true;
consumersArePaused = false;
+ producersArePaused = false;
publishLatencyRecorder.reset();
cumulativePublishLatencyRecorder.reset();
+ publishDelayLatencyRecorder.reset();
+ cumulativePublishDelayLatencyRecorder.reset();
endToEndLatencyRecorder.reset();
endToEndCumulativeLatencyRecorder.reset();
@@ -382,4 +430,16 @@ public void close() throws Exception {
}
private static final Logger log = LoggerFactory.getLogger(LocalWorker.class);
+
+ @Override
+ public void pauseProducers() throws IOException {
+ producersArePaused = true;
+ log.info("Pausing producers");
+ }
+
+ @Override
+ public void resumeProducers() throws IOException {
+ producersArePaused = false;
+ log.info("Resuming producers");
+ }
}
diff --git a/benchmark-framework/src/main/java/io/openmessaging/benchmark/worker/Topic.java b/benchmark-framework/src/main/java/io/openmessaging/benchmark/worker/Topic.java
new file mode 100644
index 00000000..2cbe92f1
--- /dev/null
+++ b/benchmark-framework/src/main/java/io/openmessaging/benchmark/worker/Topic.java
@@ -0,0 +1,32 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package io.openmessaging.benchmark.worker;
+
+public class Topic {
+ public String name;
+ public int partitions;
+
+ public Topic() {
+ }
+
+ public Topic(String name, int partitions) {
+ this.name = name;
+ this.partitions = partitions;
+ }
+}
\ No newline at end of file
diff --git a/benchmark-framework/src/main/java/io/openmessaging/benchmark/worker/Worker.java b/benchmark-framework/src/main/java/io/openmessaging/benchmark/worker/Worker.java
index d7bcb96d..5c6edc51 100644
--- a/benchmark-framework/src/main/java/io/openmessaging/benchmark/worker/Worker.java
+++ b/benchmark-framework/src/main/java/io/openmessaging/benchmark/worker/Worker.java
@@ -33,7 +33,10 @@ public interface Worker extends AutoCloseable {
void initializeDriver(File configurationFile) throws IOException;
- List createTopics(TopicsInfo topicsInfo) throws IOException;
+ List createTopics(TopicsInfo topicsInfo) throws IOException;
+
+ // Let other workers know when a new topic(s) is created
+ void notifyTopicCreation(List topics) throws IOException;
void createProducers(List topics) throws IOException;
@@ -49,6 +52,10 @@ public interface Worker extends AutoCloseable {
void resumeConsumers() throws IOException;
+ void pauseProducers() throws IOException;
+
+ void resumeProducers() throws IOException;
+
CountersStats getCountersStats() throws IOException;
PeriodStats getPeriodStats() throws IOException;
diff --git a/benchmark-framework/src/main/java/io/openmessaging/benchmark/worker/WorkerHandler.java b/benchmark-framework/src/main/java/io/openmessaging/benchmark/worker/WorkerHandler.java
index d301b208..a2db2eb4 100644
--- a/benchmark-framework/src/main/java/io/openmessaging/benchmark/worker/WorkerHandler.java
+++ b/benchmark-framework/src/main/java/io/openmessaging/benchmark/worker/WorkerHandler.java
@@ -19,6 +19,7 @@
package io.openmessaging.benchmark.worker;
import java.io.File;
+import java.net.HttpURLConnection;
import java.nio.ByteBuffer;
import java.util.List;
@@ -26,6 +27,7 @@
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
+import com.fasterxml.jackson.core.type.TypeReference;
import com.fasterxml.jackson.databind.DeserializationFeature;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.fasterxml.jackson.databind.ObjectWriter;
@@ -49,11 +51,14 @@ public WorkerHandler(Javalin app, StatsLogger statsLogger) {
app.post("/initialize-driver", this::handleInitializeDriver);
app.post("/create-topics", this::handleCreateTopics);
+ app.post("/notify-topic-creation", this::handleNotifyTopicCreation);
app.post("/create-producers", this::handleCreateProducers);
app.post("/probe-producers", this::handleProbeProducers);
app.post("/create-consumers", this::handleCreateConsumers);
app.post("/pause-consumers", this::handlePauseConsumers);
app.post("/resume-consumers", this::handleResumeConsumers);
+ app.post("/pause-producers", this::handlePauseProducers);
+ app.post("/resume-producers", this::handleResumeProducers);
app.post("/start-load", this::handleStartLoad);
app.post("/adjust-publish-rate", this::handleAdjustPublishRate);
app.post("/stop-all", this::handleStopAll);
@@ -61,11 +66,16 @@ public WorkerHandler(Javalin app, StatsLogger statsLogger) {
app.get("/cumulative-latencies", this::handleCumulativeLatencies);
app.get("/counters-stats", this::handleCountersStats);
app.post("/reset-stats", this::handleResetStats);
+
+ app.exception(RuntimeException.class, (e, ctx) -> {
+ log.error("Request handler: {} - Exception: {}", ctx.path(), e.getMessage());
+ ctx.status(HttpURLConnection.HTTP_INTERNAL_ERROR);
+ });
}
private void handleInitializeDriver(Context ctx) throws Exception {
// Save config to temp file
- File tempFile = File.createTempFile("driver-configuration", "conf");
+ File tempFile = File.createTempFile("driver-configuration" + System.currentTimeMillis(), "conf");
Files.write(ctx.bodyAsBytes(), tempFile);
localWorker.initializeDriver(tempFile);
@@ -75,10 +85,17 @@ private void handleInitializeDriver(Context ctx) throws Exception {
private void handleCreateTopics(Context ctx) throws Exception {
TopicsInfo topicsInfo = mapper.readValue(ctx.body(), TopicsInfo.class);
log.info("Received create topics request for topics: {}", ctx.body());
- List topics = localWorker.createTopics(topicsInfo);
+ List topics = localWorker.createTopics(topicsInfo);
ctx.result(writer.writeValueAsString(topics));
}
+ private void handleNotifyTopicCreation(Context ctx) throws Exception {
+ List topics = mapper.readValue(ctx.body(), new TypeReference>() {
+ });
+ log.info("Received notify topic creation request for topics: {}", ctx.body());
+ localWorker.notifyTopicCreation(topics);
+ }
+
private void handleCreateProducers(Context ctx) throws Exception {
List topics = (List) mapper.readValue(ctx.body(), List.class);
log.info("Received create producers request for topics: {}", topics);
@@ -104,6 +121,14 @@ private void handleResumeConsumers(Context ctx) throws Exception {
localWorker.resumeConsumers();
}
+ private void handlePauseProducers(Context ctx) throws Exception {
+ localWorker.pauseProducers();
+ }
+
+ private void handleResumeProducers(Context ctx) throws Exception {
+ localWorker.resumeProducers();
+ }
+
private void handleStartLoad(Context ctx) throws Exception {
ProducerWorkAssignment producerWorkAssignment = mapper.readValue(ctx.body(), ProducerWorkAssignment.class);
@@ -135,6 +160,12 @@ private void handlePeriodStats(Context ctx) throws Exception {
histogramSerializationBuffer.flip();
histogramSerializationBuffer.get(stats.publishLatencyBytes);
+ histogramSerializationBuffer.clear();
+ stats.publishDelayLatency.encodeIntoCompressedByteBuffer(histogramSerializationBuffer);
+ stats.publishDelayLatencyBytes = new byte[histogramSerializationBuffer.position()];
+ histogramSerializationBuffer.flip();
+ histogramSerializationBuffer.get(stats.publishDelayLatencyBytes);
+
histogramSerializationBuffer.clear();
stats.endToEndLatency.encodeIntoCompressedByteBuffer(histogramSerializationBuffer);
stats.endToEndLatencyBytes = new byte[histogramSerializationBuffer.position()];
@@ -156,6 +187,12 @@ private void handleCumulativeLatencies(Context ctx) throws Exception {
histogramSerializationBuffer.flip();
histogramSerializationBuffer.get(stats.publishLatencyBytes);
+ histogramSerializationBuffer.clear();
+ stats.publishDelayLatency.encodeIntoCompressedByteBuffer(histogramSerializationBuffer);
+ stats.publishDelayLatencyBytes = new byte[histogramSerializationBuffer.position()];
+ histogramSerializationBuffer.flip();
+ histogramSerializationBuffer.get(stats.publishDelayLatencyBytes);
+
histogramSerializationBuffer.clear();
stats.endToEndLatency.encodeIntoCompressedByteBuffer(histogramSerializationBuffer);
stats.endToEndLatencyBytes = new byte[histogramSerializationBuffer.position()];
diff --git a/benchmark-framework/src/main/java/io/openmessaging/benchmark/worker/commands/CumulativeLatencies.java b/benchmark-framework/src/main/java/io/openmessaging/benchmark/worker/commands/CumulativeLatencies.java
index 7cff435f..cc5e5b6b 100644
--- a/benchmark-framework/src/main/java/io/openmessaging/benchmark/worker/commands/CumulativeLatencies.java
+++ b/benchmark-framework/src/main/java/io/openmessaging/benchmark/worker/commands/CumulativeLatencies.java
@@ -30,6 +30,10 @@ public class CumulativeLatencies {
public Histogram publishLatency = new Histogram(TimeUnit.SECONDS.toMicros(60), 5);
public byte[] publishLatencyBytes;
+ @JsonIgnore
+ public Histogram publishDelayLatency = new Histogram(TimeUnit.SECONDS.toMicros(60), 5);
+ public byte[] publishDelayLatencyBytes;
+
@JsonIgnore
public Histogram endToEndLatency = new Histogram(TimeUnit.HOURS.toMicros(12), 5);
public byte[] endToEndLatencyBytes;
diff --git a/benchmark-framework/src/main/java/io/openmessaging/benchmark/worker/commands/PeriodStats.java b/benchmark-framework/src/main/java/io/openmessaging/benchmark/worker/commands/PeriodStats.java
index f5b5704a..9ce4ff33 100644
--- a/benchmark-framework/src/main/java/io/openmessaging/benchmark/worker/commands/PeriodStats.java
+++ b/benchmark-framework/src/main/java/io/openmessaging/benchmark/worker/commands/PeriodStats.java
@@ -38,6 +38,11 @@ public class PeriodStats {
public Histogram publishLatency = new Histogram(TimeUnit.SECONDS.toMicros(60), 5);
public byte[] publishLatencyBytes;
+ @JsonIgnore
+ public Histogram publishDelayLatency = new Histogram(TimeUnit.SECONDS.toMicros(60), 5);
+ public byte[] publishDelayLatencyBytes;
+
+
@JsonIgnore
public Histogram endToEndLatency = new Histogram(TimeUnit.HOURS.toMicros(12), 5);
public byte[] endToEndLatencyBytes;
diff --git a/benchmark-framework/src/main/java/io/openmessaging/benchmark/worker/commands/TopicSubscription.java b/benchmark-framework/src/main/java/io/openmessaging/benchmark/worker/commands/TopicSubscription.java
index 8dc0d7d3..f673bb46 100644
--- a/benchmark-framework/src/main/java/io/openmessaging/benchmark/worker/commands/TopicSubscription.java
+++ b/benchmark-framework/src/main/java/io/openmessaging/benchmark/worker/commands/TopicSubscription.java
@@ -21,13 +21,15 @@
public class TopicSubscription {
public String topic;
public String subscription;
+ public int partition;
public TopicSubscription() {
}
- public TopicSubscription(String topic, String subscription) {
+ public TopicSubscription(String topic, String subscription, int partition) {
this.topic = topic;
this.subscription = subscription;
+ this.partition = partition;
}
}
diff --git a/benchmark-framework/src/main/java/io/openmessaging/benchmark/worker/commands/TopicsInfo.java b/benchmark-framework/src/main/java/io/openmessaging/benchmark/worker/commands/TopicsInfo.java
index 34bc968e..760f64de 100644
--- a/benchmark-framework/src/main/java/io/openmessaging/benchmark/worker/commands/TopicsInfo.java
+++ b/benchmark-framework/src/main/java/io/openmessaging/benchmark/worker/commands/TopicsInfo.java
@@ -29,4 +29,4 @@ public TopicsInfo(int numberOfTopics, int numberOfPartitionsPerTopic) {
this.numberOfTopics = numberOfTopics;
this.numberOfPartitionsPerTopic = numberOfPartitionsPerTopic;
}
-}
+}
\ No newline at end of file
diff --git a/bin/benchmark-worker b/bin/benchmark-worker
index 8cfaa0a3..8412eb40 100755
--- a/bin/benchmark-worker
+++ b/bin/benchmark-worker
@@ -24,7 +24,9 @@ else
CLASSPATH=benchmark-framework/target/classes:`cat benchmark-framework/target/classpath.txt`
fi
-JVM_MEM="-Xms4G -Xmx4G -XX:+UseG1GC"
+#KAFKA_JMX_OPTS="-Dcom.sun.management.jmxremote.port=9999 -Dcom.sun.management.jmxremote=true -Dcom.sun.management.jmxremote.authenticate=false -Dcom.sun.management.jmxremote.ssl=false -Djava.rmi.server.hostname=0.0.0.0 -Djava.net.preferIPv4Stack=true"
+JVM_MEM="-Xms4G -Xmx8G -XX:+UseG1GC"
+#KAFKA_OPTS="-javaagent:/opt/benchmark/jmx_prometheus_javaagent-0.13.0.jar=9090:/opt/benchmark/metrics.yml"
JVM_GC_LOG=" -XX:+PrintGCDetails -XX:+PrintGCApplicationStoppedTime -XX:+UseGCLogFileRotation -XX:NumberOfGCLogFiles=5 -XX:GCLogFileSize=64m -Xloggc:/dev/shm/benchmark-client-gc_%p.log"
-exec java -server -cp $CLASSPATH $JVM_MEM io.openmessaging.benchmark.worker.BenchmarkWorker $*
+exec java -server $KAFKA_JMX_OPTS -cp $CLASSPATH $JVM_MEM $KAFKA_OPTS io.openmessaging.benchmark.worker.BenchmarkWorker $*
diff --git a/bin/create_charts.py b/bin/create_charts.py
index f5af27fb..9e872f99 100755
--- a/bin/create_charts.py
+++ b/bin/create_charts.py
@@ -1,4 +1,4 @@
-#!/usr/bin/env python
+#!/usr/bin/env python3
#
# Licensed to the Apache Software Foundation (ASF) under one
# or more contributor license agreements. See the NOTICE file
@@ -43,22 +43,38 @@ def create_charts(test_results):
y_label='Latency (ms)',
time_series=[(x['driver'], x['publishLatency99pct']) for x in results])
+ create_chart(workload, 'Publish Delay latency 99pct',
+ y_label='Latency (us)',
+ time_series=[(x['driver'], x['publishDelayLatency99pct']) for x in results])
+
create_chart(workload, 'Publish rate',
y_label='Rate (msg/s)',
time_series=[(x['driver'], x['publishRate']) for x in results])
+ create_chart(workload, 'End To End Latency 95pct',
+ y_label='Latency (ms)',
+ time_series=[(x['driver'], x['endToEndLatency95pct']) for x in results])
+
create_chart(workload, 'Consume rate',
y_label='Rate (msg/s)',
time_series=[(x['driver'], x['consumeRate']) for x in results])
- create_chart(workload, 'endToEndLatencyAvg',
- y_label='LatencyAvg msec',
+ create_chart(workload, 'End To End Latency Avg',
+ y_label='Latency Avg (msec)',
time_series=[(x['driver'], x['endToEndLatencyAvg']) for x in results])
- create_quantile_chart(workload, 'Publish latency quantiles',
+ create_quantile_chart(workload, 'Publish Latency Quantiles',
y_label='Latency (ms)',
time_series=[(x['driver'], x['aggregatedPublishLatencyQuantiles']) for x in results])
+ create_quantile_chart(workload, 'Publish Delay Latency Quantiles',
+ y_label='Latency (us)',
+ time_series=[(x['driver'], x['aggregatedPublishDelayLatencyQuantiles']) for x in results])
+
+ create_quantile_chart(workload, 'End To End Latency Quantiles',
+ y_label='Latency (ms)',
+ time_series=[(x['driver'], x['aggregatedEndToEndLatencyQuantiles']) for x in results])
+
def create_chart(workload, title, y_label, time_series):
chart = pygal.XY(dots_size=.3,
diff --git a/bin/generate_charts.py b/bin/generate_charts.py
new file mode 100755
index 00000000..47a6589a
--- /dev/null
+++ b/bin/generate_charts.py
@@ -0,0 +1,423 @@
+#!/usr/bin/env python3
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements. See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership. The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied. See the License for the
+# specific language governing permissions and limitations
+# under the License.
+#
+
+import glob
+import json
+import math
+import argparse
+import sys
+
+import pygal
+from pygal.style import Style
+from itertools import chain
+from os import path
+from jinja2 import Template
+from collections import defaultdict
+
+graph_colors = ['#545454', # dark gray
+ '#e2401b', # redpanda red1
+ '#78909c', # light gray
+ '#ED7F66', # redpanda red2
+ '#0097A7', # blue
+ '#ffab40', # yellow
+ '#00ffff', # cyan
+ '#ff00ff'] # magenta
+chartStyle = Style(
+ background='transparent',
+ plot_background='transparent',
+ font_family='googlefont:Montserrat',
+ colors=graph_colors,
+ label_font_size=16,
+ legend_font_size=16,
+ major_label_font_size=16,
+)
+
+#theme = pygal.style.CleanStyle
+theme = chartStyle
+fill = False
+output = ''
+file_list = defaultdict(list)
+charts = defaultdict(list)
+coalesce = False
+
+
+def _clean_xy_values(values):
+ values = sorted((float(x), y) for x, y in values.items())
+ # do not restrict to any percentiles. show the max; the outliers
+ # are where the goodies lie
+ def _x_axis(x):
+ if x < 100.0: x = math.log10(100 / (100 - x))
+ # clamp
+ return min(x, 100.0)
+
+ xy_values = [(_x_axis(x), y) for x, y in values]
+ needle_idx = len(xy_values)-1
+ while needle_idx > 0:
+ x = round(xy_values[needle_idx][0],2)
+ if x < 100.00:
+ return xy_values[0:needle_idx]
+ needle_idx = needle_idx-1
+ return xy_values
+
+
+def create_quantile_chart(title, y_label, time_series):
+ def _fmt_val(x):
+ return 'p{:,.3f}'.format(100.0 - (100.0 / (10**x)))
+ chart = pygal.XY(style=theme,
+ dots_size=2,
+ legend_at_bottom=True,
+ truncate_legend=37,
+ x_value_formatter=_fmt_val,
+ show_dots=True,
+ fill=fill,
+ stroke_style={'width': 2},
+ show_y_guides=True,
+ show_x_guides=True)
+ # chart = pygal.XY()
+ chart.title = title
+ chart.human_readable = True
+ chart.y_title = y_label
+ chart.x_title = 'Percentile'
+ chart.x_labels = [0.31, 1, 3, 5]
+ chart.x_label_rotation=20
+ chart.x_labels_major_count=4
+ chart.show_minor_x_labels=False
+ chart.tooltip_border_radius=10
+ for label, values, opts in time_series:
+ xy_values = _clean_xy_values(values)
+ chart.add(label, xy_values, stroke_style=opts)
+
+ return chart.render(disable_xml_declaration=True)
+
+
+def create_multi_chart(title, y_label_1, y_label_2,
+ time_series):
+ chart = pygal.XY(style=theme,
+ dots_size=1,
+ show_dots=False,
+ stroke_style={'width': 2},
+ fill=fill,
+ legend_at_bottom=True,
+ show_x_guides=False,
+ show_y_guides=True)
+ chart.title = title
+ chart.human_readable = True
+ chart.x_title = 'Time (seconds)'
+
+ ys_1 = []
+ ys_2 = []
+
+ for label_1, values_1, label_2, values_2 in time_series:
+ ys_1.append(values_1)
+ chart.add(label_1, [(10 * x, y) for x, y in enumerate(values_1)])
+ chart.add(label_2, [(10 * x, y) for x, y in enumerate(values_2)],
+ secondary=True)
+
+ ys_1 = chain.from_iterable(ys_1)
+ ys_2 = chain.from_iterable(ys_2)
+ max_y_1 = float('-inf') # Hack for starting w/ INT64_MIN
+ max_y_2 = max_y_1
+ for y in ys_1:
+ if max_y_1 < y:
+ max_y_1 = y
+ chart.range = (0, max_y_1 * 1.20)
+ return chart.render(disable_xml_declaration=True)
+
+
+def create_bar_chart(title, y_label, x_label, data):
+ chart = pygal.Bar(style=theme,
+ dots_size=1,
+ show_dots=False,
+ stroke_style={'width': 2},
+ fill=fill,
+ show_legend=False,
+ show_x_guides=False,
+ show_y_guides=True)
+ chart.title = title
+ chart.x_labels = x_label
+ chart.value_formatter = lambda y: "{:,.0f}".format(y)
+
+ for label, points in data.items():
+ chart.add(label, points)
+
+ return chart.render(disable_xml_declaration=True)
+
+
+def create_chart(title, y_label, time_series):
+ chart = pygal.XY(style=theme,
+ dots_size=1,
+ show_dots=False,
+ stroke_style={'width': 2},
+ fill=fill,
+ legend_at_bottom=True,
+ show_x_guides=False,
+ show_y_guides=True)
+ chart.title = title
+
+ chart.human_readable = True
+ chart.y_title = y_label
+ chart.x_title = 'Time (seconds)'
+
+ ys = []
+ for label, values in time_series:
+ ys.append(values)
+ chart.add(label, [(10 * x, y) for x, y in enumerate(values)])
+ ys = chain.from_iterable(ys)
+ max_y = float('-inf') # Hack for starting w/ INT64_MIN
+ for y in ys:
+ if max_y < y:
+ max_y = y
+ chart.range = (max_y * 0.0, max_y * 1.20)
+
+ return chart.render(disable_xml_declaration=True)
+
+
+def generate_charts(files):
+ workloads = {}
+
+ # Charts are labeled based on benchmark names, we need them
+ # to be unique. A combination of (driver, workload) defines
+ # a unique benchmark and name is "{driver}-{workload}".
+ benchmark_names = set()
+ for file in sorted(files):
+ data = json.load(open(file))
+ data['workload'] = data['workload'].replace('/', '-')
+ begin_time = data['beginTime'] if 'beginTime' in data.keys() else ""
+ rp_version = data['version'] if 'version' in data.keys() and data['version'] else "unknown_version"
+ unique_name = "{ver}-{ts}-{driver}-{workload}".format(ver=rp_version, ts=begin_time, driver=data['driver'], workload=data['workload'])
+ # name used as chart label.
+ name = "{ver}-{driver}-{workload}".format(ver=rp_version, driver=data['driver'], workload=data['workload'])
+ if unique_name in benchmark_names:
+ print(f"WARN: Duplicate benchmark found: {name} in file {file}", file=sys.stderr)
+
+ benchmark_names.add(unique_name)
+
+ if coalesce:
+ workload = 'All Workloads'
+ else:
+ workload = data['workload']
+
+ benchmark_names.add(name)
+ data['name'] = name
+
+ if workload in workloads:
+ workloads[workload].append(data)
+ else:
+ workloads[workload] = [data]
+
+ for workload in workloads:
+ stats_pub_rate = []
+ stats_con_rate = []
+ stats_backlog = []
+ stats_lat_p99 = []
+ stats_lat_p999 = []
+ stats_lat_p9999 = []
+ stat_lat_avg = []
+ stat_lat_max = []
+ stat_lat_quantile = []
+ stat_e2e_lat_quantile = []
+ drivers = []
+
+ pub_rate_avg = {}
+ pub_rate_avg["Throughput (MB/s): higher is better"] = []
+
+ # Aggregate across all runs
+ count = 0
+ curated_metrics = {} # to dump to stdout
+ metrics_of_interest = [
+ "version",
+ "beginTime",
+ "endTime",
+ "aggregatedPublishLatencyAvg",
+ "aggregatedPublishLatency50pct",
+ "aggregatedPublishLatency99pct",
+ "aggregatedPublishLatencyMax",
+ "aggregatedPublishDelayLatencyAvg",
+ "aggregatedPublishDelayLatency50pct",
+ "aggregatedPublishDelayLatency99pct",
+ "aggregatedEndToEndLatencyAvg",
+ "aggregatedEndToEndLatency50pct",
+ "aggregatedEndToEndLatency99pct",
+ "aggregatedEndToEndLatency9999pct",
+ "aggregatedEndToEndLatencyMax",
+ ]
+
+ for data in workloads[workload]:
+ metrics = dict()
+ curated_metrics[data['name']] = metrics
+ stats_pub_rate.append(data['publishRate'])
+ stats_con_rate.append(data['consumeRate'])
+ stats_backlog.append(data['backlog'])
+ stats_lat_p99.append(data['publishLatency99pct'])
+ stats_lat_p999.append(data['publishLatency999pct'])
+ stats_lat_p9999.append(data['publishLatency9999pct'])
+ stat_lat_avg.append(data['publishLatencyAvg'])
+ stat_lat_max.append(data['publishLatencyMax'])
+
+ stat_lat_quantile.append(data['aggregatedPublishLatencyQuantiles'])
+ stat_e2e_lat_quantile.append(
+ data['aggregatedEndToEndLatencyQuantiles'])
+ drivers.append(data['name'])
+ throughput = (sum(data['publishRate']) / len(data['publishRate']) * 1024) / (1024.0 * 1024.0)
+ pub_rate_avg["Throughput (MB/s): higher is better"].append({
+ 'value': throughput,
+ 'color': graph_colors[count % len(graph_colors)]
+ })
+ count = count + 1
+ for metric_key in metrics_of_interest:
+ if metric_key in data.keys():
+ metrics[metric_key] = data[metric_key]
+ metrics["throughputMBps"] = throughput
+
+ # OMB tooling depends on the output of this script, do not print extra stuff to stdout unless
+ # you fully understand what you are doing.
+ print(json.dumps(curated_metrics, indent=2))
+
+ # Parse plot options
+ opts = []
+ if args.opts is None:
+ for driver in drivers:
+ opts.append({})
+ else:
+ for opt in args.opts:
+ if opt == 'Dashed':
+ opts.append({'width': 2, 'dasharray': '3, 6, 12, 24'})
+ else:
+ opts.append({})
+
+ # Generate publish rate bar-chart
+ charts[workload] = [create_bar_chart('Throughput (MB/s): higher is better', 'MB/s', drivers,
+ pub_rate_avg)]
+
+ # Generate latency quantiles
+ time_series = zip(drivers, stat_lat_quantile, opts)
+ charts[workload].append(create_quantile_chart('Publish Latency Percentiles: lower is better',
+ y_label='Latency (ms)',
+ time_series=time_series))
+
+ time_series = zip(drivers, stat_e2e_lat_quantile, opts)
+ charts[workload].append(create_quantile_chart('End-to-End Latency Percentiles: lower is better',
+ y_label='Latency (ms)',
+ time_series=time_series))
+
+ # Generate p99 latency time-series
+ time_series = zip(drivers, stats_lat_p99)
+ charts[workload].append(create_chart('Publish Latency - 99th Percentile: lower is better',
+ y_label='Latency (ms)',
+ time_series=time_series))
+
+ # Generate avg E2E latency time-series
+ time_series = zip(drivers, stat_lat_avg)
+ charts[workload].append(create_chart('End-to-end Latency - Average: lower is better',
+ y_label='Latency (ms)',
+ time_series=time_series))
+
+ # Generate publish rate
+ time_series = zip(drivers, stats_pub_rate)
+ charts[workload].append(create_chart('Publish Rate: higher is better',
+ y_label='Message/s',
+ time_series=time_series))
+
+ # Generate consume + backlog rate
+ labels_con = []
+ labels_backlog = []
+
+ for x in drivers:
+ labels_con.append(x + " - Consume Rate")
+ labels_backlog.append(x + " - Backlog")
+
+ time_series = zip(labels_con, stats_con_rate, labels_backlog,
+ stats_backlog)
+ charts[workload].append(create_multi_chart(
+ 'Consume rate (Messages/s - Left) w/ Backlog (No. of Messages - Right)',
+ 'Consume - Messages/s', 'Backlog - Messages', time_series))
+
+
+if __name__ == "__main__":
+ parser = argparse.ArgumentParser(
+ description='Plot Kafka OpenMessaging Benchmark results')
+
+ parser.add_argument(
+ '--results',
+ dest='results_dir',
+ required=True,
+ help='Directory containing the results for both Redpanda and Kafka')
+ parser.add_argument('--series-opts',
+ nargs='+',
+ dest='opts',
+ required=False,
+ type=str,
+ help='Options for each series: Dashed or Filled')
+ parser.add_argument('--output',
+ dest='output',
+ required=False,
+ type=str,
+ help='Location where all output will be stored')
+ parser.add_argument('--coalesce-workloads',
+ dest='coalesce',
+ action='store_true',
+ help='Specify put all workloads on a single set of charts')
+
+ args = parser.parse_args()
+
+ prefixes = {}
+
+ if args.output != '':
+ output = path.join(args.output, '')
+
+ coalesce = args.coalesce
+
+ # Recursively fetch all json files in the results dir.
+ filelist = glob.iglob(path.join(args.results_dir, "**/*.json"), recursive=True)
+
+ generate_charts(filelist)
+
+ html = '''
+
+
+
+
+
+
+
+
{{title}}
+ {% for workload in charts %}
+
+
{{workload}}
+ {% for chart in charts[workload] %}
+ {{chart}}
+ {% endfor %}
+
+ {% endfor %}
+
+
+
+ '''
+
+ template = Template(html)
+
+ index_html = template.render(charts=charts, title="Charts")
+
+ f = open(f"{output}index.html", "w")
+ f.write(index_html)
+ f.close()
diff --git a/bin/parse.py b/bin/parse.py
new file mode 100755
index 00000000..45914940
--- /dev/null
+++ b/bin/parse.py
@@ -0,0 +1,291 @@
+#!/usr/bin/env python3
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements. See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership. The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied. See the License for the
+# specific language governing permissions and limitations
+# under the License.
+#
+
+import json
+import sys
+import statistics
+import numpy as np
+import argparse
+import re
+import pygal
+from pygal.style import Style
+from itertools import chain
+from os import walk
+from os import path
+
+chartStyle = Style(
+ background='transparent',
+ plot_background='transparent',
+ font_family='googlefont:Montserrat',
+ # colors=('#D8365D', '#78365D')
+ colors=('#66CC69', '#173361', '#D8365D'),
+ # colors=('#66CC69', '#667C69', '#173361', '#D8365D', '#78365D'),
+)
+
+#theme = pygal.style.CleanStyle
+theme = chartStyle
+fill = False
+
+
+def create_quantile_chart(workload, title, y_label, time_series):
+ import math
+ chart = pygal.XY(style=theme, dots_size=0.5,
+ legend_at_bottom=True,
+ truncate_legend=37,
+ x_value_formatter=lambda x: '{:,.2f} %'.format(
+ 100.0 - (100.0 / (10**x))),
+ show_dots=False, fill=fill,
+ stroke_style={'width': 2},
+ show_y_guides=False, show_x_guides=True)
+ chart.title = title
+ # chart.stroke = False
+
+ chart.human_readable = True
+ chart.y_title = y_label
+ chart.x_title = 'Percentile'
+ chart.x_labels = [0.30103, 1, 2, 3]
+
+ for label, values, opts in time_series:
+ values = sorted((float(x), y) for x, y in values.items())
+ xy_values = [(math.log10(100 / (100 - x)), y)
+ for x, y in values if x <= 99.9]
+ chart.add(label, xy_values, stroke_style=opts)
+
+ chart.render_to_file('%s.svg' % workload)
+
+
+def create_multi_chart(svg_file_name, title, y_label_1, y_label_2, time_series):
+ chart = pygal.XY(style=theme, dots_size=1, show_dots=False, stroke_style={'width': 2}, fill=fill,
+ legend_at_bottom=True, show_x_guides=False, show_y_guides=False)
+ chart.title = title
+ chart.human_readable = True
+ chart.x_title = 'Time (seconds)'
+
+ ys_1 = []
+ ys_2 = []
+
+ for label_1, values_1, label_2, values_2 in time_series:
+ ys_1.append(values_1)
+ chart.add(label_1, [(10*x, y) for x, y in enumerate(values_1)])
+ chart.add(label_2, [(10*x, y)
+ for x, y in enumerate(values_2)], secondary=True)
+
+ ys_1 = chain.from_iterable(ys_1)
+ ys_2 = chain.from_iterable(ys_2)
+ max_y_1 = float('-inf') # Hack for starting w/ INT64_MIN
+ max_y_2 = max_y_1
+ for y in ys_1:
+ if max_y_1 < y:
+ max_y_1 = y
+ chart.range = (0, max_y_1 * 1.20)
+ chart.render_to_file('%s.svg' % svg_file_name)
+
+
+def create_bar_chart(workload, title, y_label, x_label, data):
+ chart = pygal.Bar(
+ style=theme, dots_size=1, show_dots=False, stroke_style={'width': 2}, fill=fill,
+ show_legend=False, show_x_guides=False, show_y_guides=True
+ )
+ chart.title = title
+ chart.x_labels = x_label
+ chart.value_formatter = lambda y: "{:,.0f}".format(y)
+
+ for label, points in data.items():
+ chart.add(label, points)
+ chart.render_to_file('%s.svg' % workload)
+
+
+def create_chart(workload, title, y_label, time_series):
+ chart = pygal.XY(style=theme, dots_size=1, show_dots=False, stroke_style={'width': 2}, fill=fill,
+ legend_at_bottom=True, show_x_guides=False, show_y_guides=False)
+ chart.title = title
+
+ chart.human_readable = True
+ chart.y_title = y_label
+ chart.x_title = 'Time (seconds)'
+ # line_chart.x_labels = [str(10 * x) for x in range(len(time_series[0][1]))]
+
+ ys = []
+ for label, values in time_series:
+ ys.append(values)
+ chart.add(label, [(10*x, y) for x, y in enumerate(values)])
+ ys = chain.from_iterable(ys)
+ max_y = float('-inf') # Hack for starting w/ INT64_MIN
+ for y in ys:
+ if max_y < y:
+ max_y = y
+ chart.range = (max_y * 0.0, max_y * 1.20)
+ chart.render_to_file('%s.svg' % workload)
+
+
+if __name__ == "__main__":
+ parser = argparse.ArgumentParser(
+ description='Plot Kafka OpenMessaging Benchmark results')
+
+ parser.add_argument('--acks', dest='ack', help='ACK level')
+ parser.add_argument('--message-size', dest='msg_size',
+ help='Message size (100b or 1kb)')
+ parser.add_argument('--results', dest='results_dir',
+ help='Directory containing the results')
+ parser.add_argument('--durability', dest='durability',
+ help='Either \'fsync\' or \'nofsync\'')
+ parser.add_argument('--files', nargs='+', dest='files', required=False, type=str,
+ help='Explicitly specify result files to plot')
+ parser.add_argument('--series-labels', nargs='+', dest='labels', required=False,
+ type=str, help='Labels for the series instead of file names on the plot')
+ parser.add_argument('--series-opts', nargs='+', dest='opts', required=False,
+ type=str, help='Options for each series: Dashed or Filled')
+ args = parser.parse_args()
+
+ if args.labels != None and len(args.labels) != len(args.files):
+ sys.exit('Number of labels specified != Number of input files')
+
+ aggregate = []
+
+ if args.files is None:
+ file_name_template = f'{args.msg_size}-run-[0123456789]+-{args.durability}-{args.ack}-acks.json'
+
+ # Get list of directories
+ for (dirpath, dirnames, filenames) in walk(args.results_dir):
+ for file in filenames:
+ if re.match(file_name_template, file):
+ file_path = path.join(dirpath, file)
+ data = json.load(open(file_path))
+ data['file'] = file
+ aggregate.append(data)
+ else:
+ for idx, file in enumerate(args.files):
+ data = json.load(open(file))
+ if args.labels != None:
+ data['file'] = args.labels[idx]
+ else:
+ data['file'] = file
+ aggregate.append(data)
+
+ stats_pub_rate = []
+ stats_con_rate = []
+ stats_backlog = []
+ stats_lat_p99 = []
+ stats_lat_p999 = []
+ stats_lat_p9999 = []
+ stat_lat_avg = []
+ stat_lat_max = []
+ stat_lat_quantile = []
+ stat_e2e_lat_quantile = []
+ drivers = []
+
+ pub_rate_avg = {}
+ pub_rate_avg["Throughput (MB/s)"] = []
+
+ # colors = ['#66CC69', '#173361', '#D8365D']
+ colors = ['#66CC69', '#667C69', '#173361', '#D8365D', '#78365D']
+
+ # Aggregate across all runs
+ count = 0
+ for data in aggregate:
+ stats_pub_rate.append(data['publishRate'])
+ stats_con_rate.append(data['consumeRate'])
+ stats_backlog.append(data['backlog'])
+ stats_lat_p99.append(data['publishLatency99pct'])
+ stats_lat_p999.append(data['publishLatency999pct'])
+ stats_lat_p9999.append(data['publishLatency9999pct'])
+ stat_lat_avg.append(data['publishLatencyAvg'])
+ stat_lat_max.append(data['publishLatencyMax'])
+
+ stat_lat_quantile.append(data['aggregatedPublishLatencyQuantiles'])
+ stat_e2e_lat_quantile.append(
+ data['aggregatedEndToEndLatencyQuantiles'])
+ drivers.append(data['file'])
+
+ # if (count >= len(aggregate)/2):
+ # pub_rate_avg[args.barlabels[1]].append(
+ # sum(data['publishRate'])/len(data['publishRate']))
+ # else:
+ # pub_rate_avg[args.barlabels[0]].append(
+ # sum(data['publishRate'])/len(data['publishRate']))
+ pub_rate_avg["Throughput (MB/s)"].append(
+ {
+ 'value': (sum(data['publishRate'])/len(data['publishRate']) * 1024)/(1024.0 * 1024.0),
+ 'color': colors[count]
+ })
+ count = count + 1
+
+ # Parse plot options
+ opts = []
+ if args.opts is None:
+ for driver in drivers:
+ opts.append({})
+ else:
+ for opt in args.opts:
+ if opt == 'Dashed':
+ opts.append({'width': 2, 'dasharray': '3, 6, 12, 24'})
+ else:
+ opts.append({})
+
+ # Generate publish rate bar-chart
+ svg = f'{args.msg_size}-{args.durability}-{args.ack}-acks-publish-rate-bar'
+ print(pub_rate_avg)
+ create_bar_chart(svg, 'Throughput (MB/s)', 'MB/s',
+ drivers, pub_rate_avg)
+
+ # Generate latency quantiles
+ time_series = zip(drivers, stat_lat_quantile, opts)
+ svg = f'{args.msg_size}-{args.durability}-{args.ack}-acks-latency-quantile'
+ create_quantile_chart(svg, 'Publish Latency Quantiles',
+ y_label='Latency (ms)',
+ time_series=time_series)
+
+ time_series = zip(drivers, stat_e2e_lat_quantile, opts)
+ svg = f'{args.msg_size}-{args.durability}-{args.ack}-acks-e2e-latency-quantile'
+ create_quantile_chart(svg, 'End-to-End Latency Quantiles',
+ y_label='Latency (ms)',
+ time_series=time_series)
+
+ # Generate p99 latency time-series
+ svg = f'{args.msg_size}-{args.durability}-{args.ack}-acks-publish-latency-p99'
+ time_series = zip(drivers, stats_lat_p99)
+ create_chart(svg, 'Publish Latency - 99th Percentile',
+ y_label='Latency (ms)', time_series=time_series)
+
+ # Generate avg E2E latency time-series
+ svg = f'{args.msg_size}-{args.durability}-{args.ack}-acks-e2e-latency-avg'
+ time_series = zip(drivers, stat_lat_avg)
+ create_chart(svg, 'End-to-end Latency - Average',
+ y_label='Latency (ms)', time_series=time_series)
+
+ # Generate publish rate
+ svg = f'{args.msg_size}-{args.durability}-{args.ack}-acks-publish-rate'
+ time_series = zip(drivers, stats_pub_rate)
+ create_chart(svg, 'Publish Rate',
+ y_label='Message/s', time_series=time_series)
+
+ # Generate consume + backlog rate
+ svg = f'{args.msg_size}-{args.durability}-{args.ack}-acks-consume-rate-backlog'
+ labels_con = []
+ labels_backlog = []
+
+ for x in drivers:
+ labels_con.append(x + " - Consume Rate")
+ labels_backlog.append(x + " - Backlog")
+
+ time_series = zip(labels_con, stats_con_rate,
+ labels_backlog, stats_backlog)
+ create_multi_chart(svg, 'Consume rate (Messages/s - Left) w/ Backlog (No. of Messages - Right)',
+ 'Consume - Messages/s', 'Backlog - Messages', time_series)
diff --git a/bin/requirements.txt b/bin/requirements.txt
new file mode 100644
index 00000000..6fab6808
--- /dev/null
+++ b/bin/requirements.txt
@@ -0,0 +1,3 @@
+numpy
+jinja2
+pygal
diff --git a/bin/workers.yaml b/bin/workers.yaml
new file mode 100644
index 00000000..c6c95f1b
--- /dev/null
+++ b/bin/workers.yaml
@@ -0,0 +1,22 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements. See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership. The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied. See the License for the
+# specific language governing permissions and limitations
+# under the License.
+#
+
+workers:
+ - http://localhost:8080
+ - http://localhost:8082
diff --git a/docker/pom.xml b/docker/pom.xml
index 3b72c526..08bb8960 100644
--- a/docker/pom.xml
+++ b/docker/pom.xml
@@ -34,7 +34,7 @@
1.3.7
- openmessaging
+ vectorized
diff --git a/driver-api/src/main/java/io/openmessaging/benchmark/driver/BenchmarkDriver.java b/driver-api/src/main/java/io/openmessaging/benchmark/driver/BenchmarkDriver.java
index 45f20ff1..09b1ed29 100644
--- a/driver-api/src/main/java/io/openmessaging/benchmark/driver/BenchmarkDriver.java
+++ b/driver-api/src/main/java/io/openmessaging/benchmark/driver/BenchmarkDriver.java
@@ -20,6 +20,7 @@
import java.io.File;
import java.io.IOException;
+import java.util.Optional;
import java.util.concurrent.CompletableFuture;
import org.apache.bookkeeper.stats.StatsLogger;
@@ -28,13 +29,14 @@
*/
public interface BenchmarkDriver extends AutoCloseable {
/**
- * Driver implementation can use this method to initialize the client libraries, with the provided configuration
- * file.
+ * Driver implementation can use this method to initialize the client libraries,
+ * with the provided configuration file.
*
- * The format of the configuration file is specific to the driver implementation.
+ * The format of the configuration file is specific to the driver
+ * implementation.
*
* @param configurationFile
- * @param statsLogger stats logger to collect stats from benchmark driver
+ * @param statsLogger stats logger to collect stats from benchmark driver
* @throws IOException
*/
void initialize(File configurationFile, StatsLogger statsLogger) throws IOException;
@@ -49,24 +51,28 @@ public interface BenchmarkDriver extends AutoCloseable {
*/
CompletableFuture createTopic(String topic, int partitions);
+ /**
+ * Notification of new topic creation with the given number of partitions
+ */
+ CompletableFuture notifyTopicCreation(String topic, int partitions);
+
/**
* Create a producer for a given topic
*/
CompletableFuture createProducer(String topic);
/**
- * Create a benchmark consumer relative to one particular topic and subscription.
+ * Create a benchmark consumer relative to one particular topic and
+ * subscription.
*
- * It is responsibility of the driver implementation to invoke the consumerCallback each time a message
- * is received.
+ * It is responsibility of the driver implementation to invoke the
+ * consumerCallback each time a message is received.
*
* @param topic
* @param subscriptionName
* @param consumerCallback
* @return
*/
- CompletableFuture createConsumer(
- String topic,
- String subscriptionName,
- ConsumerCallback consumerCallback);
+ CompletableFuture createConsumer(String topic, String subscriptionName,
+ Optional partition, ConsumerCallback consumerCallback);
}
diff --git a/driver-artemis/src/main/java/io/openmessaging/benchmark/driver/artemis/ArtemisBenchmarkDriver.java b/driver-artemis/src/main/java/io/openmessaging/benchmark/driver/artemis/ArtemisBenchmarkDriver.java
index f547109b..33845bdd 100644
--- a/driver-artemis/src/main/java/io/openmessaging/benchmark/driver/artemis/ArtemisBenchmarkDriver.java
+++ b/driver-artemis/src/main/java/io/openmessaging/benchmark/driver/artemis/ArtemisBenchmarkDriver.java
@@ -20,6 +20,7 @@
import java.io.File;
import java.io.IOException;
+import java.util.Optional;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.ForkJoinPool;
@@ -57,7 +58,7 @@ public void initialize(File configurationFile, StatsLogger statsLogger) throws I
try {
ServerLocator serverLocator = ActiveMQClient.createServerLocator(config.brokerAddress);
serverLocator.setConfirmationWindowSize(1000);
-
+
sessionFactory = serverLocator.createSessionFactory();
session = sessionFactory.createSession();
} catch (Exception e) {
@@ -90,6 +91,12 @@ public CompletableFuture createTopic(String topic, int partitions) {
return future;
}
+ @Override
+ public CompletableFuture notifyTopicCreation(String topic, int partitions) {
+ // No-op
+ return CompletableFuture.completedFuture(null);
+ }
+
@Override
public CompletableFuture createProducer(String topic) {
try {
@@ -103,7 +110,7 @@ public CompletableFuture createProducer(String topic) {
@Override
public CompletableFuture createConsumer(String topic, String subscriptionName,
- ConsumerCallback consumerCallback) {
+ Optional partition, ConsumerCallback consumerCallback) {
CompletableFuture future = new CompletableFuture<>();
ForkJoinPool.commonPool().submit(() -> {
try {
diff --git a/driver-artemis/src/main/java/io/openmessaging/benchmark/driver/artemis/ArtemisBenchmarkProducer.java b/driver-artemis/src/main/java/io/openmessaging/benchmark/driver/artemis/ArtemisBenchmarkProducer.java
index 1db656d5..458f5805 100644
--- a/driver-artemis/src/main/java/io/openmessaging/benchmark/driver/artemis/ArtemisBenchmarkProducer.java
+++ b/driver-artemis/src/main/java/io/openmessaging/benchmark/driver/artemis/ArtemisBenchmarkProducer.java
@@ -18,8 +18,10 @@
*/
package io.openmessaging.benchmark.driver.artemis;
+import java.time.Instant;
import java.util.Optional;
import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.TimeUnit;
import org.apache.activemq.artemis.api.core.ActiveMQException;
import org.apache.activemq.artemis.api.core.client.ClientMessage;
@@ -49,7 +51,9 @@ public void close() throws Exception {
@Override
public CompletableFuture sendAsync(Optional key, byte[] payload) {
ClientMessage msg = session.createMessage(true /* durable */ );
- msg.setTimestamp(System.currentTimeMillis());
+ Instant currentTime = Instant.now();
+ long currentTimeNanos = TimeUnit.SECONDS.toNanos(currentTime.getEpochSecond()) + currentTime.getNano();
+ msg.setTimestamp(currentTimeNanos);
msg.getBodyBuffer().writeBytes(payload);
CompletableFuture future = new CompletableFuture<>();
diff --git a/driver-bookkeeper/src/main/java/io/openmessaging/benchmark/driver/bookkeeper/DlogBenchmarkDriver.java b/driver-bookkeeper/src/main/java/io/openmessaging/benchmark/driver/bookkeeper/DlogBenchmarkDriver.java
index 2734eca6..390d1f03 100644
--- a/driver-bookkeeper/src/main/java/io/openmessaging/benchmark/driver/bookkeeper/DlogBenchmarkDriver.java
+++ b/driver-bookkeeper/src/main/java/io/openmessaging/benchmark/driver/bookkeeper/DlogBenchmarkDriver.java
@@ -32,6 +32,7 @@
import java.io.IOException;
import java.io.StringReader;
import java.net.URI;
+import java.util.Optional;
import java.util.concurrent.CompletableFuture;
import org.apache.bookkeeper.stats.StatsLogger;
import org.apache.commons.configuration.ConfigurationException;
@@ -50,7 +51,7 @@ public class DlogBenchmarkDriver implements BenchmarkDriver {
private static final Logger log = LoggerFactory.getLogger(DlogBenchmarkProducer.class);
private static final ObjectMapper mapper = new ObjectMapper(new YAMLFactory())
- .configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false);
+ .configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false);
private Config config;
private Namespace namespace;
@@ -70,14 +71,10 @@ public void initialize(File configurationFile, StatsLogger statsLogger) throws I
}
URI dlogUri = URI.create(config.dlogUri);
- dlshade.org.apache.bookkeeper.stats.StatsLogger dlStatsLogger =
- new CachingStatsLogger(new StatsLoggerAdaptor(statsLogger.scope("dlog")));
+ dlshade.org.apache.bookkeeper.stats.StatsLogger dlStatsLogger = new CachingStatsLogger(
+ new StatsLoggerAdaptor(statsLogger.scope("dlog")));
- namespace = NamespaceBuilder.newBuilder()
- .conf(conf)
- .uri(dlogUri)
- .statsLogger(dlStatsLogger)
- .build();
+ namespace = NamespaceBuilder.newBuilder().conf(conf).uri(dlogUri).statsLogger(dlStatsLogger).build();
log.info("Initialized distributedlog namespace at {}", dlogUri);
}
@@ -103,13 +100,18 @@ public CompletableFuture createTopic(String topic, int partitions) {
}
log.info("Successfully create topic {} with {} partitions", topic, partitions);
} catch (IOException ioe) {
- log.error("Failed to create topic {} with {} partitions",
- topic, partitions, ioe);
+ log.error("Failed to create topic {} with {} partitions", topic, partitions, ioe);
throw new RuntimeException(ioe);
}
});
}
+ @Override
+ public CompletableFuture notifyTopicCreation(String topic, int partitions) {
+ // No-op
+ return CompletableFuture.completedFuture(null);
+ }
+
@Override
public CompletableFuture createProducer(String topic) {
return CompletableFuture.supplyAsync(() -> {
@@ -120,16 +122,12 @@ public CompletableFuture createProducer(String topic) {
} catch (IOException ioe) {
throw new RuntimeException(ioe);
}
- })
- .thenCompose(dlm -> dlm.openAsyncLogWriter())
- .thenApply(writer -> new DlogBenchmarkProducer(writer));
+ }).thenCompose(dlm -> dlm.openAsyncLogWriter()).thenApply(writer -> new DlogBenchmarkProducer(writer));
}
@Override
- public CompletableFuture createConsumer(
- String topic,
- String subscriptionName,
- ConsumerCallback consumerCallback) {
+ public CompletableFuture createConsumer(String topic, String subscriptionName,
+ Optional partition, ConsumerCallback consumerCallback) {
return CompletableFuture.supplyAsync(() -> {
try {
DistributedLogManager dlm = namespace.openLog(topic);
@@ -138,8 +136,7 @@ public CompletableFuture createConsumer(
} catch (IOException ioe) {
throw new RuntimeException(ioe);
}
- })
- .thenApply(dlm -> new DlogBenchmarkConsumer(dlm, consumerCallback));
+ }).thenApply(dlm -> new DlogBenchmarkConsumer(dlm, consumerCallback));
}
@Override
diff --git a/driver-kafka/deploy/deploy.yaml b/driver-kafka/deploy/deploy.yaml
index 8c7e50ea..4418b3c3 100644
--- a/driver-kafka/deploy/deploy.yaml
+++ b/driver-kafka/deploy/deploy.yaml
@@ -17,6 +17,56 @@
# under the License.
#
+- name: Collect profiles
+ hosts: kafka, client
+ become: true
+ tags:
+ - ["never", "profile"]
+ tasks:
+ - set_fact:
+ time: "{{ ansible_date_time.epoch }}"
+ - name: Generate Flame graph
+ tags:
+ - flame
+ shell: |
+ /tmp/profiler.sh -d 120 `pidof java` -f /tmp/flame.svg
+ - name: Fetch flame graph
+ tags:
+ - flame
+ fetch:
+ src: /tmp/flame.svg
+ dest: results/{{ group_names[0] }}/{{ time }}/{{ inventory_hostname }}/flame.svg
+ flat: yes
+
+ # - name: Dump & compress heap
+ # tags:
+ # - heap
+ # shell: |
+ # jattach `pidof java` dumpheap /tmp/heap
+ # chmod 777 /tmp/heap
+ # tar -cvzf /tmp/heap.tar /tmp/heap
+ # - name: Fetch heap dump
+ # tags:
+ # - heap
+ # fetch:
+ # src: /tmp/heap.tar
+ # dest: results/{{ group_names[0] }}/{{ time }}/{{ inventory_hostname }}/heap.tar
+ # flat: yes
+
+ - name: Heap allocation profile
+ tags:
+ - heap
+ shell: |
+ /tmp/profiler.sh -d 60 -e alloc -o svg -f /tmp/heap.svg 1 `pidof java`
+ chmod 777 /tmp/heap.svg
+ - name: Fetch heap dump
+ tags:
+ - heap
+ fetch:
+ src: /tmp/heap.svg
+ dest: results/{{ group_names[0] }}/{{ time }}/{{ inventory_hostname }}/heap.svg
+ flat: yes
+
- name: Format and mount disks for Kafka hosts
hosts: kafka
connection: ssh
@@ -26,11 +76,11 @@
tuned-adm profile latency-performance
- name: Format disks
filesystem:
- fstype: xfs
- dev: '{{ item }}'
+ fstype: xfs
+ dev: "{{ item }}"
with_items:
- - '/dev/nvme0n1'
- - '/dev/nvme1n1'
+ - "/dev/nvme1n1"
+ - "/dev/nvme2n1"
- name: Mount disks
mount:
path: "{{ item.path }}"
@@ -39,42 +89,98 @@
opts: defaults,noatime,nodiscard
state: mounted
with_items:
- - { path: "/mnt/data-1", src: "/dev/nvme0n1" }
- - { path: "/mnt/data-2", src: "/dev/nvme1n1" }
+ - { path: "/mnt/data-1", src: "/dev/nvme1n1" }
+ - { path: "/mnt/data-2", src: "/dev/nvme2n1" }
+
+- name: Store server lists
+ hosts: all
+ connection: ssh
+ tasks:
+ - set_fact:
+ zookeeperServers: "{{ groups['zookeeper'] | map('extract', hostvars, ['ansible_default_ipv4', 'address']) | map('regex_replace', '^(.*)$', '\\1:2181') | join(',') }}"
+ boostrapServers: "{{ groups['kafka'] | map('extract', hostvars, ['private_ip']) | map('regex_replace', '^(.*)$', '\\1:9092') | join(',') }}"
-- name: Kafka setup
+- name: Setup common packages
hosts: all
connection: ssh
become: true
tasks:
+ - name: Remove sysstat
+ yum: pkg={{ item }} state=absent
+ with_items:
+ - sysstat
- name: Install RPM packages
yum: pkg={{ item }} state=latest
with_items:
- - wget
- - java
- - sysstat
- - vim
- - chrony
+ - wget
+ - java-11
+ - sysstat
+ - vim
+ - chrony
+ - git
+ - gcc
+ - make
+
+- name: Kafka installation
+ hosts: kafka, zookeeper
+ connection: ssh
+ become: true
+ tasks:
- file: path=/opt/kafka state=absent
- file: path=/opt/kafka state=directory
- set_fact:
zookeeperServers: "{{ groups['zookeeper'] | map('extract', hostvars, ['ansible_default_ipv4', 'address']) | map('regex_replace', '^(.*)$', '\\1:2181') | join(',') }}"
boostrapServers: "{{ groups['kafka'] | map('extract', hostvars, ['private_ip']) | map('regex_replace', '^(.*)$', '\\1:9092') | join(',') }}"
- kafkaVersion: "2.3.0"
+ kafkaVersion: "2.7.0"
- debug:
msg: "zookeeper servers: {{ zookeeperServers }}\nboostrap servers: {{ boostrapServers }}"
+
+ #################### Release ####################
- name: Download Kafka package
unarchive:
- src: http://mirrors.ocf.berkeley.edu/apache/kafka/{{ kafkaVersion }}/kafka_2.11-{{ kafkaVersion }}.tgz
+ src: http://mirrors.ocf.berkeley.edu/apache/kafka/{{ kafkaVersion }}/kafka_2.13-{{ kafkaVersion }}.tgz
remote_src: yes
dest: /opt/kafka
extra_opts: ["--strip-components=1"]
+
+ #################### Staging ####################
+ # # https://access.redhat.com/articles/5117881
+ # - name: Fix Sectigo Root CA expiration issue
+ # shell: |
+ # trust dump --filter "pkcs11:id=%ad%bd%98%7a%34%b4%26%f7%fa%c4%26%54%ef%03%bd%e0%24%cb%54%1a;type=cert" > /etc/pki/ca-trust/source/blacklist/addtrust-external-root.p11-kit
+ # update-ca-trust
+ # - name: Download Kafka package
+ # unarchive:
+ # src: https://home.apache.org/~rhauch/kafka-2.6.0-rc2/kafka_2.13-2.6.0.tgz
+ # remote_src: yes
+ # dest: /opt/kafka
+ # extra_opts: ["--strip-components=1"]
+
+ #################### Custom build ####################
+ # - name: Copy Kafka
+ # unarchive:
+ # src: /Users/anikhil/offsets_patch/core/build/distributions/kafka_2.13-2.7.0-SNAPSHOT.tgz
+ # dest: /opt/kafka
+ # extra_opts: ["--strip-components=1"]
+
- set_fact:
privateIp: "{{ hostvars[inventory_hostname]['ansible_default_ipv4']['address'] }}"
- name: Change locale to en_US.utf-8
shell: |
echo 'LANG=en_US.utf-8
LC_ALL=en_US.utf-8' > /etc/environment
+ - name: Setup JMX for Prometheus
+ copy:
+ src: "monitoring/jmx_exporter/jmx_prometheus_javaagent-0.13.0.jar"
+ dest: "/opt/kafka/jmx_prometheus_javaagent-0.13.0.jar"
+ - name: Configure JMX for Prometheus (Kafka)
+ template:
+ src: "monitoring/jmx_exporter/metrics.yml"
+ dest: "/opt/kafka/metrics.yml"
+ - name: Configure JMX for Prometheus (Zookeeper)
+ template:
+ src: "monitoring/jmx_exporter/zookeeper.yml"
+ dest: "/opt/kafka/zookeeper.yml"
- name: Setup ZooKeeper
hosts: zookeeper
@@ -104,9 +210,13 @@
hosts: kafka
connection: ssh
become: true
+ tags:
+ - configure
tasks:
- set_fact:
brokerId: "{{ groups['kafka'].index(inventory_hostname) }}"
+ privateIp: "{{ hostvars[inventory_hostname]['ansible_default_ipv4']['address'] }}"
+ zookeeperServers: "{{ groups['zookeeper'] | map('extract', hostvars, ['ansible_default_ipv4', 'address']) | map('regex_replace', '^(.*)$', '\\1:2181') | join(',') }}"
- name: Set up broker
template:
src: "templates/server.properties"
@@ -118,6 +228,15 @@
state: restarted
daemon_reload: yes
name: "kafka"
+ - git:
+ repo: "https://github.com/sysstat/sysstat"
+ dest: "/opt/sysstat"
+ - name: Compile and install sysstat from source
+ shell: |
+ cd /opt/sysstat
+ ./configure
+ make -j`nproc`
+ sudo make install
- name: Chrony setup
hosts: client
@@ -152,20 +271,28 @@
- name: Configure URL
lineinfile:
- dest: '{{ item }}'
- regexp: '^ bootstrap.servers='
- line: ' bootstrap.servers={{ boostrapServers }}'
- with_items: '{{ drivers_list.stdout_lines }}'
+ dest: "{{ item }}"
+ regexp: "^ bootstrap.servers="
+ line: " bootstrap.servers={{ boostrapServers }}"
+ with_items: "{{ drivers_list.stdout_lines }}"
- name: Configure memory
lineinfile:
- dest: /opt/benchmark/bin/benchmark-worker
- regexp: '^JVM_MEM='
- line: 'JVM_MEM="-Xms12G -Xmx12G -XX:+UseG1GC -XX:MaxGCPauseMillis=10 -XX:+ParallelRefProcEnabled -XX:+UnlockExperimentalVMOptions -XX:+AggressiveOpts -XX:+DoEscapeAnalysis -XX:ParallelGCThreads=32 -XX:ConcGCThreads=32 -XX:G1NewSizePercent=50 -XX:+DisableExplicitGC -XX:-ResizePLAB -XX:+PerfDisableSharedMem -XX:+AlwaysPreTouch -XX:-UseBiasedLocking"'
+ dest: /opt/benchmark/bin/benchmark-worker
+ regexp: "^JVM_MEM="
+ line: 'JVM_MEM="-Xms16G -Xmx16G -XX:+UseG1GC -XX:MaxGCPauseMillis=10 -XX:+ParallelRefProcEnabled -XX:+UnlockExperimentalVMOptions -XX:+AggressiveOpts -XX:+DoEscapeAnalysis -XX:ParallelGCThreads=32 -XX:ConcGCThreads=32 -XX:G1NewSizePercent=50 -XX:+DisableExplicitGC -XX:-ResizePLAB -XX:+PerfDisableSharedMem -XX:+AlwaysPreTouch -XX:-UseBiasedLocking"'
- name: Configure memory
lineinfile:
- dest: /opt/benchmark/bin/benchmark
- regexp: '^JVM_MEM='
- line: 'JVM_MEM="-Xmx1G"'
+ dest: /opt/benchmark/bin/benchmark
+ regexp: "^JVM_MEM="
+ line: 'JVM_MEM="-Xmx1G"'
+ - name: Setup JMX for Producer/Consumer clients
+ copy:
+ src: "monitoring/jmx_exporter/jmx_prometheus_javaagent-0.13.0.jar"
+ dest: "/opt/benchmark/jmx_prometheus_javaagent-0.13.0.jar"
+ - name: Configure JMX for Prometheus (Client)
+ template:
+ src: "monitoring/jmx_exporter/client.yml"
+ dest: "/opt/benchmark/metrics.yml"
- template:
src: "templates/workers.yaml"
dest: "/opt/benchmark/workers.yaml"
@@ -173,12 +300,15 @@
template:
src: "templates/benchmark-worker.service"
dest: "/etc/systemd/system/benchmark-worker.service"
- - systemd:
+ - name: "Restart benchmark worker"
+ tags:
+ - configure
+ systemd:
state: restarted
daemon_reload: yes
name: "benchmark-worker"
-- name: Hosts addresses
+- name: Hosts addresses
hosts: localhost
become: false
tasks:
@@ -191,3 +321,184 @@
- debug:
msg: "Benchmark clients {{ item }}"
with_items: "{{ groups['client'] }}"
+
+# Install the monitoring stack
+- name: Install Node Exporter
+ hosts: kafka, client
+ roles:
+ - cloudalchemy.node-exporter
+
+- hosts: prometheus
+ roles:
+ - cloudalchemy.prometheus
+ vars:
+ prometheus_scrape_configs:
+ - job_name: "broker"
+ static_configs:
+ - targets: "{{ groups['kafka'] | map('extract', hostvars, ['ansible_default_ipv4','address']) | map('regex_replace', '^(.*)$','\\1:8080') | list }}"
+ - job_name: "node"
+ static_configs:
+ - targets: "{{ groups['kafka'] | map('extract', hostvars, ['ansible_default_ipv4','address']) | map('regex_replace', '^(.*)$','\\1:9100') | list }}"
+ - targets: "{{ groups['client'] | map('extract', hostvars, ['ansible_default_ipv4','address']) | map('regex_replace', '^(.*)$','\\1:9100') | list }}"
+ - job_name: "client - jmx"
+ static_configs:
+ - targets: "{{ groups['client'] | map('extract', hostvars, ['ansible_default_ipv4', 'address']) | map('regex_replace', '^(.*)$', '\\1:9090') | list }}"
+ - job_name: "client"
+ static_configs:
+ - targets: "{{ groups['client'] | map('extract', hostvars, ['ansible_default_ipv4', 'address']) | map('regex_replace', '^(.*)$', '\\1:8081') | list }}"
+
+#- name: jmx export setup
+# hosts: prometheus
+# connection: ssh
+# become: true
+# tasks:
+# - template:
+# src: "templates/kafka-dashboard.service"
+# dest: "/etc/systemd/system/kafka-dashboard.service"
+# - systemd:
+# state: restarted
+# daemon_reload: yes
+# name: "kafka-dashboard"
+
+- hosts: prometheus
+ roles:
+ - cloudalchemy.grafana
+ vars:
+ grafana_version: 6.7.3
+ grafana_security:
+ admin_user: admin
+ admin_password: enter_your_secure_password
+ grafana_datasources:
+ - name: prometheus
+ type: prometheus
+ access: proxy
+ url: 'http://localhost:9090'
+ basicAuth: false
+ grafana_dashboards:
+ - dashboard_id: 1860
+ revision_id: 21
+ datasource: prometheus
+
+# Setup the Prometheus/Grafana monitoring stack
+#- name: Prometheus installation
+# hosts: prometheus
+# connection: ssh
+# become: true
+# tasks:
+# - name: Add Extras Repo
+# shell: yum-config-manager --enable rhui-REGION-rhel-server-extras
+# - name: Install RPM packages
+# yum: pkg={{ item }} state=latest
+# with_items:
+# - docker
+# - set_fact:
+# prometheusVersion: "2.19.3"
+# - file: path=/opt/prometheus state=absent
+# - file: path=/opt/prometheus state=directory
+# - name: Download Prometheus Binary Package
+# unarchive:
+# src: "https://github.com/prometheus/prometheus/releases/download/v{{ prometheusVersion }}/prometheus-{{ prometheusVersion }}.linux-amd64.tar.gz"
+# remote_src: yes
+# dest: /opt/prometheus
+# extra_opts: ["--strip-components=1"]
+# - set_fact:
+# private_ip: "{{ hostvars[inventory_hostname]['ansible_default_ipv4']['address'] }}"
+#
+#- name: Prometheus setup
+# hosts: prometheus
+# connection: ssh
+# become: true
+# tasks:
+# - file:
+# path: "/opt/prometheus/{{ item }}"
+# state: directory
+# with_items:
+# - data
+# - template:
+# src: "templates/prometheus.service"
+# dest: "/etc/systemd/system/prometheus.service"
+# - template:
+# src: "templates/kafka-dashboard.service"
+# dest: "/etc/systemd/system/kafka-dashboard.service"
+# - template:
+# src: "templates/prometheus.yml"
+# dest: "/opt/prometheus/prometheus.yml"
+# - systemd:
+# state: restarted
+# daemon_reload: yes
+# name: "kafka-dashboard"
+#
+#- name: Install Node exporter on Brokers to collect system metrics
+# hosts: kafka
+# become: true
+# tasks:
+# - name: Add user node_exporter
+# user:
+# name: node_exporter
+# shell: /bin/false
+# system: true
+# create_home: no
+#
+# - name: Download and extract
+# unarchive:
+# src: https://github.com/prometheus/node_exporter/releases/download/v1.0.1/node_exporter-1.0.1.linux-amd64.tar.gz
+# dest: /tmp
+# remote_src: yes
+#
+# - name: Copy bin node_exporter to /usr/local/bin
+# copy:
+# src: /tmp/node_exporter-1.0.1.linux-amd64/node_exporter
+# remote_src: yes
+# dest: /usr/local/bin/
+# owner: node_exporter
+# group: node_exporter
+# mode: u+x,g+x,o+x
+#
+# - name: Create service node_exporter.service
+# blockinfile:
+# path: /etc/systemd/system/node_exporter.service
+# block: |
+# [Unit]
+# Description=Prometheus Node Exporter
+# Wants=network-online.target
+# After=network-online.target
+# [Service]
+# User=node_exporter
+# Group=node_exporter
+# Type=simple
+# ExecStart=/usr/local/bin/node_exporter
+# [Install]
+# WantedBy=multi-user.target
+# create: true
+#
+# - name: systemctl daemon_reload
+# systemd:
+# daemon_reload: yes
+#
+# - name: Start and Enable node_exporter
+# service:
+# name: node_exporter
+# state: started
+# enabled: yes
+#
+# - name: Check whether port 9100 is available
+# wait_for:
+# port: 9100
+# state: started
+# timeout: 5
+
+- name: Install jattach and async-profiler
+ hosts: all
+ become: true
+ tasks:
+ - name: Download and extract async-profiler
+ unarchive:
+ src: https://github.com/jvm-profiling-tools/async-profiler/releases/download/v1.7.1/async-profiler-1.7.1-linux-x64.tar.gz
+ dest: /tmp
+ remote_src: yes
+
+ - name: Download and install jattach
+ get_url:
+ url: https://github.com/apangin/jattach/releases/download/v1.5/jattach
+ dest: /usr/bin/jattach
+ mode: "0755"
diff --git a/driver-kafka/deploy/monitoring/Dockerfile b/driver-kafka/deploy/monitoring/Dockerfile
new file mode 100644
index 00000000..0ae4874d
--- /dev/null
+++ b/driver-kafka/deploy/monitoring/Dockerfile
@@ -0,0 +1,54 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements. See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership. The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied. See the License for the
+# specific language governing permissions and limitations
+# under the License.
+#
+
+FROM grafana/grafana:7.0.5
+
+USER root
+
+# Install the Grafana Image Rendering Plugin
+ENV GF_PATHS_PLUGINS="/var/lib/grafana-plugins"
+RUN mkdir -p "$GF_PATHS_PLUGINS" && \
+ chown -R grafana:grafana "$GF_PATHS_PLUGINS"
+RUN echo "http://dl-cdn.alpinelinux.org/alpine/edge/community" >> /etc/apk/repositories && \
+ echo "http://dl-cdn.alpinelinux.org/alpine/edge/main" >> /etc/apk/repositories && \
+ echo "http://dl-cdn.alpinelinux.org/alpine/edge/testing" >> /etc/apk/repositories && \
+ apk --no-cache upgrade && \
+ apk add --no-cache udev ttf-opensans chromium && \
+ rm -rf /tmp/* && \
+ rm -rf /usr/share/grafana/tools/phantomjs;
+USER grafana
+ENV GF_RENDERER_PLUGIN_CHROME_BIN="/usr/bin/chromium-browser"
+RUN grafana-cli \
+ --pluginsDir "$GF_PATHS_PLUGINS" \
+ --pluginUrl https://github.com/grafana/grafana-image-renderer/releases/latest/download/plugin-linux-x64-glibc-no-chromium.zip \
+ plugins install grafana-image-renderer;
+
+USER root
+COPY dashboards.yaml /etc/grafana/provisioning/dashboards/dashboards.yaml
+RUN apk add curl
+USER grafana
+COPY grafana.ini /etc/grafana/grafana.ini
+COPY dashboards/* /var/lib/grafana/dashboards/
+COPY start.sh /start.sh
+
+EXPOSE 3000
+
+ENV PROMETHEUS_URL http://prometheus:9090
+
+ENTRYPOINT ["/start.sh"]
diff --git a/driver-kafka/deploy/monitoring/README.md b/driver-kafka/deploy/monitoring/README.md
new file mode 100644
index 00000000..1db30933
--- /dev/null
+++ b/driver-kafka/deploy/monitoring/README.md
@@ -0,0 +1,3 @@
+# Kafka dashboard for the Open Messaging Benchmark using the Prometheus/Grafana stack
+
+- Builds a docker image
diff --git a/driver-kafka/deploy/monitoring/dashboards.yaml b/driver-kafka/deploy/monitoring/dashboards.yaml
new file mode 100644
index 00000000..4a9c9c4a
--- /dev/null
+++ b/driver-kafka/deploy/monitoring/dashboards.yaml
@@ -0,0 +1,45 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements. See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership. The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied. See the License for the
+# specific language governing permissions and limitations
+# under the License.
+#
+
+apiVersion: 1
+
+providers:
+ # an unique provider name. Required
+ - name: "Default Dashboards"
+ # Org id. Default to 1
+ orgId: 1
+ # name of the dashboard folder.
+ folder: ""
+ # folder UID. will be automatically generated if not specified
+ folderUid: ""
+ # provider type. Default to 'file'
+ type: file
+ # disable dashboard deletion
+ disableDeletion: true
+ # enable dashboard editing
+ editable: true
+ # how often Grafana will scan for changed dashboards
+ updateIntervalSeconds: 10
+ # allow updating provisioned dashboards from the UI
+ allowUiUpdates: true
+ options:
+ # path to dashboard files on disk. Required when using the 'file' type
+ path: /var/lib/grafana/dashboards
+ # use folder names from filesystem to create folders in Grafana
+ foldersFromFilesStructure: true
diff --git a/driver-kafka/deploy/monitoring/dashboards/jvm.json b/driver-kafka/deploy/monitoring/dashboards/jvm.json
new file mode 100644
index 00000000..8d5ee19b
--- /dev/null
+++ b/driver-kafka/deploy/monitoring/dashboards/jvm.json
@@ -0,0 +1,2557 @@
+{
+ "annotations": {
+ "list": []
+ },
+ "editable": true,
+ "gnetId": null,
+ "graphTooltip": 0,
+ "hideControls": false,
+ "id": 2,
+ "links": [],
+ "refresh": "1m",
+ "rows": [
+ {
+ "collapse": false,
+ "height": 221,
+ "panels": [
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": null,
+ "decimals": 1,
+ "fill": 1,
+ "id": 4,
+ "legend": {
+ "avg": false,
+ "current": false,
+ "max": false,
+ "min": false,
+ "show": false,
+ "total": false,
+ "values": false
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "nullPointMode": "null",
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "scopedVars": {
+ "instance": {
+ "selected": false,
+ "text": "bookie-232tg",
+ "value": "bookie-232tg"
+ }
+ },
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "span": 3,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "irate(process_cpu_seconds_total{cluster=\"$cluster\", kubernetes_pod_name=~\"$instance\"}[30s]) * 100\n\n",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "{{kubernetes_pod_name}}",
+ "metric": "process_cpu_seconds_total",
+ "refId": "A",
+ "step": 20
+ }
+ ],
+ "thresholds": [
+ {
+ "colorMode": "ok",
+ "fill": false,
+ "line": true,
+ "op": "gt",
+ "value": 100
+ }
+ ],
+ "timeFrom": null,
+ "timeShift": null,
+ "title": "CPU",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "percent",
+ "label": "",
+ "logBase": 1,
+ "max": null,
+ "min": "0",
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ }
+ ]
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "Prometheus",
+ "fill": 1,
+ "id": 1,
+ "legend": {
+ "alignAsTable": false,
+ "avg": false,
+ "current": false,
+ "hideEmpty": false,
+ "max": false,
+ "min": false,
+ "rightSide": false,
+ "show": false,
+ "total": false,
+ "values": false
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "nullPointMode": "null",
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "scopedVars": {
+ "instance": {
+ "selected": false,
+ "text": "bookie-232tg",
+ "value": "bookie-232tg"
+ }
+ },
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "span": 3,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "sum(jvm_memory_bytes_committed{cluster=\"$cluster\", kubernetes_pod_name=~\"$instance\"})",
+ "format": "time_series",
+ "hide": false,
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "Heap size",
+ "metric": "jvm_memory_bytes_committed",
+ "refId": "A",
+ "step": 20
+ },
+ {
+ "expr": "sum(jvm_memory_bytes_used{cluster=\"$cluster\", kubernetes_pod_name=~\"$instance\"})",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "Used",
+ "metric": "jvm_memory_bytes_used",
+ "refId": "B",
+ "step": 20
+ },
+ {
+ "expr": "sum(jvm_memory_bytes_max{cluster=\"$cluster\", kubernetes_pod_name=~\"$instance\"})",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "Max",
+ "metric": "jvm_memory_bytes_max",
+ "refId": "C",
+ "step": 20
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeShift": null,
+ "title": "Heap Memory",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "decbytes",
+ "label": "",
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ]
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": null,
+ "fill": 1,
+ "id": 2,
+ "legend": {
+ "avg": false,
+ "current": false,
+ "max": false,
+ "min": false,
+ "show": false,
+ "total": false,
+ "values": false
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "nullPointMode": "null",
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "scopedVars": {
+ "instance": {
+ "selected": false,
+ "text": "bookie-232tg",
+ "value": "bookie-232tg"
+ }
+ },
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "span": 3,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "jvm_memory_direct_bytes_used{cluster=\"$cluster\", kubernetes_pod_name=~\"$instance\"}",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "Used",
+ "metric": "jvm_memory_direct_bytes_used",
+ "refId": "A",
+ "step": 20
+ },
+ {
+ "expr": "jvm_memory_direct_bytes_max{cluster=\"$cluster\", kubernetes_pod_name=~\"$instance\"}",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "Max",
+ "metric": "jvm_memory_direct_bytes_max",
+ "refId": "B",
+ "step": 20
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeShift": null,
+ "title": "Direct Memory",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "decbytes",
+ "label": "",
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": "",
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ }
+ ]
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": null,
+ "fill": 1,
+ "id": 3,
+ "legend": {
+ "avg": false,
+ "current": false,
+ "max": false,
+ "min": false,
+ "show": false,
+ "total": false,
+ "values": false
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "nullPointMode": "null",
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "scopedVars": {
+ "instance": {
+ "selected": false,
+ "text": "bookie-232tg",
+ "value": "bookie-232tg"
+ }
+ },
+ "seriesOverrides": [
+ {
+ "alias": "old count",
+ "yaxis": 2
+ },
+ {
+ "alias": "young count",
+ "yaxis": 2
+ }
+ ],
+ "spaceLength": 10,
+ "span": 3,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "sum(increase(jvm_gc_collection_seconds_sum{cluster=\"$cluster\", kubernetes_pod_name=~\"$instance\"}[30s]))",
+ "format": "time_series",
+ "hide": false,
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "Pause",
+ "metric": "jvm_gc_collection_seconds_count",
+ "refId": "B",
+ "step": 20
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeShift": null,
+ "title": "GC Pauses",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "s",
+ "label": "",
+ "logBase": 1,
+ "max": null,
+ "min": "0",
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": "",
+ "logBase": 1,
+ "max": null,
+ "min": "0",
+ "show": false
+ }
+ ]
+ }
+ ],
+ "repeat": "instance",
+ "repeatIteration": null,
+ "repeatRowId": null,
+ "showTitle": true,
+ "title": "$component $instance",
+ "titleSize": "h4"
+ },
+ {
+ "collapse": false,
+ "height": 221,
+ "panels": [
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": null,
+ "decimals": 1,
+ "fill": 1,
+ "id": 5,
+ "legend": {
+ "avg": false,
+ "current": false,
+ "max": false,
+ "min": false,
+ "show": false,
+ "total": false,
+ "values": false
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "nullPointMode": "null",
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "scopedVars": {
+ "instance": {
+ "selected": false,
+ "text": "bookie-44jrt",
+ "value": "bookie-44jrt"
+ }
+ },
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "span": 3,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "irate(process_cpu_seconds_total{cluster=\"$cluster\", kubernetes_pod_name=~\"$instance\"}[30s]) * 100\n\n",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "{{kubernetes_pod_name}}",
+ "metric": "process_cpu_seconds_total",
+ "refId": "A",
+ "step": 20
+ }
+ ],
+ "thresholds": [
+ {
+ "colorMode": "ok",
+ "fill": false,
+ "line": true,
+ "op": "gt",
+ "value": 100
+ }
+ ],
+ "timeFrom": null,
+ "timeShift": null,
+ "title": "CPU",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "percent",
+ "label": "",
+ "logBase": 1,
+ "max": null,
+ "min": "0",
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ }
+ ]
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "Prometheus",
+ "fill": 1,
+ "id": 6,
+ "legend": {
+ "alignAsTable": false,
+ "avg": false,
+ "current": false,
+ "hideEmpty": false,
+ "max": false,
+ "min": false,
+ "rightSide": false,
+ "show": false,
+ "total": false,
+ "values": false
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "nullPointMode": "null",
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "scopedVars": {
+ "instance": {
+ "selected": false,
+ "text": "bookie-44jrt",
+ "value": "bookie-44jrt"
+ }
+ },
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "span": 3,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "sum(jvm_memory_bytes_committed{cluster=\"$cluster\", kubernetes_pod_name=~\"$instance\"})",
+ "format": "time_series",
+ "hide": false,
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "Heap size",
+ "metric": "jvm_memory_bytes_committed",
+ "refId": "A",
+ "step": 20
+ },
+ {
+ "expr": "sum(jvm_memory_bytes_used{cluster=\"$cluster\", kubernetes_pod_name=~\"$instance\"})",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "Used",
+ "metric": "jvm_memory_bytes_used",
+ "refId": "B",
+ "step": 20
+ },
+ {
+ "expr": "sum(jvm_memory_bytes_max{cluster=\"$cluster\", kubernetes_pod_name=~\"$instance\"})",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "Max",
+ "metric": "jvm_memory_bytes_max",
+ "refId": "C",
+ "step": 20
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeShift": null,
+ "title": "Heap Memory",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "decbytes",
+ "label": "",
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ]
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": null,
+ "fill": 1,
+ "id": 7,
+ "legend": {
+ "avg": false,
+ "current": false,
+ "max": false,
+ "min": false,
+ "show": false,
+ "total": false,
+ "values": false
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "nullPointMode": "null",
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "scopedVars": {
+ "instance": {
+ "selected": false,
+ "text": "bookie-44jrt",
+ "value": "bookie-44jrt"
+ }
+ },
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "span": 3,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "jvm_memory_direct_bytes_used{cluster=\"$cluster\", kubernetes_pod_name=~\"$instance\"}",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "Used",
+ "metric": "jvm_memory_direct_bytes_used",
+ "refId": "A",
+ "step": 20
+ },
+ {
+ "expr": "jvm_memory_direct_bytes_max{cluster=\"$cluster\", kubernetes_pod_name=~\"$instance\"}",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "Max",
+ "metric": "jvm_memory_direct_bytes_max",
+ "refId": "B",
+ "step": 20
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeShift": null,
+ "title": "Direct Memory",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "decbytes",
+ "label": "",
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": "",
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ }
+ ]
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": null,
+ "fill": 1,
+ "id": 8,
+ "legend": {
+ "avg": false,
+ "current": false,
+ "max": false,
+ "min": false,
+ "show": false,
+ "total": false,
+ "values": false
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "nullPointMode": "null",
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "scopedVars": {
+ "instance": {
+ "selected": false,
+ "text": "bookie-44jrt",
+ "value": "bookie-44jrt"
+ }
+ },
+ "seriesOverrides": [
+ {
+ "alias": "old count",
+ "yaxis": 2
+ },
+ {
+ "alias": "young count",
+ "yaxis": 2
+ }
+ ],
+ "spaceLength": 10,
+ "span": 3,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "sum(increase(jvm_gc_collection_seconds_sum{cluster=\"$cluster\", kubernetes_pod_name=~\"$instance\"}[30s]))",
+ "format": "time_series",
+ "hide": false,
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "Pause",
+ "metric": "jvm_gc_collection_seconds_count",
+ "refId": "B",
+ "step": 20
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeShift": null,
+ "title": "GC Pauses",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "s",
+ "label": "",
+ "logBase": 1,
+ "max": null,
+ "min": "0",
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": "",
+ "logBase": 1,
+ "max": null,
+ "min": "0",
+ "show": false
+ }
+ ]
+ }
+ ],
+ "repeat": null,
+ "repeatIteration": 1496162786419,
+ "repeatRowId": 1,
+ "showTitle": true,
+ "title": "$component $instance",
+ "titleSize": "h4"
+ },
+ {
+ "collapse": false,
+ "height": 221,
+ "panels": [
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": null,
+ "decimals": 1,
+ "fill": 1,
+ "id": 9,
+ "legend": {
+ "avg": false,
+ "current": false,
+ "max": false,
+ "min": false,
+ "show": false,
+ "total": false,
+ "values": false
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "nullPointMode": "null",
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "scopedVars": {
+ "instance": {
+ "selected": false,
+ "text": "bookie-73k0d",
+ "value": "bookie-73k0d"
+ }
+ },
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "span": 3,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "irate(process_cpu_seconds_total{cluster=\"$cluster\", kubernetes_pod_name=~\"$instance\"}[30s]) * 100\n\n",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "{{kubernetes_pod_name}}",
+ "metric": "process_cpu_seconds_total",
+ "refId": "A",
+ "step": 20
+ }
+ ],
+ "thresholds": [
+ {
+ "colorMode": "ok",
+ "fill": false,
+ "line": true,
+ "op": "gt",
+ "value": 100
+ }
+ ],
+ "timeFrom": null,
+ "timeShift": null,
+ "title": "CPU",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "percent",
+ "label": "",
+ "logBase": 1,
+ "max": null,
+ "min": "0",
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ }
+ ]
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "Prometheus",
+ "fill": 1,
+ "id": 10,
+ "legend": {
+ "alignAsTable": false,
+ "avg": false,
+ "current": false,
+ "hideEmpty": false,
+ "max": false,
+ "min": false,
+ "rightSide": false,
+ "show": false,
+ "total": false,
+ "values": false
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "nullPointMode": "null",
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "scopedVars": {
+ "instance": {
+ "selected": false,
+ "text": "bookie-73k0d",
+ "value": "bookie-73k0d"
+ }
+ },
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "span": 3,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "sum(jvm_memory_bytes_committed{cluster=\"$cluster\", kubernetes_pod_name=~\"$instance\"})",
+ "format": "time_series",
+ "hide": false,
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "Heap size",
+ "metric": "jvm_memory_bytes_committed",
+ "refId": "A",
+ "step": 20
+ },
+ {
+ "expr": "sum(jvm_memory_bytes_used{cluster=\"$cluster\", kubernetes_pod_name=~\"$instance\"})",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "Used",
+ "metric": "jvm_memory_bytes_used",
+ "refId": "B",
+ "step": 20
+ },
+ {
+ "expr": "sum(jvm_memory_bytes_max{cluster=\"$cluster\", kubernetes_pod_name=~\"$instance\"})",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "Max",
+ "metric": "jvm_memory_bytes_max",
+ "refId": "C",
+ "step": 20
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeShift": null,
+ "title": "Heap Memory",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "decbytes",
+ "label": "",
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ]
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": null,
+ "fill": 1,
+ "id": 11,
+ "legend": {
+ "avg": false,
+ "current": false,
+ "max": false,
+ "min": false,
+ "show": false,
+ "total": false,
+ "values": false
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "nullPointMode": "null",
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "scopedVars": {
+ "instance": {
+ "selected": false,
+ "text": "bookie-73k0d",
+ "value": "bookie-73k0d"
+ }
+ },
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "span": 3,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "jvm_memory_direct_bytes_used{cluster=\"$cluster\", kubernetes_pod_name=~\"$instance\"}",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "Used",
+ "metric": "jvm_memory_direct_bytes_used",
+ "refId": "A",
+ "step": 20
+ },
+ {
+ "expr": "jvm_memory_direct_bytes_max{cluster=\"$cluster\", kubernetes_pod_name=~\"$instance\"}",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "Max",
+ "metric": "jvm_memory_direct_bytes_max",
+ "refId": "B",
+ "step": 20
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeShift": null,
+ "title": "Direct Memory",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "decbytes",
+ "label": "",
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": "",
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ }
+ ]
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": null,
+ "fill": 1,
+ "id": 12,
+ "legend": {
+ "avg": false,
+ "current": false,
+ "max": false,
+ "min": false,
+ "show": false,
+ "total": false,
+ "values": false
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "nullPointMode": "null",
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "scopedVars": {
+ "instance": {
+ "selected": false,
+ "text": "bookie-73k0d",
+ "value": "bookie-73k0d"
+ }
+ },
+ "seriesOverrides": [
+ {
+ "alias": "old count",
+ "yaxis": 2
+ },
+ {
+ "alias": "young count",
+ "yaxis": 2
+ }
+ ],
+ "spaceLength": 10,
+ "span": 3,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "sum(increase(jvm_gc_collection_seconds_sum{cluster=\"$cluster\", kubernetes_pod_name=~\"$instance\"}[30s]))",
+ "format": "time_series",
+ "hide": false,
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "Pause",
+ "metric": "jvm_gc_collection_seconds_count",
+ "refId": "B",
+ "step": 20
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeShift": null,
+ "title": "GC Pauses",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "s",
+ "label": "",
+ "logBase": 1,
+ "max": null,
+ "min": "0",
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": "",
+ "logBase": 1,
+ "max": null,
+ "min": "0",
+ "show": false
+ }
+ ]
+ }
+ ],
+ "repeat": null,
+ "repeatIteration": 1496162786419,
+ "repeatRowId": 1,
+ "showTitle": true,
+ "title": "$component $instance",
+ "titleSize": "h4"
+ },
+ {
+ "collapse": false,
+ "height": 221,
+ "panels": [
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": null,
+ "decimals": 1,
+ "fill": 1,
+ "id": 13,
+ "legend": {
+ "avg": false,
+ "current": false,
+ "max": false,
+ "min": false,
+ "show": false,
+ "total": false,
+ "values": false
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "nullPointMode": "null",
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "scopedVars": {
+ "instance": {
+ "selected": false,
+ "text": "bookie-h2q4k",
+ "value": "bookie-h2q4k"
+ }
+ },
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "span": 3,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "irate(process_cpu_seconds_total{cluster=\"$cluster\", kubernetes_pod_name=~\"$instance\"}[30s]) * 100\n\n",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "{{kubernetes_pod_name}}",
+ "metric": "process_cpu_seconds_total",
+ "refId": "A",
+ "step": 20
+ }
+ ],
+ "thresholds": [
+ {
+ "colorMode": "ok",
+ "fill": false,
+ "line": true,
+ "op": "gt",
+ "value": 100
+ }
+ ],
+ "timeFrom": null,
+ "timeShift": null,
+ "title": "CPU",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "percent",
+ "label": "",
+ "logBase": 1,
+ "max": null,
+ "min": "0",
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ }
+ ]
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "Prometheus",
+ "fill": 1,
+ "id": 14,
+ "legend": {
+ "alignAsTable": false,
+ "avg": false,
+ "current": false,
+ "hideEmpty": false,
+ "max": false,
+ "min": false,
+ "rightSide": false,
+ "show": false,
+ "total": false,
+ "values": false
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "nullPointMode": "null",
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "scopedVars": {
+ "instance": {
+ "selected": false,
+ "text": "bookie-h2q4k",
+ "value": "bookie-h2q4k"
+ }
+ },
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "span": 3,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "sum(jvm_memory_bytes_committed{cluster=\"$cluster\", kubernetes_pod_name=~\"$instance\"})",
+ "format": "time_series",
+ "hide": false,
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "Heap size",
+ "metric": "jvm_memory_bytes_committed",
+ "refId": "A",
+ "step": 20
+ },
+ {
+ "expr": "sum(jvm_memory_bytes_used{cluster=\"$cluster\", kubernetes_pod_name=~\"$instance\"})",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "Used",
+ "metric": "jvm_memory_bytes_used",
+ "refId": "B",
+ "step": 20
+ },
+ {
+ "expr": "sum(jvm_memory_bytes_max{cluster=\"$cluster\", kubernetes_pod_name=~\"$instance\"})",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "Max",
+ "metric": "jvm_memory_bytes_max",
+ "refId": "C",
+ "step": 20
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeShift": null,
+ "title": "Heap Memory",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "decbytes",
+ "label": "",
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ]
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": null,
+ "fill": 1,
+ "id": 15,
+ "legend": {
+ "avg": false,
+ "current": false,
+ "max": false,
+ "min": false,
+ "show": false,
+ "total": false,
+ "values": false
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "nullPointMode": "null",
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "scopedVars": {
+ "instance": {
+ "selected": false,
+ "text": "bookie-h2q4k",
+ "value": "bookie-h2q4k"
+ }
+ },
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "span": 3,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "jvm_memory_direct_bytes_used{cluster=\"$cluster\", kubernetes_pod_name=~\"$instance\"}",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "Used",
+ "metric": "jvm_memory_direct_bytes_used",
+ "refId": "A",
+ "step": 20
+ },
+ {
+ "expr": "jvm_memory_direct_bytes_max{cluster=\"$cluster\", kubernetes_pod_name=~\"$instance\"}",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "Max",
+ "metric": "jvm_memory_direct_bytes_max",
+ "refId": "B",
+ "step": 20
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeShift": null,
+ "title": "Direct Memory",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "decbytes",
+ "label": "",
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": "",
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ }
+ ]
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": null,
+ "fill": 1,
+ "id": 16,
+ "legend": {
+ "avg": false,
+ "current": false,
+ "max": false,
+ "min": false,
+ "show": false,
+ "total": false,
+ "values": false
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "nullPointMode": "null",
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "scopedVars": {
+ "instance": {
+ "selected": false,
+ "text": "bookie-h2q4k",
+ "value": "bookie-h2q4k"
+ }
+ },
+ "seriesOverrides": [
+ {
+ "alias": "old count",
+ "yaxis": 2
+ },
+ {
+ "alias": "young count",
+ "yaxis": 2
+ }
+ ],
+ "spaceLength": 10,
+ "span": 3,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "sum(increase(jvm_gc_collection_seconds_sum{cluster=\"$cluster\", kubernetes_pod_name=~\"$instance\"}[30s]))",
+ "format": "time_series",
+ "hide": false,
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "Pause",
+ "metric": "jvm_gc_collection_seconds_count",
+ "refId": "B",
+ "step": 20
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeShift": null,
+ "title": "GC Pauses",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "s",
+ "label": "",
+ "logBase": 1,
+ "max": null,
+ "min": "0",
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": "",
+ "logBase": 1,
+ "max": null,
+ "min": "0",
+ "show": false
+ }
+ ]
+ }
+ ],
+ "repeat": null,
+ "repeatIteration": 1496162786419,
+ "repeatRowId": 1,
+ "showTitle": true,
+ "title": "$component $instance",
+ "titleSize": "h4"
+ },
+ {
+ "collapse": false,
+ "height": 221,
+ "panels": [
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": null,
+ "decimals": 1,
+ "fill": 1,
+ "id": 17,
+ "legend": {
+ "avg": false,
+ "current": false,
+ "max": false,
+ "min": false,
+ "show": false,
+ "total": false,
+ "values": false
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "nullPointMode": "null",
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "scopedVars": {
+ "instance": {
+ "selected": false,
+ "text": "bookie-m2bp9",
+ "value": "bookie-m2bp9"
+ }
+ },
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "span": 3,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "irate(process_cpu_seconds_total{cluster=\"$cluster\", kubernetes_pod_name=~\"$instance\"}[30s]) * 100\n\n",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "{{kubernetes_pod_name}}",
+ "metric": "process_cpu_seconds_total",
+ "refId": "A",
+ "step": 20
+ }
+ ],
+ "thresholds": [
+ {
+ "colorMode": "ok",
+ "fill": false,
+ "line": true,
+ "op": "gt",
+ "value": 100
+ }
+ ],
+ "timeFrom": null,
+ "timeShift": null,
+ "title": "CPU",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "percent",
+ "label": "",
+ "logBase": 1,
+ "max": null,
+ "min": "0",
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ }
+ ]
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "Prometheus",
+ "fill": 1,
+ "id": 18,
+ "legend": {
+ "alignAsTable": false,
+ "avg": false,
+ "current": false,
+ "hideEmpty": false,
+ "max": false,
+ "min": false,
+ "rightSide": false,
+ "show": false,
+ "total": false,
+ "values": false
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "nullPointMode": "null",
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "scopedVars": {
+ "instance": {
+ "selected": false,
+ "text": "bookie-m2bp9",
+ "value": "bookie-m2bp9"
+ }
+ },
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "span": 3,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "sum(jvm_memory_bytes_committed{cluster=\"$cluster\", kubernetes_pod_name=~\"$instance\"})",
+ "format": "time_series",
+ "hide": false,
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "Heap size",
+ "metric": "jvm_memory_bytes_committed",
+ "refId": "A",
+ "step": 20
+ },
+ {
+ "expr": "sum(jvm_memory_bytes_used{cluster=\"$cluster\", kubernetes_pod_name=~\"$instance\"})",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "Used",
+ "metric": "jvm_memory_bytes_used",
+ "refId": "B",
+ "step": 20
+ },
+ {
+ "expr": "sum(jvm_memory_bytes_max{cluster=\"$cluster\", kubernetes_pod_name=~\"$instance\"})",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "Max",
+ "metric": "jvm_memory_bytes_max",
+ "refId": "C",
+ "step": 20
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeShift": null,
+ "title": "Heap Memory",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "decbytes",
+ "label": "",
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ]
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": null,
+ "fill": 1,
+ "id": 19,
+ "legend": {
+ "avg": false,
+ "current": false,
+ "max": false,
+ "min": false,
+ "show": false,
+ "total": false,
+ "values": false
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "nullPointMode": "null",
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "scopedVars": {
+ "instance": {
+ "selected": false,
+ "text": "bookie-m2bp9",
+ "value": "bookie-m2bp9"
+ }
+ },
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "span": 3,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "jvm_memory_direct_bytes_used{cluster=\"$cluster\", kubernetes_pod_name=~\"$instance\"}",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "Used",
+ "metric": "jvm_memory_direct_bytes_used",
+ "refId": "A",
+ "step": 20
+ },
+ {
+ "expr": "jvm_memory_direct_bytes_max{cluster=\"$cluster\", kubernetes_pod_name=~\"$instance\"}",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "Max",
+ "metric": "jvm_memory_direct_bytes_max",
+ "refId": "B",
+ "step": 20
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeShift": null,
+ "title": "Direct Memory",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "decbytes",
+ "label": "",
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": "",
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ }
+ ]
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": null,
+ "fill": 1,
+ "id": 20,
+ "legend": {
+ "avg": false,
+ "current": false,
+ "max": false,
+ "min": false,
+ "show": false,
+ "total": false,
+ "values": false
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "nullPointMode": "null",
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "scopedVars": {
+ "instance": {
+ "selected": false,
+ "text": "bookie-m2bp9",
+ "value": "bookie-m2bp9"
+ }
+ },
+ "seriesOverrides": [
+ {
+ "alias": "old count",
+ "yaxis": 2
+ },
+ {
+ "alias": "young count",
+ "yaxis": 2
+ }
+ ],
+ "spaceLength": 10,
+ "span": 3,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "sum(increase(jvm_gc_collection_seconds_sum{cluster=\"$cluster\", kubernetes_pod_name=~\"$instance\"}[30s]))",
+ "format": "time_series",
+ "hide": false,
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "Pause",
+ "metric": "jvm_gc_collection_seconds_count",
+ "refId": "B",
+ "step": 20
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeShift": null,
+ "title": "GC Pauses",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "s",
+ "label": "",
+ "logBase": 1,
+ "max": null,
+ "min": "0",
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": "",
+ "logBase": 1,
+ "max": null,
+ "min": "0",
+ "show": false
+ }
+ ]
+ }
+ ],
+ "repeat": null,
+ "repeatIteration": 1496162786419,
+ "repeatRowId": 1,
+ "showTitle": true,
+ "title": "$component $instance",
+ "titleSize": "h4"
+ },
+ {
+ "collapse": false,
+ "height": 221,
+ "panels": [
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": null,
+ "decimals": 1,
+ "fill": 1,
+ "id": 21,
+ "legend": {
+ "avg": false,
+ "current": false,
+ "max": false,
+ "min": false,
+ "show": false,
+ "total": false,
+ "values": false
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "nullPointMode": "null",
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "scopedVars": {
+ "instance": {
+ "selected": false,
+ "text": "bookie-z6djd",
+ "value": "bookie-z6djd"
+ }
+ },
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "span": 3,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "irate(process_cpu_seconds_total{cluster=\"$cluster\", kubernetes_pod_name=~\"$instance\"}[30s]) * 100\n\n",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "{{kubernetes_pod_name}}",
+ "metric": "process_cpu_seconds_total",
+ "refId": "A",
+ "step": 20
+ }
+ ],
+ "thresholds": [
+ {
+ "colorMode": "ok",
+ "fill": false,
+ "line": true,
+ "op": "gt",
+ "value": 100
+ }
+ ],
+ "timeFrom": null,
+ "timeShift": null,
+ "title": "CPU",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "percent",
+ "label": "",
+ "logBase": 1,
+ "max": null,
+ "min": "0",
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ }
+ ]
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "Prometheus",
+ "fill": 1,
+ "id": 22,
+ "legend": {
+ "alignAsTable": false,
+ "avg": false,
+ "current": false,
+ "hideEmpty": false,
+ "max": false,
+ "min": false,
+ "rightSide": false,
+ "show": false,
+ "total": false,
+ "values": false
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "nullPointMode": "null",
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "scopedVars": {
+ "instance": {
+ "selected": false,
+ "text": "bookie-z6djd",
+ "value": "bookie-z6djd"
+ }
+ },
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "span": 3,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "sum(jvm_memory_bytes_committed{cluster=\"$cluster\", kubernetes_pod_name=~\"$instance\"})",
+ "format": "time_series",
+ "hide": false,
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "Heap size",
+ "metric": "jvm_memory_bytes_committed",
+ "refId": "A",
+ "step": 20
+ },
+ {
+ "expr": "sum(jvm_memory_bytes_used{cluster=\"$cluster\", kubernetes_pod_name=~\"$instance\"})",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "Used",
+ "metric": "jvm_memory_bytes_used",
+ "refId": "B",
+ "step": 20
+ },
+ {
+ "expr": "sum(jvm_memory_bytes_max{cluster=\"$cluster\", kubernetes_pod_name=~\"$instance\"})",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "Max",
+ "metric": "jvm_memory_bytes_max",
+ "refId": "C",
+ "step": 20
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeShift": null,
+ "title": "Heap Memory",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "decbytes",
+ "label": "",
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ]
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": null,
+ "fill": 1,
+ "id": 23,
+ "legend": {
+ "avg": false,
+ "current": false,
+ "max": false,
+ "min": false,
+ "show": false,
+ "total": false,
+ "values": false
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "nullPointMode": "null",
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "scopedVars": {
+ "instance": {
+ "selected": false,
+ "text": "bookie-z6djd",
+ "value": "bookie-z6djd"
+ }
+ },
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "span": 3,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "jvm_memory_direct_bytes_used{cluster=\"$cluster\", kubernetes_pod_name=~\"$instance\"}",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "Used",
+ "metric": "jvm_memory_direct_bytes_used",
+ "refId": "A",
+ "step": 20
+ },
+ {
+ "expr": "jvm_memory_direct_bytes_max{cluster=\"$cluster\", kubernetes_pod_name=~\"$instance\"}",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "Max",
+ "metric": "jvm_memory_direct_bytes_max",
+ "refId": "B",
+ "step": 20
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeShift": null,
+ "title": "Direct Memory",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "decbytes",
+ "label": "",
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": "",
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ }
+ ]
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": null,
+ "fill": 1,
+ "id": 24,
+ "legend": {
+ "avg": false,
+ "current": false,
+ "max": false,
+ "min": false,
+ "show": false,
+ "total": false,
+ "values": false
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "nullPointMode": "null",
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "scopedVars": {
+ "instance": {
+ "selected": false,
+ "text": "bookie-z6djd",
+ "value": "bookie-z6djd"
+ }
+ },
+ "seriesOverrides": [
+ {
+ "alias": "old count",
+ "yaxis": 2
+ },
+ {
+ "alias": "young count",
+ "yaxis": 2
+ }
+ ],
+ "spaceLength": 10,
+ "span": 3,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "sum(increase(jvm_gc_collection_seconds_sum{cluster=\"$cluster\", kubernetes_pod_name=~\"$instance\"}[30s]))",
+ "format": "time_series",
+ "hide": false,
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "Pause",
+ "metric": "jvm_gc_collection_seconds_count",
+ "refId": "B",
+ "step": 20
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeShift": null,
+ "title": "GC Pauses",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "s",
+ "label": "",
+ "logBase": 1,
+ "max": null,
+ "min": "0",
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": "",
+ "logBase": 1,
+ "max": null,
+ "min": "0",
+ "show": false
+ }
+ ]
+ }
+ ],
+ "repeat": null,
+ "repeatIteration": 1496162786419,
+ "repeatRowId": 1,
+ "showTitle": true,
+ "title": "$component $instance",
+ "titleSize": "h4"
+ }
+ ],
+ "schemaVersion": 14,
+ "style": "dark",
+ "tags": [
+ "kafka"
+ ],
+ "templating": {
+ "list": [
+ {
+ "allValue": null,
+ "current": {
+ "text": "us-central",
+ "value": "us-central"
+ },
+ "datasource": "Prometheus",
+ "hide": 0,
+ "includeAll": false,
+ "label": "Cluster",
+ "multi": false,
+ "name": "cluster",
+ "options": [],
+ "query": "{cluster=~\".+\"}",
+ "refresh": 1,
+ "regex": "/.*[^_]cluster=\\\"([^\\\"]+)\\\".*/",
+ "sort": 0,
+ "tagValuesQuery": "",
+ "tags": [],
+ "tagsQuery": "",
+ "type": "query",
+ "useTags": false
+ },
+ {
+ "allValue": null,
+ "current": {
+ "tags": [],
+ "text": "bookkeeper",
+ "value": "bookkeeper"
+ },
+ "datasource": "Prometheus",
+ "hide": 0,
+ "includeAll": false,
+ "label": "Component",
+ "multi": false,
+ "name": "component",
+ "options": [],
+ "query": "{job=~\".+\"}",
+ "refresh": 1,
+ "regex": "/.*[^_]job=\\\"([^\\\"]+)\\\".*/",
+ "sort": 1,
+ "tagValuesQuery": "",
+ "tags": [],
+ "tagsQuery": "",
+ "type": "query",
+ "useTags": false
+ },
+ {
+ "allValue": null,
+ "current": {
+ "text": "All",
+ "value": [
+ "$__all"
+ ]
+ },
+ "datasource": "Prometheus",
+ "hide": 0,
+ "includeAll": true,
+ "label": "Pod",
+ "multi": true,
+ "name": "instance",
+ "options": [],
+ "query": "jvm_memory_bytes_used{cluster=~\"$cluster\", job=~\"$component\", kubernetes_pod_name=~\".+\"}",
+ "refresh": 2,
+ "regex": "/.*[^_]kubernetes_pod_name=\\\"([^\\\"]+)\\\".*/",
+ "sort": 1,
+ "tagValuesQuery": "",
+ "tags": [],
+ "tagsQuery": "",
+ "type": "query",
+ "useTags": false
+ }
+ ]
+ },
+ "time": {
+ "from": "now-1h",
+ "to": "now"
+ },
+ "timepicker": {
+ "refresh_intervals": [
+ "5s",
+ "10s",
+ "30s",
+ "1m",
+ "5m",
+ "15m",
+ "30m",
+ "1h",
+ "2h",
+ "1d"
+ ],
+ "time_options": [
+ "5m",
+ "15m",
+ "1h",
+ "6h",
+ "12h",
+ "24h",
+ "2d",
+ "7d",
+ "30d"
+ ]
+ },
+ "timezone": "browser",
+ "title": "Kafka - JVM",
+ "version": 27
+}
\ No newline at end of file
diff --git a/driver-kafka/deploy/monitoring/dashboards/perf.json b/driver-kafka/deploy/monitoring/dashboards/perf.json
new file mode 100644
index 00000000..04e503da
--- /dev/null
+++ b/driver-kafka/deploy/monitoring/dashboards/perf.json
@@ -0,0 +1,878 @@
+{
+ "annotations": {
+ "list": [
+ {
+ "builtIn": 1,
+ "datasource": "-- Grafana --",
+ "enable": true,
+ "hide": true,
+ "iconColor": "rgba(0, 211, 255, 1)",
+ "name": "Annotations & Alerts",
+ "type": "dashboard"
+ }
+ ]
+ },
+ "editable": true,
+ "gnetId": null,
+ "graphTooltip": 0,
+ "id": 6,
+ "links": [],
+ "panels": [
+ {
+ "aliasColors": {
+ "Broker - Read": "#66cc69",
+ "Broker - Write": "#00afba",
+ "Consume rate": "blue",
+ "DD - Max throughput": "#d8365d",
+ "Produce rate": "#d8365d",
+ "{device=\"nvme1n1\", group=\"broker-os\", instance=\"10.0.0.207:9100\", job=\"kafka\"}": "green",
+ "{device=\"nvme1n1\", group=\"broker-os\", instance=\"10.0.0.212:9100\", job=\"kafka\"}": "yellow",
+ "{device=\"nvme1n1\", group=\"broker-os\", instance=\"10.0.0.5:9100\", job=\"kafka\"}": "red",
+ "{device=\"nvme2n1\", group=\"broker-os\", instance=\"10.0.0.207:9100\", job=\"kafka\"}": "blue",
+ "{device=\"nvme2n1\", group=\"broker-os\", instance=\"10.0.0.212:9100\", job=\"kafka\"}": "orange",
+ "{device=\"nvme2n1\", group=\"broker-os\", instance=\"10.0.0.5:9100\", job=\"kafka\"}": "purple"
+ },
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "Prometheus",
+ "fieldConfig": {
+ "defaults": {
+ "custom": {
+ "align": null
+ },
+ "mappings": [],
+ "thresholds": {
+ "mode": "absolute",
+ "steps": [
+ {
+ "color": "green",
+ "value": null
+ },
+ {
+ "color": "red",
+ "value": 80
+ }
+ ]
+ }
+ },
+ "overrides": []
+ },
+ "fill": 5,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 11,
+ "w": 12,
+ "x": 0,
+ "y": 0
+ },
+ "hiddenSeries": false,
+ "id": 12,
+ "legend": {
+ "alignAsTable": false,
+ "avg": false,
+ "current": false,
+ "max": false,
+ "min": false,
+ "show": true,
+ "total": false,
+ "values": false
+ },
+ "lines": false,
+ "linewidth": 1,
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pluginVersion": "7.0.5",
+ "pointradius": 2,
+ "points": true,
+ "renderer": "flot",
+ "seriesOverrides": [
+ {
+ "alias": "Consume rate",
+ "dashes": true,
+ "fill": 0,
+ "linewidth": 2,
+ "yaxis": 1
+ },
+ {
+ "alias": "Produce rate",
+ "dashes": true,
+ "fill": 0,
+ "linewidth": 2,
+ "yaxis": 1
+ },
+ {
+ "alias": "DD - Max throughput",
+ "fill": 0,
+ "lines": true,
+ "linewidth": 3,
+ "points": false
+ }
+ ],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "sum by (instance) (rate(node_disk_written_bytes_total{device=~\"nvme1n1|nvme2n1\"}[1m]))",
+ "hide": false,
+ "interval": "",
+ "intervalFactor": 1,
+ "legendFormat": "Broker - {{instance}} - Write",
+ "refId": "A"
+ },
+ {
+ "expr": "sum(rate(node_disk_read_bytes_total{device=~\"nvme1n1|nvme2n1\"}[1m]))",
+ "hide": true,
+ "interval": "",
+ "legendFormat": "Broker - Read",
+ "refId": "B"
+ },
+ {
+ "expr": "sum(rate(benchmark_producer_bytes_sent[1m]))",
+ "hide": false,
+ "interval": "",
+ "legendFormat": "Produce rate",
+ "refId": "C"
+ },
+ {
+ "expr": "sum(rate(benchmark_consumer_bytes_recv[1m]))",
+ "hide": false,
+ "interval": "",
+ "legendFormat": "Consume rate",
+ "refId": "D"
+ },
+ {
+ "expr": "655000000",
+ "interval": "",
+ "legendFormat": "DD - Max throughput",
+ "refId": "E"
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "I/O throughput + Producer/Consume Throughput",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "Bps",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ },
+ {
+ "format": "Bps",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {
+ "{device=\"nvme1n1\", group=\"broker-os\", instance=\"10.0.0.207:9100\", job=\"kafka\"}": "green",
+ "{device=\"nvme1n1\", group=\"broker-os\", instance=\"10.0.0.212:9100\", job=\"kafka\"}": "yellow",
+ "{device=\"nvme1n1\", group=\"broker-os\", instance=\"10.0.0.5:9100\", job=\"kafka\"}": "red",
+ "{device=\"nvme2n1\", group=\"broker-os\", instance=\"10.0.0.207:9100\", job=\"kafka\"}": "blue",
+ "{device=\"nvme2n1\", group=\"broker-os\", instance=\"10.0.0.212:9100\", job=\"kafka\"}": "orange",
+ "{device=\"nvme2n1\", group=\"broker-os\", instance=\"10.0.0.5:9100\", job=\"kafka\"}": "purple"
+ },
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "Prometheus",
+ "fieldConfig": {
+ "defaults": {
+ "custom": {}
+ },
+ "overrides": []
+ },
+ "fill": 1,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 9,
+ "w": 12,
+ "x": 12,
+ "y": 0
+ },
+ "hiddenSeries": false,
+ "id": 6,
+ "legend": {
+ "avg": false,
+ "current": false,
+ "max": false,
+ "min": false,
+ "show": true,
+ "total": false,
+ "values": false
+ },
+ "lines": true,
+ "linewidth": 1,
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 2,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "sum by (instance) (delta(node_disk_written_bytes_total{device=~\"nvme1n1|nvme2n1\"}[1m])/60)",
+ "interval": "",
+ "intervalFactor": 1,
+ "legendFormat": "Broker - {{instance}}",
+ "refId": "A"
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "I/O throughput",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "Bps",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": null,
+ "fieldConfig": {
+ "defaults": {
+ "custom": {}
+ },
+ "overrides": []
+ },
+ "fill": 1,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 8,
+ "w": 12,
+ "x": 12,
+ "y": 9
+ },
+ "hiddenSeries": false,
+ "id": 5,
+ "legend": {
+ "avg": false,
+ "current": false,
+ "hideEmpty": true,
+ "hideZero": true,
+ "max": false,
+ "min": false,
+ "show": false,
+ "total": false,
+ "values": false
+ },
+ "lines": true,
+ "linewidth": 1,
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 2,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "sum(delta(benchmark_producer_bytes_sent[1m])/60)",
+ "interval": "",
+ "legendFormat": "",
+ "refId": "A"
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "Publish throughput (MB/sec)",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "Bps",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {
+ "{device=\"nvme1n1\", group=\"broker-os\", instance=\"10.0.0.207:9100\", job=\"kafka\"}": "green",
+ "{device=\"nvme1n1\", group=\"broker-os\", instance=\"10.0.0.212:9100\", job=\"kafka\"}": "yellow",
+ "{device=\"nvme1n1\", group=\"broker-os\", instance=\"10.0.0.5:9100\", job=\"kafka\"}": "red",
+ "{device=\"nvme2n1\", group=\"broker-os\", instance=\"10.0.0.207:9100\", job=\"kafka\"}": "blue",
+ "{device=\"nvme2n1\", group=\"broker-os\", instance=\"10.0.0.212:9100\", job=\"kafka\"}": "orange",
+ "{device=\"nvme2n1\", group=\"broker-os\", instance=\"10.0.0.5:9100\", job=\"kafka\"}": "purple"
+ },
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "Prometheus",
+ "fieldConfig": {
+ "defaults": {
+ "custom": {}
+ },
+ "overrides": []
+ },
+ "fill": 1,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 9,
+ "w": 12,
+ "x": 0,
+ "y": 11
+ },
+ "hiddenSeries": false,
+ "id": 2,
+ "legend": {
+ "avg": false,
+ "current": false,
+ "hideEmpty": true,
+ "hideZero": true,
+ "max": false,
+ "min": false,
+ "show": true,
+ "total": false,
+ "values": false
+ },
+ "lines": true,
+ "linewidth": 1,
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 2,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "delta(node_disk_io_time_seconds_total{device=~\"nvme1n1|nvme2n1\"}[1m])/60",
+ "interval": "",
+ "intervalFactor": 1,
+ "legendFormat": "Broker - {{instance}} ➝ Disk - {{device}}",
+ "refId": "A"
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "I/O Utilization",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "percentunit",
+ "label": null,
+ "logBase": 1,
+ "max": "1",
+ "min": "0",
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": null,
+ "fieldConfig": {
+ "defaults": {
+ "custom": {}
+ },
+ "overrides": []
+ },
+ "fill": 1,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 8,
+ "w": 12,
+ "x": 0,
+ "y": 20
+ },
+ "hiddenSeries": false,
+ "id": 4,
+ "legend": {
+ "avg": false,
+ "current": false,
+ "hideEmpty": true,
+ "hideZero": true,
+ "max": false,
+ "min": false,
+ "show": false,
+ "total": false,
+ "values": false
+ },
+ "lines": true,
+ "linewidth": 1,
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 2,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "sum(delta(benchmark_producer_messages_sent[1m])/60)",
+ "interval": "",
+ "legendFormat": "",
+ "refId": "A"
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "Publish rate (messages/sec)",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "Prometheus",
+ "fieldConfig": {
+ "defaults": {
+ "custom": {}
+ },
+ "overrides": []
+ },
+ "fill": 1,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 8,
+ "w": 12,
+ "x": 0,
+ "y": 28
+ },
+ "hiddenSeries": false,
+ "id": 10,
+ "legend": {
+ "avg": false,
+ "current": false,
+ "hideEmpty": true,
+ "hideZero": true,
+ "max": false,
+ "min": false,
+ "show": true,
+ "total": false,
+ "values": false
+ },
+ "lines": true,
+ "linewidth": 1,
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 2,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "rate(kafka_log_logflushstats_logflushrateandtimems{quantile=\"0.99\"}[1m])",
+ "interval": "",
+ "legendFormat": "Broker - {{instance}}",
+ "refId": "A"
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "Log Flush Time (p99)",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "ms",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {
+ "Idle - Waiting for something to happen": "#052B51",
+ "Iowait - Waiting for I/O to complete": "blue",
+ "Nice - Niced processes executing in user mode": "yellow",
+ "Softirq - Servicing softirqs": "yellow",
+ "System - Processes executing in kernel mode": "dark-purple",
+ "User - Normal processes executing in user mode": "red",
+ "guest": "#9AC48A",
+ "idle": "#052B51",
+ "iowait": "#EAB839",
+ "irq": "#BF1B00",
+ "nice": "#C15C17",
+ "softirq": "#E24D42",
+ "steal": "#FCE2DE",
+ "system": "#508642",
+ "user": "#5195CE"
+ },
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": null,
+ "decimals": 2,
+ "description": "",
+ "fieldConfig": {
+ "defaults": {
+ "custom": {}
+ },
+ "overrides": []
+ },
+ "fill": 4,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 13,
+ "w": 24,
+ "x": 0,
+ "y": 36
+ },
+ "hiddenSeries": false,
+ "id": 8,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "max": true,
+ "min": true,
+ "rightSide": false,
+ "show": true,
+ "sideWidth": 250,
+ "sort": null,
+ "sortDesc": null,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "maxPerRow": 6,
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": true,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "repeat": null,
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "stack": true,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "sum by (mode)(irate(node_cpu_seconds_total{mode=\"user\"}[5m])) * 100",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "User - Normal processes executing in user mode",
+ "refId": "B",
+ "step": 240
+ },
+ {
+ "expr": "sum by (mode)(irate(node_cpu_seconds_total{mode=\"system\"}[5m])) * 100",
+ "format": "time_series",
+ "interval": "10s",
+ "intervalFactor": 2,
+ "legendFormat": "System - Processes executing in kernel mode",
+ "refId": "A",
+ "step": 20
+ },
+ {
+ "expr": "sum by (mode)(irate(node_cpu_seconds_total{mode=\"iowait\"}[5m])) * 100",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "Iowait - Waiting for I/O to complete",
+ "refId": "E",
+ "step": 240
+ },
+ {
+ "expr": "sum by (mode)(irate(node_cpu_seconds_total{mode=\"softirq\"}[5m])) * 100",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "Softirq - Servicing softirqs",
+ "refId": "G",
+ "step": 240
+ },
+ {
+ "expr": "sum by (mode)(irate(node_cpu_seconds_total{mode=\"idle\"}[5m])) * 100",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "Idle - Waiting for something to happen",
+ "refId": "D",
+ "step": 240
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "CPU Usage across all Brokers",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "short",
+ "label": "percentage",
+ "logBase": 1,
+ "max": "100",
+ "min": "0",
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ }
+ ],
+ "refresh": false,
+ "schemaVersion": 25,
+ "style": "dark",
+ "tags": [],
+ "templating": {
+ "list": []
+ },
+ "time": {
+ "from": "now-6h",
+ "to": "now"
+ },
+ "timepicker": {
+ "refresh_intervals": [
+ "10s",
+ "30s",
+ "1m",
+ "5m",
+ "15m",
+ "30m",
+ "1h",
+ "2h",
+ "1d"
+ ]
+ },
+ "timezone": "utc",
+ "title": "Performance Metrics - Kafka",
+ "uid": "vYWrwI7Mkafka",
+ "version": 5
+}
\ No newline at end of file
diff --git a/driver-kafka/deploy/monitoring/dashboards/prometheus.json b/driver-kafka/deploy/monitoring/dashboards/prometheus.json
new file mode 100644
index 00000000..40ebcf83
--- /dev/null
+++ b/driver-kafka/deploy/monitoring/dashboards/prometheus.json
@@ -0,0 +1,719 @@
+{
+ "annotations": {
+ "list": []
+ },
+ "editable": true,
+ "gnetId": null,
+ "graphTooltip": 0,
+ "hideControls": true,
+ "id": 1,
+ "links": [
+ {
+ "icon": "info",
+ "tags": [],
+ "targetBlank": true,
+ "title": "Grafana Docs",
+ "tooltip": "",
+ "type": "link",
+ "url": "http://docs.grafana.org/"
+ },
+ {
+ "icon": "info",
+ "tags": [],
+ "targetBlank": true,
+ "title": "Prometheus Docs",
+ "type": "link",
+ "url": "http://prometheus.io/docs/introduction/overview/"
+ }
+ ],
+ "refresh": false,
+ "revision": "1.0",
+ "rows": [
+ {
+ "collapse": false,
+ "height": 178,
+ "panels": [
+ {
+ "cacheTimeout": null,
+ "colorBackground": false,
+ "colorValue": false,
+ "colors": [
+ "rgba(245, 54, 54, 0.9)",
+ "rgba(237, 129, 40, 0.89)",
+ "rgba(50, 172, 45, 0.97)"
+ ],
+ "datasource": "Prometheus",
+ "decimals": 1,
+ "editable": true,
+ "error": false,
+ "format": "s",
+ "gauge": {
+ "maxValue": 100,
+ "minValue": 0,
+ "show": false,
+ "thresholdLabels": false,
+ "thresholdMarkers": true
+ },
+ "id": 5,
+ "interval": null,
+ "links": [],
+ "mappingType": 1,
+ "mappingTypes": [
+ {
+ "name": "value to text",
+ "value": 1
+ },
+ {
+ "name": "range to text",
+ "value": 2
+ }
+ ],
+ "maxDataPoints": 100,
+ "nullPointMode": "connected",
+ "nullText": null,
+ "postfix": "",
+ "postfixFontSize": "50%",
+ "prefix": "",
+ "prefixFontSize": "50%",
+ "rangeMaps": [
+ {
+ "from": "null",
+ "text": "N/A",
+ "to": "null"
+ }
+ ],
+ "span": 3,
+ "sparkline": {
+ "fillColor": "rgba(31, 118, 189, 0.18)",
+ "full": false,
+ "lineColor": "rgb(31, 120, 193)",
+ "show": false
+ },
+ "tableColumn": "",
+ "targets": [
+ {
+ "expr": "(time() - process_start_time_seconds{job=\"prometheus\"})",
+ "intervalFactor": 2,
+ "refId": "A",
+ "step": 4
+ }
+ ],
+ "thresholds": "",
+ "title": "Uptime",
+ "type": "singlestat",
+ "valueFontSize": "80%",
+ "valueMaps": [
+ {
+ "op": "=",
+ "text": "N/A",
+ "value": "null"
+ }
+ ],
+ "valueName": "current"
+ },
+ {
+ "cacheTimeout": null,
+ "colorBackground": false,
+ "colorValue": false,
+ "colors": [
+ "rgba(50, 172, 45, 0.97)",
+ "rgba(237, 129, 40, 0.89)",
+ "rgba(245, 54, 54, 0.9)"
+ ],
+ "datasource": "Prometheus",
+ "editable": true,
+ "error": false,
+ "format": "none",
+ "gauge": {
+ "maxValue": 100,
+ "minValue": 0,
+ "show": false,
+ "thresholdLabels": false,
+ "thresholdMarkers": true
+ },
+ "id": 6,
+ "interval": null,
+ "links": [],
+ "mappingType": 1,
+ "mappingTypes": [
+ {
+ "name": "value to text",
+ "value": 1
+ },
+ {
+ "name": "range to text",
+ "value": 2
+ }
+ ],
+ "maxDataPoints": 100,
+ "nullPointMode": "connected",
+ "nullText": null,
+ "postfix": "",
+ "postfixFontSize": "50%",
+ "prefix": "",
+ "prefixFontSize": "50%",
+ "rangeMaps": [
+ {
+ "from": "null",
+ "text": "N/A",
+ "to": "null"
+ }
+ ],
+ "span": 3,
+ "sparkline": {
+ "fillColor": "rgba(31, 118, 189, 0.18)",
+ "full": false,
+ "lineColor": "rgb(31, 120, 193)",
+ "show": true
+ },
+ "tableColumn": "",
+ "targets": [
+ {
+ "expr": "prometheus_local_storage_memory_series",
+ "intervalFactor": 2,
+ "refId": "A",
+ "step": 4
+ }
+ ],
+ "thresholds": "1,5",
+ "title": "Local Storage Memory Series",
+ "type": "singlestat",
+ "valueFontSize": "70%",
+ "valueMaps": [],
+ "valueName": "current"
+ },
+ {
+ "cacheTimeout": null,
+ "colorBackground": false,
+ "colorValue": true,
+ "colors": [
+ "rgba(50, 172, 45, 0.97)",
+ "rgba(237, 129, 40, 0.89)",
+ "rgba(245, 54, 54, 0.9)"
+ ],
+ "datasource": "Prometheus",
+ "editable": true,
+ "error": false,
+ "format": "none",
+ "gauge": {
+ "maxValue": 100,
+ "minValue": 0,
+ "show": false,
+ "thresholdLabels": false,
+ "thresholdMarkers": true
+ },
+ "id": 7,
+ "interval": null,
+ "links": [],
+ "mappingType": 1,
+ "mappingTypes": [
+ {
+ "name": "value to text",
+ "value": 1
+ },
+ {
+ "name": "range to text",
+ "value": 2
+ }
+ ],
+ "maxDataPoints": 100,
+ "nullPointMode": "connected",
+ "nullText": null,
+ "postfix": "",
+ "postfixFontSize": "50%",
+ "prefix": "",
+ "prefixFontSize": "50%",
+ "rangeMaps": [
+ {
+ "from": "null",
+ "text": "N/A",
+ "to": "null"
+ }
+ ],
+ "span": 3,
+ "sparkline": {
+ "fillColor": "rgba(31, 118, 189, 0.18)",
+ "full": false,
+ "lineColor": "rgb(31, 120, 193)",
+ "show": true
+ },
+ "tableColumn": "",
+ "targets": [
+ {
+ "expr": "prometheus_local_storage_indexing_queue_length",
+ "intervalFactor": 2,
+ "refId": "A",
+ "step": 4
+ }
+ ],
+ "thresholds": "500,4000",
+ "title": "Internal Storage Queue Length",
+ "type": "singlestat",
+ "valueFontSize": "70%",
+ "valueMaps": [
+ {
+ "op": "=",
+ "text": "Empty",
+ "value": "0"
+ }
+ ],
+ "valueName": "current"
+ },
+ {
+ "content": " \nPrometheus \n\nYou're using Prometheus, an open-source systems monitoring and alerting toolkit originally built at SoundCloud. For more information, check out the Grafana and Prometheus projects.
",
+ "editable": true,
+ "error": false,
+ "id": 9,
+ "links": [],
+ "mode": "html",
+ "span": 3,
+ "style": {},
+ "title": "",
+ "transparent": true,
+ "type": "text"
+ }
+ ],
+ "repeat": null,
+ "repeatIteration": null,
+ "repeatRowId": null,
+ "showTitle": false,
+ "title": "New row",
+ "titleSize": "h6"
+ },
+ {
+ "collapse": false,
+ "height": 227,
+ "panels": [
+ {
+ "aliasColors": {
+ "prometheus": "#C15C17",
+ "{instance=\"localhost:9090\",job=\"prometheus\"}": "#C15C17"
+ },
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "Prometheus",
+ "editable": true,
+ "error": false,
+ "fill": 1,
+ "grid": {},
+ "id": 3,
+ "legend": {
+ "avg": false,
+ "current": false,
+ "max": false,
+ "min": false,
+ "show": true,
+ "total": false,
+ "values": false
+ },
+ "lines": true,
+ "linewidth": 2,
+ "links": [],
+ "nullPointMode": "connected",
+ "percentage": false,
+ "pointradius": 2,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "span": 9,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "rate(prometheus_local_storage_ingested_samples_total[5m])",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "{{job}}",
+ "metric": "",
+ "refId": "A",
+ "step": 2
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeShift": null,
+ "title": "Samples ingested (rate-5m)",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "cumulative"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "short",
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ },
+ {
+ "format": "short",
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ }
+ ]
+ },
+ {
+ "content": "#### Samples Ingested\nThis graph displays the count of samples ingested by the Prometheus server, as measured over the last 5 minutes, per time series in the range vector. When troubleshooting an issue on IRC or Github, this is often the first stat requested by the Prometheus team. ",
+ "editable": true,
+ "error": false,
+ "id": 8,
+ "links": [],
+ "mode": "markdown",
+ "span": 2.995914043583536,
+ "style": {},
+ "title": "",
+ "transparent": true,
+ "type": "text"
+ }
+ ],
+ "repeat": null,
+ "repeatIteration": null,
+ "repeatRowId": null,
+ "showTitle": false,
+ "title": "New row",
+ "titleSize": "h6"
+ },
+ {
+ "collapse": false,
+ "height": "250px",
+ "panels": [
+ {
+ "aliasColors": {
+ "prometheus": "#F9BA8F",
+ "{instance=\"localhost:9090\",interval=\"5s\",job=\"prometheus\"}": "#F9BA8F"
+ },
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "Prometheus",
+ "editable": true,
+ "error": false,
+ "fill": 1,
+ "grid": {},
+ "id": 2,
+ "legend": {
+ "avg": false,
+ "current": false,
+ "max": false,
+ "min": false,
+ "show": true,
+ "total": false,
+ "values": false
+ },
+ "lines": true,
+ "linewidth": 2,
+ "links": [],
+ "nullPointMode": "connected",
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "span": 5,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "rate(prometheus_target_interval_length_seconds_count[5m])",
+ "intervalFactor": 2,
+ "legendFormat": "{{job}}",
+ "refId": "A",
+ "step": 2
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeShift": null,
+ "title": "Target Scrapes (last 5m)",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "cumulative"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "short",
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ },
+ {
+ "format": "short",
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ }
+ ]
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "Prometheus",
+ "editable": true,
+ "error": false,
+ "fill": 1,
+ "grid": {},
+ "id": 14,
+ "legend": {
+ "avg": false,
+ "current": false,
+ "max": false,
+ "min": false,
+ "show": true,
+ "total": false,
+ "values": false
+ },
+ "lines": true,
+ "linewidth": 2,
+ "links": [],
+ "nullPointMode": "connected",
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "span": 4,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "prometheus_target_interval_length_seconds{quantile!=\"0.01\", quantile!=\"0.05\"}",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "{{quantile}} ({{interval}})",
+ "metric": "",
+ "refId": "A",
+ "step": 2
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeShift": null,
+ "title": "Scrape Duration",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "cumulative"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "short",
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ },
+ {
+ "format": "short",
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ }
+ ]
+ },
+ {
+ "content": "#### Scrapes\nPrometheus scrapes metrics from instrumented jobs, either directly or via an intermediary push gateway for short-lived jobs. Target scrapes will show how frequently targets are scraped, as measured over the last 5 minutes, per time series in the range vector. Scrape Duration will show how long the scrapes are taking, with percentiles available as series. ",
+ "editable": true,
+ "error": false,
+ "id": 11,
+ "links": [],
+ "mode": "markdown",
+ "span": 3,
+ "style": {},
+ "title": "",
+ "transparent": true,
+ "type": "text"
+ }
+ ],
+ "repeat": null,
+ "repeatIteration": null,
+ "repeatRowId": null,
+ "showTitle": false,
+ "title": "New row",
+ "titleSize": "h6"
+ },
+ {
+ "collapse": false,
+ "height": "250px",
+ "panels": [
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "Prometheus",
+ "decimals": null,
+ "editable": true,
+ "error": false,
+ "fill": 1,
+ "grid": {},
+ "id": 12,
+ "legend": {
+ "alignAsTable": false,
+ "avg": false,
+ "current": false,
+ "hideEmpty": true,
+ "max": false,
+ "min": false,
+ "show": true,
+ "total": false,
+ "values": false
+ },
+ "lines": true,
+ "linewidth": 2,
+ "links": [],
+ "nullPointMode": "connected",
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "span": 9,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "prometheus_evaluator_duration_seconds{quantile!=\"0.01\", quantile!=\"0.05\"}",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "{{quantile}}",
+ "refId": "A",
+ "step": 2
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeShift": null,
+ "title": "Rule Eval Duration",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "cumulative"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "percentunit",
+ "label": "",
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ },
+ {
+ "format": "short",
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ }
+ ]
+ },
+ {
+ "content": "#### Rule Evaluation Duration\nThis graph panel plots the duration for all evaluations to execute. The 50th percentile, 90th percentile and 99th percentile are shown as three separate series to help identify outliers that may be skewing the data.",
+ "editable": true,
+ "error": false,
+ "id": 15,
+ "links": [],
+ "mode": "markdown",
+ "span": 3,
+ "style": {},
+ "title": "",
+ "transparent": true,
+ "type": "text"
+ }
+ ],
+ "repeat": null,
+ "repeatIteration": null,
+ "repeatRowId": null,
+ "showTitle": false,
+ "title": "New row",
+ "titleSize": "h6"
+ }
+ ],
+ "schemaVersion": 14,
+ "style": "dark",
+ "tags": [
+ "prometheus"
+ ],
+ "templating": {
+ "list": []
+ },
+ "time": {
+ "from": "now-5m",
+ "to": "now"
+ },
+ "timepicker": {
+ "now": true,
+ "refresh_intervals": [
+ "5s",
+ "10s",
+ "30s",
+ "1m",
+ "5m",
+ "15m",
+ "30m",
+ "1h",
+ "2h",
+ "1d"
+ ],
+ "time_options": [
+ "5m",
+ "15m",
+ "1h",
+ "6h",
+ "12h",
+ "24h",
+ "2d",
+ "7d",
+ "30d"
+ ]
+ },
+ "timezone": "browser",
+ "title": "Prometheus Stats",
+ "version": 0
+}
\ No newline at end of file
diff --git a/driver-kafka/deploy/monitoring/dashboards/pulsar_perf.json b/driver-kafka/deploy/monitoring/dashboards/pulsar_perf.json
new file mode 100644
index 00000000..069dcf21
--- /dev/null
+++ b/driver-kafka/deploy/monitoring/dashboards/pulsar_perf.json
@@ -0,0 +1,1036 @@
+{
+ "annotations": {
+ "list": [
+ {
+ "builtIn": 1,
+ "datasource": "-- Grafana --",
+ "enable": true,
+ "hide": true,
+ "iconColor": "rgba(0, 211, 255, 1)",
+ "name": "Annotations & Alerts",
+ "type": "dashboard"
+ }
+ ]
+ },
+ "editable": true,
+ "gnetId": null,
+ "graphTooltip": 0,
+ "id": 2,
+ "links": [],
+ "panels": [
+ {
+ "aliasColors": {
+ "Bookie Add Entry Latency (p99)": "#d8365d",
+ "Broker - Read": "#66cc69",
+ "Broker - Write": "#00afba",
+ "Consume rate": "#173361",
+ "DD - Max throughput": "#d8365d",
+ "Produce rate": "#d8365d",
+ "{device=\"nvme1n1\", group=\"broker-os\", instance=\"10.0.0.207:9100\", job=\"kafka\"}": "green",
+ "{device=\"nvme1n1\", group=\"broker-os\", instance=\"10.0.0.212:9100\", job=\"kafka\"}": "yellow",
+ "{device=\"nvme1n1\", group=\"broker-os\", instance=\"10.0.0.5:9100\", job=\"kafka\"}": "red",
+ "{device=\"nvme2n1\", group=\"broker-os\", instance=\"10.0.0.207:9100\", job=\"kafka\"}": "blue",
+ "{device=\"nvme2n1\", group=\"broker-os\", instance=\"10.0.0.212:9100\", job=\"kafka\"}": "orange",
+ "{device=\"nvme2n1\", group=\"broker-os\", instance=\"10.0.0.5:9100\", job=\"kafka\"}": "purple"
+ },
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "Prometheus",
+ "description": "",
+ "fieldConfig": {
+ "defaults": {
+ "custom": {
+ "align": null
+ },
+ "mappings": [],
+ "thresholds": {
+ "mode": "absolute",
+ "steps": [
+ {
+ "color": "green",
+ "value": null
+ },
+ {
+ "color": "red",
+ "value": 80
+ }
+ ]
+ }
+ },
+ "overrides": []
+ },
+ "fill": 5,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 11,
+ "w": 12,
+ "x": 0,
+ "y": 0
+ },
+ "hiddenSeries": false,
+ "id": 12,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": false,
+ "hideEmpty": true,
+ "hideZero": true,
+ "max": true,
+ "min": true,
+ "show": true,
+ "total": false,
+ "values": true
+ },
+ "lines": false,
+ "linewidth": 1,
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pluginVersion": "7.0.5",
+ "pointradius": 2,
+ "points": true,
+ "renderer": "flot",
+ "seriesOverrides": [
+ {
+ "alias": "Consume rate",
+ "dashes": true,
+ "fill": 0,
+ "linewidth": 2,
+ "yaxis": 1
+ },
+ {
+ "alias": "Produce rate",
+ "dashes": true,
+ "fill": 0,
+ "linewidth": 2,
+ "yaxis": 1
+ },
+ {
+ "alias": "DD - Max throughput",
+ "fill": 0,
+ "lines": true,
+ "linewidth": 3,
+ "points": false
+ },
+ {
+ "alias": "Bookie Add Entry Rate",
+ "lines": true
+ },
+ {
+ "alias": "Bookie Add Entry Latency (p99)",
+ "dashes": true,
+ "linewidth": 2,
+ "points": false,
+ "yaxis": 2
+ }
+ ],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "sum by (instance) (rate(node_disk_written_bytes_total{device=~\"nvme1n1|nvme2n1\"}[1m]))",
+ "hide": true,
+ "interval": "",
+ "intervalFactor": 1,
+ "legendFormat": "Broker - {{instance}} - Write",
+ "refId": "A"
+ },
+ {
+ "expr": "sum(rate(node_disk_read_bytes_total{device=~\"nvme1n1|nvme2n1\"}[1m]))",
+ "hide": true,
+ "interval": "",
+ "legendFormat": "Broker - Read",
+ "refId": "B"
+ },
+ {
+ "expr": "sum(pulsar_throughput_in)",
+ "hide": false,
+ "interval": "",
+ "legendFormat": "Produce rate",
+ "refId": "C"
+ },
+ {
+ "expr": "sum(pulsar_throughput_out)",
+ "hide": false,
+ "interval": "",
+ "legendFormat": "Consume rate",
+ "refId": "D"
+ },
+ {
+ "expr": "655000000",
+ "hide": true,
+ "interval": "",
+ "legendFormat": "DD - Max throughput",
+ "refId": "E"
+ },
+ {
+ "expr": "sum(rate(bookie_WRITE_BYTES[30s]))",
+ "hide": false,
+ "interval": "",
+ "legendFormat": "Bookie Add Entry Rate",
+ "refId": "F"
+ },
+ {
+ "expr": "avg(bookkeeper_server_ADD_ENTRY_REQUEST{success=\"true\", quantile=\"0.99\"})",
+ "hide": false,
+ "interval": "",
+ "legendFormat": "Bookie Add Entry Latency (p99)",
+ "refId": "G"
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "I/O throughput + Producer/Consume Throughput",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "Bps",
+ "label": "Throughput",
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ },
+ {
+ "format": "ms",
+ "label": "Latency",
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {
+ "Bookie Add Entry Latency (p99)": "#d8365d",
+ "Broker - Read": "#66cc69",
+ "Broker - Write": "#00afba",
+ "Consume rate": "#173361",
+ "DD - Max throughput": "#d8365d",
+ "Produce rate": "#d8365d",
+ "Value": "#d8365d",
+ "{device=\"nvme1n1\", group=\"broker-os\", instance=\"10.0.0.207:9100\", job=\"kafka\"}": "green",
+ "{device=\"nvme1n1\", group=\"broker-os\", instance=\"10.0.0.212:9100\", job=\"kafka\"}": "yellow",
+ "{device=\"nvme1n1\", group=\"broker-os\", instance=\"10.0.0.5:9100\", job=\"kafka\"}": "red",
+ "{device=\"nvme2n1\", group=\"broker-os\", instance=\"10.0.0.207:9100\", job=\"kafka\"}": "blue",
+ "{device=\"nvme2n1\", group=\"broker-os\", instance=\"10.0.0.212:9100\", job=\"kafka\"}": "orange",
+ "{device=\"nvme2n1\", group=\"broker-os\", instance=\"10.0.0.5:9100\", job=\"kafka\"}": "purple"
+ },
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "Prometheus",
+ "fieldConfig": {
+ "defaults": {
+ "custom": {
+ "align": null
+ },
+ "mappings": [],
+ "thresholds": {
+ "mode": "absolute",
+ "steps": [
+ {
+ "color": "green",
+ "value": null
+ },
+ {
+ "color": "red",
+ "value": 80
+ }
+ ]
+ }
+ },
+ "overrides": []
+ },
+ "fill": 5,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 11,
+ "w": 12,
+ "x": 12,
+ "y": 0
+ },
+ "hiddenSeries": false,
+ "id": 13,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": false,
+ "hideEmpty": true,
+ "hideZero": true,
+ "max": true,
+ "min": true,
+ "show": true,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pluginVersion": "7.0.5",
+ "pointradius": 2,
+ "points": true,
+ "renderer": "flot",
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "bookie_journal_JOURNAL_CB_QUEUE_SIZE",
+ "interval": "",
+ "legendFormat": "Number of entries - {{instance}}",
+ "refId": "A"
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "Journal Callback Queue Length",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "short",
+ "label": "Entries",
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ },
+ {
+ "format": "ms",
+ "label": "Latency",
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {
+ "{device=\"nvme1n1\", group=\"broker-os\", instance=\"10.0.0.207:9100\", job=\"kafka\"}": "green",
+ "{device=\"nvme1n1\", group=\"broker-os\", instance=\"10.0.0.212:9100\", job=\"kafka\"}": "yellow",
+ "{device=\"nvme1n1\", group=\"broker-os\", instance=\"10.0.0.5:9100\", job=\"kafka\"}": "red",
+ "{device=\"nvme2n1\", group=\"broker-os\", instance=\"10.0.0.207:9100\", job=\"kafka\"}": "blue",
+ "{device=\"nvme2n1\", group=\"broker-os\", instance=\"10.0.0.212:9100\", job=\"kafka\"}": "orange",
+ "{device=\"nvme2n1\", group=\"broker-os\", instance=\"10.0.0.5:9100\", job=\"kafka\"}": "purple"
+ },
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "Prometheus",
+ "fieldConfig": {
+ "defaults": {
+ "custom": {}
+ },
+ "overrides": []
+ },
+ "fill": 1,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 9,
+ "w": 12,
+ "x": 0,
+ "y": 11
+ },
+ "hiddenSeries": false,
+ "id": 2,
+ "legend": {
+ "avg": false,
+ "current": false,
+ "hideEmpty": true,
+ "hideZero": true,
+ "max": false,
+ "min": false,
+ "show": true,
+ "total": false,
+ "values": false
+ },
+ "lines": true,
+ "linewidth": 1,
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 2,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "delta(node_disk_io_time_seconds_total{device=~\"nvme1n1|nvme2n1\"}[1m])/60",
+ "interval": "",
+ "intervalFactor": 1,
+ "legendFormat": "Broker - {{instance}} ➝ Disk - {{device}}",
+ "refId": "A"
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "I/O Utilization",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "percentunit",
+ "label": null,
+ "logBase": 1,
+ "max": "1",
+ "min": "0",
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {
+ "{device=\"nvme1n1\", group=\"broker-os\", instance=\"10.0.0.207:9100\", job=\"kafka\"}": "green",
+ "{device=\"nvme1n1\", group=\"broker-os\", instance=\"10.0.0.212:9100\", job=\"kafka\"}": "yellow",
+ "{device=\"nvme1n1\", group=\"broker-os\", instance=\"10.0.0.5:9100\", job=\"kafka\"}": "red",
+ "{device=\"nvme2n1\", group=\"broker-os\", instance=\"10.0.0.207:9100\", job=\"kafka\"}": "blue",
+ "{device=\"nvme2n1\", group=\"broker-os\", instance=\"10.0.0.212:9100\", job=\"kafka\"}": "orange",
+ "{device=\"nvme2n1\", group=\"broker-os\", instance=\"10.0.0.5:9100\", job=\"kafka\"}": "purple"
+ },
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "Prometheus",
+ "fieldConfig": {
+ "defaults": {
+ "custom": {}
+ },
+ "overrides": []
+ },
+ "fill": 1,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 9,
+ "w": 12,
+ "x": 12,
+ "y": 11
+ },
+ "hiddenSeries": false,
+ "id": 6,
+ "legend": {
+ "avg": false,
+ "current": false,
+ "max": false,
+ "min": false,
+ "show": true,
+ "total": false,
+ "values": false
+ },
+ "lines": true,
+ "linewidth": 1,
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 2,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "sum by (instance) (delta(node_disk_written_bytes_total{device=~\"nvme1n1|nvme2n1\"}[1m])/60)",
+ "interval": "",
+ "intervalFactor": 1,
+ "legendFormat": "Broker - {{instance}}",
+ "refId": "A"
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "I/O throughput",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "Bps",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": null,
+ "fieldConfig": {
+ "defaults": {
+ "custom": {}
+ },
+ "overrides": []
+ },
+ "fill": 1,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 8,
+ "w": 12,
+ "x": 0,
+ "y": 20
+ },
+ "hiddenSeries": false,
+ "id": 4,
+ "legend": {
+ "avg": false,
+ "current": false,
+ "hideEmpty": true,
+ "hideZero": true,
+ "max": false,
+ "min": false,
+ "show": false,
+ "total": false,
+ "values": false
+ },
+ "lines": true,
+ "linewidth": 1,
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 2,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "sum(delta(benchmark_producer_messages_sent[1m])/60)",
+ "interval": "",
+ "legendFormat": "",
+ "refId": "A"
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "Publish rate (messages/sec)",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": null,
+ "fieldConfig": {
+ "defaults": {
+ "custom": {}
+ },
+ "overrides": []
+ },
+ "fill": 1,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 8,
+ "w": 12,
+ "x": 12,
+ "y": 20
+ },
+ "hiddenSeries": false,
+ "id": 5,
+ "legend": {
+ "avg": false,
+ "current": false,
+ "hideEmpty": true,
+ "hideZero": true,
+ "max": false,
+ "min": false,
+ "show": false,
+ "total": false,
+ "values": false
+ },
+ "lines": true,
+ "linewidth": 1,
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 2,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "sum(delta(benchmark_producer_bytes_sent[1m])/60)",
+ "interval": "",
+ "legendFormat": "",
+ "refId": "A"
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "Publish throughput (MB/sec)",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "Bps",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "Prometheus",
+ "fieldConfig": {
+ "defaults": {
+ "custom": {}
+ },
+ "overrides": []
+ },
+ "fill": 1,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 8,
+ "w": 12,
+ "x": 0,
+ "y": 28
+ },
+ "hiddenSeries": false,
+ "id": 10,
+ "legend": {
+ "avg": false,
+ "current": false,
+ "hideEmpty": true,
+ "hideZero": true,
+ "max": false,
+ "min": false,
+ "show": true,
+ "total": false,
+ "values": false
+ },
+ "lines": true,
+ "linewidth": 1,
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 2,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "rate(kafka_log_logflushstats_logflushrateandtimems{quantile=\"0.99\"}[1m])",
+ "interval": "",
+ "legendFormat": "Broker - {{instance}}",
+ "refId": "A"
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "Log Flush Time (p99)",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "ms",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {
+ "Idle - Waiting for something to happen": "#052B51",
+ "Iowait - Waiting for I/O to complete": "blue",
+ "Nice - Niced processes executing in user mode": "yellow",
+ "Softirq - Servicing softirqs": "yellow",
+ "System - Processes executing in kernel mode": "dark-purple",
+ "User - Normal processes executing in user mode": "red",
+ "guest": "#9AC48A",
+ "idle": "#052B51",
+ "iowait": "#EAB839",
+ "irq": "#BF1B00",
+ "nice": "#C15C17",
+ "softirq": "#E24D42",
+ "steal": "#FCE2DE",
+ "system": "#508642",
+ "user": "#5195CE"
+ },
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": null,
+ "decimals": 2,
+ "description": "",
+ "fieldConfig": {
+ "defaults": {
+ "custom": {}
+ },
+ "overrides": []
+ },
+ "fill": 4,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 13,
+ "w": 24,
+ "x": 0,
+ "y": 36
+ },
+ "hiddenSeries": false,
+ "id": 8,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "max": true,
+ "min": true,
+ "rightSide": false,
+ "show": true,
+ "sideWidth": 250,
+ "sort": null,
+ "sortDesc": null,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "maxPerRow": 6,
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": true,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "repeat": null,
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "stack": true,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "sum by (mode)(irate(node_cpu_seconds_total{mode=\"user\"}[5m])) * 100",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "User - Normal processes executing in user mode",
+ "refId": "B",
+ "step": 240
+ },
+ {
+ "expr": "sum by (mode)(irate(node_cpu_seconds_total{mode=\"system\"}[5m])) * 100",
+ "format": "time_series",
+ "interval": "10s",
+ "intervalFactor": 2,
+ "legendFormat": "System - Processes executing in kernel mode",
+ "refId": "A",
+ "step": 20
+ },
+ {
+ "expr": "sum by (mode)(irate(node_cpu_seconds_total{mode=\"iowait\"}[5m])) * 100",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "Iowait - Waiting for I/O to complete",
+ "refId": "E",
+ "step": 240
+ },
+ {
+ "expr": "sum by (mode)(irate(node_cpu_seconds_total{mode=\"softirq\"}[5m])) * 100",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "Softirq - Servicing softirqs",
+ "refId": "G",
+ "step": 240
+ },
+ {
+ "expr": "sum by (mode)(irate(node_cpu_seconds_total{mode=\"idle\"}[5m])) * 100",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "Idle - Waiting for something to happen",
+ "refId": "D",
+ "step": 240
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "CPU Usage across all Brokers",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "short",
+ "label": "percentage",
+ "logBase": 1,
+ "max": "100",
+ "min": "0",
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ }
+ ],
+ "refresh": false,
+ "schemaVersion": 25,
+ "style": "dark",
+ "tags": [],
+ "templating": {
+ "list": []
+ },
+ "time": {
+ "from": "now-6h",
+ "to": "now"
+ },
+ "timepicker": {
+ "refresh_intervals": [
+ "10s",
+ "30s",
+ "1m",
+ "5m",
+ "15m",
+ "30m",
+ "1h",
+ "2h",
+ "1d"
+ ]
+ },
+ "timezone": "utc",
+ "title": "Performance Metrics - Pulsar",
+ "uid": "vYWrwI7Mk",
+ "version": 8
+}
\ No newline at end of file
diff --git a/driver-kafka/deploy/monitoring/dashboards/rabbit_perf.json b/driver-kafka/deploy/monitoring/dashboards/rabbit_perf.json
new file mode 100644
index 00000000..c2e5a017
--- /dev/null
+++ b/driver-kafka/deploy/monitoring/dashboards/rabbit_perf.json
@@ -0,0 +1,892 @@
+{
+ "annotations": {
+ "list": [
+ {
+ "builtIn": 1,
+ "datasource": "-- Grafana --",
+ "enable": true,
+ "hide": true,
+ "iconColor": "rgba(0, 211, 255, 1)",
+ "name": "Annotations & Alerts",
+ "type": "dashboard"
+ }
+ ]
+ },
+ "editable": true,
+ "gnetId": null,
+ "graphTooltip": 0,
+ "id": 7,
+ "links": [],
+ "panels": [
+ {
+ "aliasColors": {
+ "Broker - Read": "#66cc69",
+ "Broker - Write": "#00afba",
+ "CPU Usage": "#66cc69",
+ "Consume rate": "#ffc40c",
+ "DD - Max throughput": "#d8365d",
+ "Produce rate": "#173361",
+ "{device=\"nvme1n1\", group=\"broker-os\", instance=\"10.0.0.207:9100\", job=\"kafka\"}": "green",
+ "{device=\"nvme1n1\", group=\"broker-os\", instance=\"10.0.0.212:9100\", job=\"kafka\"}": "yellow",
+ "{device=\"nvme1n1\", group=\"broker-os\", instance=\"10.0.0.5:9100\", job=\"kafka\"}": "red",
+ "{device=\"nvme2n1\", group=\"broker-os\", instance=\"10.0.0.207:9100\", job=\"kafka\"}": "blue",
+ "{device=\"nvme2n1\", group=\"broker-os\", instance=\"10.0.0.212:9100\", job=\"kafka\"}": "orange",
+ "{device=\"nvme2n1\", group=\"broker-os\", instance=\"10.0.0.5:9100\", job=\"kafka\"}": "purple"
+ },
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "Prometheus",
+ "fieldConfig": {
+ "defaults": {
+ "custom": {
+ "align": null
+ },
+ "mappings": [],
+ "thresholds": {
+ "mode": "absolute",
+ "steps": [
+ {
+ "color": "green",
+ "value": null
+ },
+ {
+ "color": "red",
+ "value": 80
+ }
+ ]
+ }
+ },
+ "overrides": []
+ },
+ "fill": 5,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 11,
+ "w": 12,
+ "x": 0,
+ "y": 0
+ },
+ "hiddenSeries": false,
+ "id": 12,
+ "legend": {
+ "alignAsTable": false,
+ "avg": false,
+ "current": false,
+ "max": false,
+ "min": false,
+ "show": true,
+ "total": false,
+ "values": false
+ },
+ "lines": false,
+ "linewidth": 1,
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pluginVersion": "7.0.5",
+ "pointradius": 2,
+ "points": true,
+ "renderer": "flot",
+ "seriesOverrides": [
+ {
+ "alias": "Consume rate",
+ "dashes": true,
+ "fill": 0,
+ "linewidth": 2,
+ "yaxis": 1
+ },
+ {
+ "alias": "Produce rate",
+ "dashes": true,
+ "fill": 0,
+ "linewidth": 2,
+ "yaxis": 1
+ },
+ {
+ "alias": "DD - Max throughput",
+ "fill": 0,
+ "lines": true,
+ "linewidth": 3,
+ "points": false
+ },
+ {
+ "alias": "CPU Usage",
+ "lines": true,
+ "points": false,
+ "yaxis": 2
+ }
+ ],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "sum by (instance) (rate(node_disk_written_bytes_total{device=~\"nvme1n1|nvme2n1\"}[1m]))",
+ "hide": true,
+ "interval": "",
+ "intervalFactor": 1,
+ "legendFormat": "Broker - {{instance}} - Write",
+ "refId": "A"
+ },
+ {
+ "expr": "sum(rate(node_disk_read_bytes_total{device=~\"nvme1n1|nvme2n1\"}[1m]))",
+ "hide": true,
+ "interval": "",
+ "legendFormat": "Broker - Read",
+ "refId": "B"
+ },
+ {
+ "expr": "sum(rate(benchmark_producer_bytes_sent[1m]))",
+ "hide": false,
+ "interval": "",
+ "legendFormat": "Produce rate",
+ "refId": "C"
+ },
+ {
+ "expr": "sum(rate(benchmark_consumer_bytes_recv[1m]))",
+ "hide": false,
+ "interval": "",
+ "legendFormat": "Consume rate",
+ "refId": "D"
+ },
+ {
+ "expr": "655000000",
+ "hide": true,
+ "interval": "",
+ "legendFormat": "DD - Max throughput",
+ "refId": "E"
+ },
+ {
+ "expr": "(((count(count(avg(node_cpu_seconds_total)) by (cpu))) - avg(sum by (mode)(avg(irate(node_cpu_seconds_total{mode='idle'}[5m]))))) * 100) / count(count(avg(node_cpu_seconds_total)) by (cpu))",
+ "interval": "",
+ "legendFormat": "CPU Usage",
+ "refId": "F"
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "Producer/Consume Throughput + CPU Usage",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "Bps",
+ "label": "I/O throughput",
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ },
+ {
+ "format": "percent",
+ "label": "CPU Usage",
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {
+ "{device=\"nvme1n1\", group=\"broker-os\", instance=\"10.0.0.207:9100\", job=\"kafka\"}": "green",
+ "{device=\"nvme1n1\", group=\"broker-os\", instance=\"10.0.0.212:9100\", job=\"kafka\"}": "yellow",
+ "{device=\"nvme1n1\", group=\"broker-os\", instance=\"10.0.0.5:9100\", job=\"kafka\"}": "red",
+ "{device=\"nvme2n1\", group=\"broker-os\", instance=\"10.0.0.207:9100\", job=\"kafka\"}": "blue",
+ "{device=\"nvme2n1\", group=\"broker-os\", instance=\"10.0.0.212:9100\", job=\"kafka\"}": "orange",
+ "{device=\"nvme2n1\", group=\"broker-os\", instance=\"10.0.0.5:9100\", job=\"kafka\"}": "purple"
+ },
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "Prometheus",
+ "fieldConfig": {
+ "defaults": {
+ "custom": {}
+ },
+ "overrides": []
+ },
+ "fill": 1,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 9,
+ "w": 12,
+ "x": 12,
+ "y": 0
+ },
+ "hiddenSeries": false,
+ "id": 6,
+ "legend": {
+ "avg": false,
+ "current": false,
+ "max": false,
+ "min": false,
+ "show": true,
+ "total": false,
+ "values": false
+ },
+ "lines": true,
+ "linewidth": 1,
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 2,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "sum by (instance) (delta(node_disk_written_bytes_total{device=~\"nvme1n1|nvme2n1\"}[1m])/60)",
+ "interval": "",
+ "intervalFactor": 1,
+ "legendFormat": "Broker - {{instance}}",
+ "refId": "A"
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "I/O throughput",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "Bps",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": null,
+ "fieldConfig": {
+ "defaults": {
+ "custom": {}
+ },
+ "overrides": []
+ },
+ "fill": 1,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 8,
+ "w": 12,
+ "x": 12,
+ "y": 9
+ },
+ "hiddenSeries": false,
+ "id": 5,
+ "legend": {
+ "avg": false,
+ "current": false,
+ "hideEmpty": true,
+ "hideZero": true,
+ "max": false,
+ "min": false,
+ "show": false,
+ "total": false,
+ "values": false
+ },
+ "lines": true,
+ "linewidth": 1,
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 2,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "sum(delta(benchmark_producer_bytes_sent[1m])/60)",
+ "interval": "",
+ "legendFormat": "",
+ "refId": "A"
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "Publish throughput (MB/sec)",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "Bps",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {
+ "{device=\"nvme1n1\", group=\"broker-os\", instance=\"10.0.0.207:9100\", job=\"kafka\"}": "green",
+ "{device=\"nvme1n1\", group=\"broker-os\", instance=\"10.0.0.212:9100\", job=\"kafka\"}": "yellow",
+ "{device=\"nvme1n1\", group=\"broker-os\", instance=\"10.0.0.5:9100\", job=\"kafka\"}": "red",
+ "{device=\"nvme2n1\", group=\"broker-os\", instance=\"10.0.0.207:9100\", job=\"kafka\"}": "blue",
+ "{device=\"nvme2n1\", group=\"broker-os\", instance=\"10.0.0.212:9100\", job=\"kafka\"}": "orange",
+ "{device=\"nvme2n1\", group=\"broker-os\", instance=\"10.0.0.5:9100\", job=\"kafka\"}": "purple"
+ },
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "Prometheus",
+ "fieldConfig": {
+ "defaults": {
+ "custom": {}
+ },
+ "overrides": []
+ },
+ "fill": 1,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 9,
+ "w": 12,
+ "x": 0,
+ "y": 11
+ },
+ "hiddenSeries": false,
+ "id": 2,
+ "legend": {
+ "avg": false,
+ "current": false,
+ "hideEmpty": true,
+ "hideZero": true,
+ "max": false,
+ "min": false,
+ "show": true,
+ "total": false,
+ "values": false
+ },
+ "lines": true,
+ "linewidth": 1,
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 2,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "delta(node_disk_io_time_seconds_total{device=~\"nvme1n1|nvme2n1\"}[1m])/60",
+ "interval": "",
+ "intervalFactor": 1,
+ "legendFormat": "Broker - {{instance}} ➝ Disk - {{device}}",
+ "refId": "A"
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "I/O Utilization",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "percentunit",
+ "label": null,
+ "logBase": 1,
+ "max": "1",
+ "min": "0",
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": null,
+ "fieldConfig": {
+ "defaults": {
+ "custom": {}
+ },
+ "overrides": []
+ },
+ "fill": 1,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 8,
+ "w": 12,
+ "x": 0,
+ "y": 20
+ },
+ "hiddenSeries": false,
+ "id": 4,
+ "legend": {
+ "avg": false,
+ "current": false,
+ "hideEmpty": true,
+ "hideZero": true,
+ "max": false,
+ "min": false,
+ "show": false,
+ "total": false,
+ "values": false
+ },
+ "lines": true,
+ "linewidth": 1,
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 2,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "sum(delta(benchmark_producer_messages_sent[1m])/60)",
+ "interval": "",
+ "legendFormat": "",
+ "refId": "A"
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "Publish rate (messages/sec)",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "Prometheus",
+ "fieldConfig": {
+ "defaults": {
+ "custom": {}
+ },
+ "overrides": []
+ },
+ "fill": 1,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 8,
+ "w": 12,
+ "x": 0,
+ "y": 28
+ },
+ "hiddenSeries": false,
+ "id": 10,
+ "legend": {
+ "avg": false,
+ "current": false,
+ "hideEmpty": true,
+ "hideZero": true,
+ "max": false,
+ "min": false,
+ "show": true,
+ "total": false,
+ "values": false
+ },
+ "lines": true,
+ "linewidth": 1,
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 2,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "rate(kafka_log_logflushstats_logflushrateandtimems{quantile=\"0.99\"}[1m])",
+ "interval": "",
+ "legendFormat": "Broker - {{instance}}",
+ "refId": "A"
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "Log Flush Time (p99)",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "ms",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {
+ "Idle - Waiting for something to happen": "#052B51",
+ "Iowait - Waiting for I/O to complete": "blue",
+ "Nice - Niced processes executing in user mode": "yellow",
+ "Softirq - Servicing softirqs": "yellow",
+ "System - Processes executing in kernel mode": "dark-purple",
+ "User - Normal processes executing in user mode": "red",
+ "guest": "#9AC48A",
+ "idle": "#052B51",
+ "iowait": "#EAB839",
+ "irq": "#BF1B00",
+ "nice": "#C15C17",
+ "softirq": "#E24D42",
+ "steal": "#FCE2DE",
+ "system": "#508642",
+ "user": "#5195CE"
+ },
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": null,
+ "decimals": 2,
+ "description": "",
+ "fieldConfig": {
+ "defaults": {
+ "custom": {}
+ },
+ "overrides": []
+ },
+ "fill": 4,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 13,
+ "w": 24,
+ "x": 0,
+ "y": 36
+ },
+ "hiddenSeries": false,
+ "id": 8,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "max": true,
+ "min": true,
+ "rightSide": false,
+ "show": true,
+ "sideWidth": 250,
+ "sort": null,
+ "sortDesc": null,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "maxPerRow": 6,
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": true,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "repeat": null,
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "stack": true,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "sum by (mode)(irate(node_cpu_seconds_total{mode=\"user\"}[5m])) * 100",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "User - Normal processes executing in user mode",
+ "refId": "B",
+ "step": 240
+ },
+ {
+ "expr": "sum by (mode)(irate(node_cpu_seconds_total{mode=\"system\"}[5m])) * 100",
+ "format": "time_series",
+ "interval": "10s",
+ "intervalFactor": 2,
+ "legendFormat": "System - Processes executing in kernel mode",
+ "refId": "A",
+ "step": 20
+ },
+ {
+ "expr": "sum by (mode)(irate(node_cpu_seconds_total{mode=\"iowait\"}[5m])) * 100",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "Iowait - Waiting for I/O to complete",
+ "refId": "E",
+ "step": 240
+ },
+ {
+ "expr": "sum by (mode)(irate(node_cpu_seconds_total{mode=\"softirq\"}[5m])) * 100",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "Softirq - Servicing softirqs",
+ "refId": "G",
+ "step": 240
+ },
+ {
+ "expr": "sum by (mode)(irate(node_cpu_seconds_total{mode=\"idle\"}[5m])) * 100",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "Idle - Waiting for something to happen",
+ "refId": "D",
+ "step": 240
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "CPU Usage across all Brokers",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "short",
+ "label": "percentage",
+ "logBase": 1,
+ "max": "100",
+ "min": "0",
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ }
+ ],
+ "refresh": false,
+ "schemaVersion": 25,
+ "style": "dark",
+ "tags": [],
+ "templating": {
+ "list": []
+ },
+ "time": {
+ "from": "now-6h",
+ "to": "now"
+ },
+ "timepicker": {
+ "refresh_intervals": [
+ "10s",
+ "30s",
+ "1m",
+ "5m",
+ "15m",
+ "30m",
+ "1h",
+ "2h",
+ "1d"
+ ]
+ },
+ "timezone": "utc",
+ "title": "Performance Metrics - RabbitMQ",
+ "uid": "vYWrwI7Mkrabbit",
+ "version": 3
+}
\ No newline at end of file
diff --git a/driver-kafka/deploy/monitoring/dashboards/system.json b/driver-kafka/deploy/monitoring/dashboards/system.json
new file mode 100644
index 00000000..aa51c95f
--- /dev/null
+++ b/driver-kafka/deploy/monitoring/dashboards/system.json
@@ -0,0 +1,14051 @@
+{
+ "annotations": {
+ "list": [
+ {
+ "builtIn": 1,
+ "datasource": "-- Grafana --",
+ "enable": true,
+ "hide": true,
+ "iconColor": "rgba(0, 211, 255, 1)",
+ "name": "Annotations & Alerts",
+ "type": "dashboard"
+ }
+ ]
+ },
+ "editable": true,
+ "gnetId": 1860,
+ "graphTooltip": 0,
+ "id": 2,
+ "iteration": 1597269377076,
+ "links": [],
+ "panels": [
+ {
+ "aliasColors": {
+ "benchmark_producer_produce_latency{group=\"client\", instance=\"10.0.0.23:8081\", job=\"kafka\", quantile=\"0.99\", success=\"true\"}": "blue",
+ "benchmark_producer_produce_latency{group=\"client\", instance=\"10.0.0.61:8081\", job=\"kafka\", quantile=\"0.99\", success=\"true\"}": "yellow"
+ },
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "Prometheus",
+ "fieldConfig": {
+ "defaults": {
+ "custom": {}
+ },
+ "overrides": []
+ },
+ "fill": 1,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 8,
+ "w": 24,
+ "x": 0,
+ "y": 0
+ },
+ "hiddenSeries": false,
+ "id": 317,
+ "legend": {
+ "avg": false,
+ "current": false,
+ "hideEmpty": true,
+ "max": false,
+ "min": false,
+ "show": true,
+ "total": false,
+ "values": false
+ },
+ "lines": false,
+ "linewidth": 1,
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 0.5,
+ "points": true,
+ "renderer": "flot",
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "benchmark_producer_produce_latency{quantile=\"0.99\"}",
+ "interval": "",
+ "legendFormat": "",
+ "refId": "A"
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "Publish latency - p99",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "ms",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "collapsed": true,
+ "datasource": "${DS_PROMETHEUS}",
+ "gridPos": {
+ "h": 1,
+ "w": 24,
+ "x": 0,
+ "y": 8
+ },
+ "id": 261,
+ "panels": [
+ {
+ "cacheTimeout": null,
+ "datasource": "${DS_PROMETHEUS}",
+ "description": "Busy state of all CPU cores together",
+ "fieldConfig": {
+ "defaults": {
+ "color": {
+ "mode": "thresholds"
+ },
+ "custom": {},
+ "mappings": [
+ {
+ "id": 0,
+ "op": "=",
+ "text": "N/A",
+ "type": 1,
+ "value": "null"
+ }
+ ],
+ "max": 100,
+ "min": 0,
+ "nullValueMode": "null",
+ "thresholds": {
+ "mode": "absolute",
+ "steps": [
+ {
+ "color": "rgba(50, 172, 45, 0.97)",
+ "value": null
+ },
+ {
+ "color": "rgba(237, 129, 40, 0.89)",
+ "value": 85
+ },
+ {
+ "color": "rgba(245, 54, 54, 0.9)",
+ "value": 95
+ }
+ ]
+ },
+ "unit": "percent"
+ },
+ "overrides": []
+ },
+ "gridPos": {
+ "h": 4,
+ "w": 3,
+ "x": 0,
+ "y": 9
+ },
+ "id": 20,
+ "links": [],
+ "options": {
+ "orientation": "horizontal",
+ "reduceOptions": {
+ "calcs": [
+ "lastNotNull"
+ ],
+ "fields": "",
+ "values": false
+ },
+ "showThresholdLabels": false,
+ "showThresholdMarkers": true
+ },
+ "pluginVersion": "7.0.5",
+ "targets": [
+ {
+ "expr": "(((count(count(node_cpu_seconds_total{instance=~\"$node:$port\",job=~\"$job\"}) by (cpu))) - avg(sum by (mode)(irate(node_cpu_seconds_total{mode='idle',instance=~\"$node:$port\",job=~\"$job\"}[5m])))) * 100) / count(count(node_cpu_seconds_total{instance=~\"$node:$port\",job=~\"$job\"}) by (cpu))",
+ "hide": false,
+ "intervalFactor": 1,
+ "legendFormat": "",
+ "refId": "A",
+ "step": 900
+ }
+ ],
+ "title": "CPU Busy",
+ "type": "gauge"
+ },
+ {
+ "cacheTimeout": null,
+ "datasource": "${DS_PROMETHEUS}",
+ "description": "Busy state of all CPU cores together (5 min average)",
+ "fieldConfig": {
+ "defaults": {
+ "color": {
+ "mode": "thresholds"
+ },
+ "custom": {},
+ "mappings": [
+ {
+ "id": 0,
+ "op": "=",
+ "text": "N/A",
+ "type": 1,
+ "value": "null"
+ }
+ ],
+ "max": 100,
+ "min": 0,
+ "nullValueMode": "null",
+ "thresholds": {
+ "mode": "absolute",
+ "steps": [
+ {
+ "color": "rgba(50, 172, 45, 0.97)",
+ "value": null
+ },
+ {
+ "color": "rgba(237, 129, 40, 0.89)",
+ "value": 85
+ },
+ {
+ "color": "rgba(245, 54, 54, 0.9)",
+ "value": 95
+ }
+ ]
+ },
+ "unit": "percent"
+ },
+ "overrides": []
+ },
+ "gridPos": {
+ "h": 4,
+ "w": 3,
+ "x": 3,
+ "y": 9
+ },
+ "id": 155,
+ "links": [],
+ "options": {
+ "orientation": "horizontal",
+ "reduceOptions": {
+ "calcs": [
+ "lastNotNull"
+ ],
+ "fields": "",
+ "values": false
+ },
+ "showThresholdLabels": false,
+ "showThresholdMarkers": true
+ },
+ "pluginVersion": "7.0.5",
+ "targets": [
+ {
+ "expr": "avg(node_load5{instance=~\"$node:$port\",job=~\"$job\"}) / count(count(node_cpu_seconds_total{instance=~\"$node:$port\",job=~\"$job\"}) by (cpu)) * 100",
+ "format": "time_series",
+ "hide": false,
+ "intervalFactor": 1,
+ "refId": "A",
+ "step": 900
+ }
+ ],
+ "title": "Sys Load (5m avg)",
+ "type": "gauge"
+ },
+ {
+ "cacheTimeout": null,
+ "datasource": "${DS_PROMETHEUS}",
+ "description": "Busy state of all CPU cores together (15 min average)",
+ "fieldConfig": {
+ "defaults": {
+ "color": {
+ "mode": "thresholds"
+ },
+ "custom": {},
+ "mappings": [
+ {
+ "id": 0,
+ "op": "=",
+ "text": "N/A",
+ "type": 1,
+ "value": "null"
+ }
+ ],
+ "max": 100,
+ "min": 0,
+ "nullValueMode": "null",
+ "thresholds": {
+ "mode": "absolute",
+ "steps": [
+ {
+ "color": "rgba(50, 172, 45, 0.97)",
+ "value": null
+ },
+ {
+ "color": "rgba(237, 129, 40, 0.89)",
+ "value": 85
+ },
+ {
+ "color": "rgba(245, 54, 54, 0.9)",
+ "value": 95
+ }
+ ]
+ },
+ "unit": "percent"
+ },
+ "overrides": []
+ },
+ "gridPos": {
+ "h": 4,
+ "w": 3,
+ "x": 6,
+ "y": 9
+ },
+ "id": 19,
+ "links": [],
+ "options": {
+ "orientation": "horizontal",
+ "reduceOptions": {
+ "calcs": [
+ "lastNotNull"
+ ],
+ "fields": "",
+ "values": false
+ },
+ "showThresholdLabels": false,
+ "showThresholdMarkers": true
+ },
+ "pluginVersion": "7.0.5",
+ "targets": [
+ {
+ "expr": "avg(node_load15{instance=~\"$node:$port\",job=~\"$job\"}) / count(count(node_cpu_seconds_total{instance=~\"$node:$port\",job=~\"$job\"}) by (cpu)) * 100",
+ "hide": false,
+ "intervalFactor": 1,
+ "refId": "A",
+ "step": 900
+ }
+ ],
+ "title": "Sys Load (15m avg)",
+ "type": "gauge"
+ },
+ {
+ "cacheTimeout": null,
+ "datasource": "${DS_PROMETHEUS}",
+ "description": "Non available RAM memory",
+ "fieldConfig": {
+ "defaults": {
+ "color": {
+ "mode": "thresholds"
+ },
+ "custom": {},
+ "decimals": 0,
+ "mappings": [],
+ "max": 100,
+ "min": 0,
+ "nullValueMode": "null",
+ "thresholds": {
+ "mode": "absolute",
+ "steps": [
+ {
+ "color": "rgba(50, 172, 45, 0.97)",
+ "value": null
+ },
+ {
+ "color": "rgba(237, 129, 40, 0.89)",
+ "value": 80
+ },
+ {
+ "color": "rgba(245, 54, 54, 0.9)",
+ "value": 90
+ }
+ ]
+ },
+ "unit": "percent"
+ },
+ "overrides": []
+ },
+ "gridPos": {
+ "h": 4,
+ "w": 3,
+ "x": 9,
+ "y": 9
+ },
+ "hideTimeOverride": false,
+ "id": 16,
+ "links": [],
+ "options": {
+ "orientation": "horizontal",
+ "reduceOptions": {
+ "calcs": [
+ "lastNotNull"
+ ],
+ "fields": "",
+ "values": false
+ },
+ "showThresholdLabels": false,
+ "showThresholdMarkers": true
+ },
+ "pluginVersion": "7.0.5",
+ "targets": [
+ {
+ "expr": "((node_memory_MemTotal_bytes{instance=~\"$node:$port\",job=~\"$job\"} - node_memory_MemFree_bytes{instance=~\"$node:$port\",job=~\"$job\"}) / (node_memory_MemTotal_bytes{instance=~\"$node:$port\",job=~\"$job\"} )) * 100",
+ "format": "time_series",
+ "hide": true,
+ "intervalFactor": 1,
+ "refId": "A",
+ "step": 900
+ },
+ {
+ "expr": "100 - ((node_memory_MemAvailable_bytes{instance=~\"$node:$port\",job=~\"$job\"} * 100) / node_memory_MemTotal_bytes{instance=~\"$node:$port\",job=~\"$job\"})",
+ "format": "time_series",
+ "hide": false,
+ "intervalFactor": 1,
+ "refId": "B",
+ "step": 900
+ }
+ ],
+ "title": "RAM Used",
+ "type": "gauge"
+ },
+ {
+ "cacheTimeout": null,
+ "datasource": "${DS_PROMETHEUS}",
+ "description": "Used Swap",
+ "fieldConfig": {
+ "defaults": {
+ "color": {
+ "mode": "thresholds"
+ },
+ "custom": {},
+ "mappings": [
+ {
+ "id": 0,
+ "op": "=",
+ "text": "N/A",
+ "type": 1,
+ "value": "null"
+ }
+ ],
+ "max": 100,
+ "min": 0,
+ "nullValueMode": "null",
+ "thresholds": {
+ "mode": "absolute",
+ "steps": [
+ {
+ "color": "rgba(50, 172, 45, 0.97)",
+ "value": null
+ },
+ {
+ "color": "rgba(237, 129, 40, 0.89)",
+ "value": 10
+ },
+ {
+ "color": "rgba(245, 54, 54, 0.9)",
+ "value": 25
+ }
+ ]
+ },
+ "unit": "percent"
+ },
+ "overrides": []
+ },
+ "gridPos": {
+ "h": 4,
+ "w": 3,
+ "x": 12,
+ "y": 9
+ },
+ "id": 21,
+ "links": [],
+ "options": {
+ "orientation": "horizontal",
+ "reduceOptions": {
+ "calcs": [
+ "lastNotNull"
+ ],
+ "fields": "",
+ "values": false
+ },
+ "showThresholdLabels": false,
+ "showThresholdMarkers": true
+ },
+ "pluginVersion": "7.0.5",
+ "targets": [
+ {
+ "expr": "((node_memory_SwapTotal_bytes{instance=~\"$node:$port\",job=~\"$job\"} - node_memory_SwapFree_bytes{instance=~\"$node:$port\",job=~\"$job\"}) / (node_memory_SwapTotal_bytes{instance=~\"$node:$port\",job=~\"$job\"} )) * 100",
+ "intervalFactor": 1,
+ "refId": "A",
+ "step": 900
+ }
+ ],
+ "title": "SWAP Used",
+ "type": "gauge"
+ },
+ {
+ "cacheTimeout": null,
+ "datasource": "${DS_PROMETHEUS}",
+ "description": "Used Root FS",
+ "fieldConfig": {
+ "defaults": {
+ "color": {
+ "mode": "thresholds"
+ },
+ "custom": {},
+ "mappings": [
+ {
+ "id": 0,
+ "op": "=",
+ "text": "N/A",
+ "type": 1,
+ "value": "null"
+ }
+ ],
+ "max": 100,
+ "min": 0,
+ "nullValueMode": "null",
+ "thresholds": {
+ "mode": "absolute",
+ "steps": [
+ {
+ "color": "rgba(50, 172, 45, 0.97)",
+ "value": null
+ },
+ {
+ "color": "rgba(237, 129, 40, 0.89)",
+ "value": 80
+ },
+ {
+ "color": "rgba(245, 54, 54, 0.9)",
+ "value": 90
+ }
+ ]
+ },
+ "unit": "percent"
+ },
+ "overrides": []
+ },
+ "gridPos": {
+ "h": 4,
+ "w": 3,
+ "x": 15,
+ "y": 9
+ },
+ "id": 154,
+ "links": [],
+ "options": {
+ "orientation": "horizontal",
+ "reduceOptions": {
+ "calcs": [
+ "lastNotNull"
+ ],
+ "fields": "",
+ "values": false
+ },
+ "showThresholdLabels": false,
+ "showThresholdMarkers": true
+ },
+ "pluginVersion": "7.0.5",
+ "targets": [
+ {
+ "expr": "100 - ((node_filesystem_avail_bytes{instance=~\"$node:$port\",job=~\"$job\",mountpoint=\"/\",fstype!=\"rootfs\"} * 100) / node_filesystem_size_bytes{instance=~\"$node:$port\",job=~\"$job\",mountpoint=\"/\",fstype!=\"rootfs\"})",
+ "format": "time_series",
+ "intervalFactor": 1,
+ "refId": "A",
+ "step": 900
+ }
+ ],
+ "title": "Root FS Used",
+ "type": "gauge"
+ },
+ {
+ "cacheTimeout": null,
+ "colorBackground": false,
+ "colorValue": false,
+ "colors": [
+ "rgba(245, 54, 54, 0.9)",
+ "rgba(237, 129, 40, 0.89)",
+ "rgba(50, 172, 45, 0.97)"
+ ],
+ "datasource": "${DS_PROMETHEUS}",
+ "description": "Total number of CPU cores",
+ "fieldConfig": {
+ "defaults": {
+ "custom": {}
+ },
+ "overrides": []
+ },
+ "format": "short",
+ "gauge": {
+ "maxValue": 100,
+ "minValue": 0,
+ "show": false,
+ "thresholdLabels": false,
+ "thresholdMarkers": true
+ },
+ "gridPos": {
+ "h": 2,
+ "w": 2,
+ "x": 18,
+ "y": 9
+ },
+ "id": 14,
+ "interval": null,
+ "links": [],
+ "mappingType": 1,
+ "mappingTypes": [
+ {
+ "name": "value to text",
+ "value": 1
+ },
+ {
+ "name": "range to text",
+ "value": 2
+ }
+ ],
+ "maxDataPoints": 100,
+ "maxPerRow": 6,
+ "nullPointMode": "null",
+ "nullText": null,
+ "postfix": "",
+ "postfixFontSize": "50%",
+ "prefix": "",
+ "prefixFontSize": "50%",
+ "rangeMaps": [
+ {
+ "from": "null",
+ "text": "N/A",
+ "to": "null"
+ }
+ ],
+ "sparkline": {
+ "fillColor": "rgba(31, 118, 189, 0.18)",
+ "full": false,
+ "lineColor": "rgb(31, 120, 193)",
+ "show": false
+ },
+ "tableColumn": "",
+ "targets": [
+ {
+ "expr": "count(count(node_cpu_seconds_total{instance=~\"$node:$port\",job=~\"$job\"}) by (cpu))",
+ "interval": "",
+ "intervalFactor": 1,
+ "legendFormat": "",
+ "refId": "A",
+ "step": 900
+ }
+ ],
+ "thresholds": "",
+ "title": "CPU Cores",
+ "type": "singlestat",
+ "valueFontSize": "50%",
+ "valueMaps": [
+ {
+ "op": "=",
+ "text": "N/A",
+ "value": "null"
+ }
+ ],
+ "valueName": "current"
+ },
+ {
+ "cacheTimeout": null,
+ "colorBackground": false,
+ "colorValue": false,
+ "colors": [
+ "rgba(245, 54, 54, 0.9)",
+ "rgba(237, 129, 40, 0.89)",
+ "rgba(50, 172, 45, 0.97)"
+ ],
+ "datasource": "${DS_PROMETHEUS}",
+ "decimals": 1,
+ "description": "System uptime",
+ "fieldConfig": {
+ "defaults": {
+ "custom": {}
+ },
+ "overrides": []
+ },
+ "format": "s",
+ "gauge": {
+ "maxValue": 100,
+ "minValue": 0,
+ "show": false,
+ "thresholdLabels": false,
+ "thresholdMarkers": true
+ },
+ "gridPos": {
+ "h": 2,
+ "w": 4,
+ "x": 20,
+ "y": 9
+ },
+ "hideTimeOverride": true,
+ "id": 15,
+ "interval": null,
+ "links": [],
+ "mappingType": 1,
+ "mappingTypes": [
+ {
+ "name": "value to text",
+ "value": 1
+ },
+ {
+ "name": "range to text",
+ "value": 2
+ }
+ ],
+ "maxDataPoints": 100,
+ "nullPointMode": "null",
+ "nullText": null,
+ "postfix": "s",
+ "postfixFontSize": "50%",
+ "prefix": "",
+ "prefixFontSize": "50%",
+ "rangeMaps": [
+ {
+ "from": "null",
+ "text": "N/A",
+ "to": "null"
+ }
+ ],
+ "sparkline": {
+ "fillColor": "rgba(31, 118, 189, 0.18)",
+ "full": false,
+ "lineColor": "rgb(31, 120, 193)",
+ "show": false
+ },
+ "tableColumn": "{group=\"broker-os\", instance=\"10.0.0.15:9100\", job=\"kafka\"}",
+ "targets": [
+ {
+ "expr": "node_time_seconds{instance=~\"$node:$port\",job=~\"$job\"} - node_boot_time_seconds{instance=~\"$node:$port\",job=~\"$job\"}",
+ "intervalFactor": 2,
+ "refId": "A",
+ "step": 1800
+ }
+ ],
+ "thresholds": "",
+ "title": "Uptime",
+ "type": "singlestat",
+ "valueFontSize": "50%",
+ "valueMaps": [
+ {
+ "op": "=",
+ "text": "N/A",
+ "value": "null"
+ }
+ ],
+ "valueName": "current"
+ },
+ {
+ "cacheTimeout": null,
+ "colorBackground": false,
+ "colorValue": false,
+ "colors": [
+ "rgba(50, 172, 45, 0.97)",
+ "rgba(237, 129, 40, 0.89)",
+ "rgba(245, 54, 54, 0.9)"
+ ],
+ "datasource": "${DS_PROMETHEUS}",
+ "decimals": 0,
+ "description": "Total RootFS",
+ "fieldConfig": {
+ "defaults": {
+ "custom": {}
+ },
+ "overrides": []
+ },
+ "format": "bytes",
+ "gauge": {
+ "maxValue": 100,
+ "minValue": 0,
+ "show": false,
+ "thresholdLabels": false,
+ "thresholdMarkers": true
+ },
+ "gridPos": {
+ "h": 2,
+ "w": 2,
+ "x": 18,
+ "y": 11
+ },
+ "id": 23,
+ "interval": null,
+ "links": [],
+ "mappingType": 1,
+ "mappingTypes": [
+ {
+ "name": "value to text",
+ "value": 1
+ },
+ {
+ "name": "range to text",
+ "value": 2
+ }
+ ],
+ "maxDataPoints": 100,
+ "maxPerRow": 6,
+ "nullPointMode": "null",
+ "nullText": null,
+ "postfix": "",
+ "postfixFontSize": "50%",
+ "prefix": "",
+ "prefixFontSize": "50%",
+ "rangeMaps": [
+ {
+ "from": "null",
+ "text": "N/A",
+ "to": "null"
+ }
+ ],
+ "sparkline": {
+ "fillColor": "rgba(31, 118, 189, 0.18)",
+ "full": false,
+ "lineColor": "rgb(31, 120, 193)",
+ "show": false
+ },
+ "tableColumn": "node_filesystem_size_bytes{device=\"/dev/nvme0n1p2\", fstype=\"xfs\", group=\"broker-os\", instance=\"10.0.0.15:9100\", job=\"kafka\", mountpoint=\"/\"}",
+ "targets": [
+ {
+ "expr": "node_filesystem_size_bytes{instance=~\"$node:$port\",job=~\"$job\",mountpoint=\"/\",fstype!=\"rootfs\"}",
+ "format": "time_series",
+ "hide": false,
+ "intervalFactor": 1,
+ "refId": "A",
+ "step": 900
+ }
+ ],
+ "thresholds": "70,90",
+ "title": "RootFS Total",
+ "type": "singlestat",
+ "valueFontSize": "50%",
+ "valueMaps": [
+ {
+ "op": "=",
+ "text": "N/A",
+ "value": "null"
+ }
+ ],
+ "valueName": "current"
+ },
+ {
+ "cacheTimeout": null,
+ "colorBackground": false,
+ "colorValue": false,
+ "colors": [
+ "rgba(245, 54, 54, 0.9)",
+ "rgba(237, 129, 40, 0.89)",
+ "rgba(50, 172, 45, 0.97)"
+ ],
+ "datasource": "${DS_PROMETHEUS}",
+ "decimals": 0,
+ "description": "Total RAM",
+ "fieldConfig": {
+ "defaults": {
+ "custom": {}
+ },
+ "overrides": []
+ },
+ "format": "bytes",
+ "gauge": {
+ "maxValue": 100,
+ "minValue": 0,
+ "show": false,
+ "thresholdLabels": false,
+ "thresholdMarkers": true
+ },
+ "gridPos": {
+ "h": 2,
+ "w": 2,
+ "x": 20,
+ "y": 11
+ },
+ "id": 75,
+ "interval": null,
+ "links": [],
+ "mappingType": 1,
+ "mappingTypes": [
+ {
+ "name": "value to text",
+ "value": 1
+ },
+ {
+ "name": "range to text",
+ "value": 2
+ }
+ ],
+ "maxDataPoints": 100,
+ "maxPerRow": 6,
+ "nullPointMode": "null",
+ "nullText": null,
+ "postfix": "",
+ "postfixFontSize": "70%",
+ "prefix": "",
+ "prefixFontSize": "50%",
+ "rangeMaps": [
+ {
+ "from": "null",
+ "text": "N/A",
+ "to": "null"
+ }
+ ],
+ "sparkline": {
+ "fillColor": "rgba(31, 118, 189, 0.18)",
+ "full": false,
+ "lineColor": "rgb(31, 120, 193)",
+ "show": false
+ },
+ "tableColumn": "node_memory_MemTotal_bytes{group=\"broker-os\", instance=\"10.0.0.15:9100\", job=\"kafka\"}",
+ "targets": [
+ {
+ "expr": "node_memory_MemTotal_bytes{instance=~\"$node:$port\",job=~\"$job\"}",
+ "intervalFactor": 1,
+ "refId": "A",
+ "step": 900
+ }
+ ],
+ "thresholds": "",
+ "title": "RAM Total",
+ "type": "singlestat",
+ "valueFontSize": "50%",
+ "valueMaps": [
+ {
+ "op": "=",
+ "text": "N/A",
+ "value": "null"
+ }
+ ],
+ "valueName": "current"
+ },
+ {
+ "cacheTimeout": null,
+ "colorBackground": false,
+ "colorValue": false,
+ "colors": [
+ "rgba(245, 54, 54, 0.9)",
+ "rgba(237, 129, 40, 0.89)",
+ "rgba(50, 172, 45, 0.97)"
+ ],
+ "datasource": "${DS_PROMETHEUS}",
+ "decimals": 0,
+ "description": "Total SWAP",
+ "fieldConfig": {
+ "defaults": {
+ "custom": {}
+ },
+ "overrides": []
+ },
+ "format": "bytes",
+ "gauge": {
+ "maxValue": 100,
+ "minValue": 0,
+ "show": false,
+ "thresholdLabels": false,
+ "thresholdMarkers": true
+ },
+ "gridPos": {
+ "h": 2,
+ "w": 2,
+ "x": 22,
+ "y": 11
+ },
+ "id": 18,
+ "interval": null,
+ "links": [],
+ "mappingType": 1,
+ "mappingTypes": [
+ {
+ "name": "value to text",
+ "value": 1
+ },
+ {
+ "name": "range to text",
+ "value": 2
+ }
+ ],
+ "maxDataPoints": 100,
+ "maxPerRow": 6,
+ "nullPointMode": "null",
+ "nullText": null,
+ "postfix": "",
+ "postfixFontSize": "70%",
+ "prefix": "",
+ "prefixFontSize": "50%",
+ "rangeMaps": [
+ {
+ "from": "null",
+ "text": "N/A",
+ "to": "null"
+ }
+ ],
+ "sparkline": {
+ "fillColor": "rgba(31, 118, 189, 0.18)",
+ "full": false,
+ "lineColor": "rgb(31, 120, 193)",
+ "show": false
+ },
+ "tableColumn": "node_memory_SwapTotal_bytes{group=\"broker-os\", instance=\"10.0.0.15:9100\", job=\"kafka\"}",
+ "targets": [
+ {
+ "expr": "node_memory_SwapTotal_bytes{instance=~\"$node:$port\",job=~\"$job\"}",
+ "intervalFactor": 1,
+ "refId": "A",
+ "step": 900
+ }
+ ],
+ "thresholds": "",
+ "title": "SWAP Total",
+ "type": "singlestat",
+ "valueFontSize": "50%",
+ "valueMaps": [
+ {
+ "op": "=",
+ "text": "N/A",
+ "value": "null"
+ }
+ ],
+ "valueName": "current"
+ }
+ ],
+ "repeat": null,
+ "title": "Quick CPU / Mem / Disk",
+ "type": "row"
+ },
+ {
+ "collapsed": true,
+ "datasource": "${DS_PROMETHEUS}",
+ "gridPos": {
+ "h": 1,
+ "w": 24,
+ "x": 0,
+ "y": 9
+ },
+ "id": 263,
+ "panels": [
+ {
+ "aliasColors": {
+ "Busy": "#EAB839",
+ "Busy Iowait": "#890F02",
+ "Busy other": "#1F78C1",
+ "Idle": "#052B51",
+ "Idle - Waiting for something to happen": "#052B51",
+ "guest": "#9AC48A",
+ "idle": "#052B51",
+ "iowait": "#EAB839",
+ "irq": "#BF1B00",
+ "nice": "#C15C17",
+ "softirq": "#E24D42",
+ "steal": "#FCE2DE",
+ "system": "#508642",
+ "user": "#5195CE"
+ },
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "decimals": 2,
+ "description": "Basic CPU info",
+ "fieldConfig": {
+ "defaults": {
+ "custom": {}
+ },
+ "overrides": []
+ },
+ "fill": 4,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 7,
+ "w": 12,
+ "x": 0,
+ "y": 10
+ },
+ "hiddenSeries": false,
+ "id": 77,
+ "legend": {
+ "alignAsTable": false,
+ "avg": false,
+ "current": false,
+ "max": false,
+ "min": false,
+ "rightSide": false,
+ "show": true,
+ "sideWidth": 250,
+ "sort": null,
+ "sortDesc": null,
+ "total": false,
+ "values": false
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "maxPerRow": 6,
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": true,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [
+ {
+ "alias": "Busy Iowait",
+ "color": "#890F02"
+ },
+ {
+ "alias": "Idle",
+ "color": "#7EB26D"
+ },
+ {
+ "alias": "Busy System",
+ "color": "#EAB839"
+ },
+ {
+ "alias": "Busy User",
+ "color": "#0A437C"
+ },
+ {
+ "alias": "Busy Other",
+ "color": "#6D1F62"
+ }
+ ],
+ "spaceLength": 10,
+ "stack": true,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "sum by (instance)(irate(node_cpu_seconds_total{mode=\"system\",instance=~\"$node:$port\",job=~\"$job\"}[5m])) * 100",
+ "format": "time_series",
+ "hide": false,
+ "intervalFactor": 2,
+ "legendFormat": "Busy System",
+ "refId": "A",
+ "step": 240
+ },
+ {
+ "expr": "sum by (instance)(irate(node_cpu_seconds_total{mode='user',instance=~\"$node:$port\",job=~\"$job\"}[5m])) * 100",
+ "format": "time_series",
+ "hide": false,
+ "intervalFactor": 2,
+ "legendFormat": "Busy User",
+ "refId": "B",
+ "step": 240
+ },
+ {
+ "expr": "sum by (instance)(irate(node_cpu_seconds_total{mode='iowait',instance=~\"$node:$port\",job=~\"$job\"}[5m])) * 100",
+ "format": "time_series",
+ "intervalFactor": 2,
+ "legendFormat": "Busy Iowait",
+ "refId": "C",
+ "step": 240
+ },
+ {
+ "expr": "sum by (instance)(irate(node_cpu_seconds_total{mode=~\".*irq\",instance=~\"$node:$port\",job=~\"$job\"}[5m])) * 100",
+ "format": "time_series",
+ "intervalFactor": 2,
+ "legendFormat": "Busy IRQs",
+ "refId": "D",
+ "step": 240
+ },
+ {
+ "expr": "sum (irate(node_cpu_seconds_total{mode!='idle',mode!='user',mode!='system',mode!='iowait',mode!='irq',mode!='softirq',instance=~\"$node:$port\",job=~\"$job\"}[5m])) * 100",
+ "format": "time_series",
+ "intervalFactor": 2,
+ "legendFormat": "Busy Other",
+ "refId": "E",
+ "step": 240
+ },
+ {
+ "expr": "sum by (mode)(irate(node_cpu_seconds_total{mode='idle',instance=~\"$node:$port\",job=~\"$job\"}[5m])) * 100",
+ "format": "time_series",
+ "intervalFactor": 2,
+ "legendFormat": "Idle",
+ "refId": "F",
+ "step": 240
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "CPU Basic",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "short",
+ "label": "",
+ "logBase": 1,
+ "max": "100",
+ "min": "0",
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {
+ "Apps": "#629E51",
+ "Buffers": "#614D93",
+ "Cache": "#6D1F62",
+ "Cached": "#511749",
+ "Committed": "#508642",
+ "Free": "#0A437C",
+ "Hardware Corrupted - Amount of RAM that the kernel identified as corrupted / not working": "#CFFAFF",
+ "Inactive": "#584477",
+ "PageTables": "#0A50A1",
+ "Page_Tables": "#0A50A1",
+ "RAM_Free": "#E0F9D7",
+ "SWAP Used": "#BF1B00",
+ "Slab": "#806EB7",
+ "Slab_Cache": "#E0752D",
+ "Swap": "#BF1B00",
+ "Swap Used": "#BF1B00",
+ "Swap_Cache": "#C15C17",
+ "Swap_Free": "#2F575E",
+ "Unused": "#EAB839"
+ },
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "decimals": 2,
+ "description": "Basic memory usage",
+ "fieldConfig": {
+ "defaults": {
+ "custom": {}
+ },
+ "overrides": []
+ },
+ "fill": 4,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 7,
+ "w": 12,
+ "x": 12,
+ "y": 10
+ },
+ "hiddenSeries": false,
+ "id": 78,
+ "legend": {
+ "alignAsTable": false,
+ "avg": false,
+ "current": false,
+ "max": false,
+ "min": false,
+ "rightSide": false,
+ "show": true,
+ "sideWidth": 350,
+ "total": false,
+ "values": false
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "maxPerRow": 6,
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [
+ {
+ "alias": "RAM Total",
+ "color": "#E0F9D7",
+ "fill": 0,
+ "stack": false
+ },
+ {
+ "alias": "RAM Cache + Buffer",
+ "color": "#052B51"
+ },
+ {
+ "alias": "RAM Free",
+ "color": "#7EB26D"
+ },
+ {
+ "alias": "Avaliable",
+ "color": "#DEDAF7",
+ "fill": 0,
+ "stack": false
+ }
+ ],
+ "spaceLength": 10,
+ "stack": true,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "node_memory_MemTotal_bytes{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "hide": false,
+ "intervalFactor": 2,
+ "legendFormat": "RAM Total",
+ "refId": "A",
+ "step": 240
+ },
+ {
+ "expr": "node_memory_MemTotal_bytes{instance=~\"$node:$port\",job=~\"$job\"} - node_memory_MemFree_bytes{instance=~\"$node:$port\",job=~\"$job\"} - (node_memory_Cached_bytes{instance=~\"$node:$port\",job=~\"$job\"} + node_memory_Buffers_bytes{instance=~\"$node:$port\",job=~\"$job\"})",
+ "format": "time_series",
+ "hide": false,
+ "intervalFactor": 2,
+ "legendFormat": "RAM Used",
+ "refId": "B",
+ "step": 240
+ },
+ {
+ "expr": "node_memory_Cached_bytes{instance=~\"$node:$port\",job=~\"$job\"} + node_memory_Buffers_bytes{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "intervalFactor": 2,
+ "legendFormat": "RAM Cache + Buffer",
+ "refId": "C",
+ "step": 240
+ },
+ {
+ "expr": "node_memory_MemFree_bytes{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "intervalFactor": 2,
+ "legendFormat": "RAM Free",
+ "refId": "D",
+ "step": 240
+ },
+ {
+ "expr": "(node_memory_SwapTotal_bytes{instance=~\"$node:$port\",job=~\"$job\"} - node_memory_SwapFree_bytes{instance=~\"$node:$port\",job=~\"$job\"})",
+ "format": "time_series",
+ "intervalFactor": 2,
+ "legendFormat": "SWAP Used",
+ "refId": "E",
+ "step": 240
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "Memory Basic",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "bytes",
+ "label": "",
+ "logBase": 1,
+ "max": null,
+ "min": "0",
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {
+ "Recv_bytes_eth2": "#7EB26D",
+ "Recv_bytes_lo": "#0A50A1",
+ "Recv_drop_eth2": "#6ED0E0",
+ "Recv_drop_lo": "#E0F9D7",
+ "Recv_errs_eth2": "#BF1B00",
+ "Recv_errs_lo": "#CCA300",
+ "Trans_bytes_eth2": "#7EB26D",
+ "Trans_bytes_lo": "#0A50A1",
+ "Trans_drop_eth2": "#6ED0E0",
+ "Trans_drop_lo": "#E0F9D7",
+ "Trans_errs_eth2": "#BF1B00",
+ "Trans_errs_lo": "#CCA300",
+ "recv_bytes_lo": "#0A50A1",
+ "recv_drop_eth0": "#99440A",
+ "recv_drop_lo": "#967302",
+ "recv_errs_eth0": "#BF1B00",
+ "recv_errs_lo": "#890F02",
+ "trans_bytes_eth0": "#7EB26D",
+ "trans_bytes_lo": "#0A50A1",
+ "trans_drop_eth0": "#99440A",
+ "trans_drop_lo": "#967302",
+ "trans_errs_eth0": "#BF1B00",
+ "trans_errs_lo": "#890F02"
+ },
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "description": "Basic network info per interface",
+ "fieldConfig": {
+ "defaults": {
+ "custom": {}
+ },
+ "overrides": []
+ },
+ "fill": 4,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 7,
+ "w": 12,
+ "x": 0,
+ "y": 17
+ },
+ "hiddenSeries": false,
+ "id": 74,
+ "legend": {
+ "alignAsTable": false,
+ "avg": false,
+ "current": false,
+ "hideEmpty": false,
+ "hideZero": false,
+ "max": false,
+ "min": false,
+ "rightSide": false,
+ "show": true,
+ "sort": "current",
+ "sortDesc": true,
+ "total": false,
+ "values": false
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [
+ {
+ "alias": "/.*trans.*/",
+ "transform": "negative-Y"
+ }
+ ],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "irate(node_network_receive_bytes_total{instance=~\"$node:$port\",job=~\"$job\"}[5m])*8",
+ "format": "time_series",
+ "intervalFactor": 2,
+ "legendFormat": "recv {{device}}",
+ "refId": "A",
+ "step": 240
+ },
+ {
+ "expr": "irate(node_network_transmit_bytes_total{instance=~\"$node:$port\",job=~\"$job\"}[5m])*8",
+ "format": "time_series",
+ "intervalFactor": 2,
+ "legendFormat": "trans {{device}} ",
+ "refId": "B",
+ "step": 240
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "Network Traffic Basic",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "bps",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ },
+ {
+ "format": "pps",
+ "label": "",
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "decimals": 3,
+ "description": "Disk space used of all filesystems mounted",
+ "fieldConfig": {
+ "defaults": {
+ "custom": {}
+ },
+ "overrides": []
+ },
+ "fill": 4,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 7,
+ "w": 12,
+ "x": 12,
+ "y": 17
+ },
+ "height": "",
+ "hiddenSeries": false,
+ "id": 152,
+ "legend": {
+ "alignAsTable": false,
+ "avg": false,
+ "current": false,
+ "max": false,
+ "min": false,
+ "rightSide": false,
+ "show": true,
+ "sort": "current",
+ "sortDesc": false,
+ "total": false,
+ "values": false
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "maxPerRow": 6,
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "100 - ((node_filesystem_avail_bytes{instance=~\"$node:$port\",job=~\"$job\",device!~'rootfs'} * 100) / node_filesystem_size_bytes{instance=~\"$node:$port\",job=~\"$job\",device!~'rootfs'})",
+ "format": "time_series",
+ "intervalFactor": 2,
+ "legendFormat": "{{mountpoint}}",
+ "refId": "A",
+ "step": 240
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "Disk Space Used Basic",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "percent",
+ "label": null,
+ "logBase": 1,
+ "max": "100",
+ "min": "0",
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ }
+ ],
+ "repeat": null,
+ "title": "Basic CPU / Mem / Net / Disk",
+ "type": "row"
+ },
+ {
+ "collapsed": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "gridPos": {
+ "h": 1,
+ "w": 24,
+ "x": 0,
+ "y": 10
+ },
+ "id": 265,
+ "panels": [],
+ "repeat": null,
+ "title": "CPU / Memory / Net / Disk",
+ "type": "row"
+ },
+ {
+ "aliasColors": {
+ "Idle - Waiting for something to happen": "#052B51",
+ "guest": "#9AC48A",
+ "idle": "#052B51",
+ "iowait": "#EAB839",
+ "irq": "#BF1B00",
+ "nice": "#C15C17",
+ "softirq": "#E24D42",
+ "steal": "#FCE2DE",
+ "system": "#508642",
+ "user": "#5195CE"
+ },
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "decimals": 2,
+ "description": "",
+ "fieldConfig": {
+ "defaults": {
+ "custom": {}
+ },
+ "overrides": []
+ },
+ "fill": 4,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 12,
+ "w": 12,
+ "x": 0,
+ "y": 11
+ },
+ "hiddenSeries": false,
+ "id": 3,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "max": true,
+ "min": true,
+ "rightSide": false,
+ "show": true,
+ "sideWidth": 250,
+ "sort": null,
+ "sortDesc": null,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "maxPerRow": 6,
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": true,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "repeat": null,
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "stack": true,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "sum by (mode)(irate(node_cpu_seconds_total{mode=\"system\",instance=~\"$node:$port\",job=~\"$job\"}[5m])) * 100",
+ "format": "time_series",
+ "interval": "10s",
+ "intervalFactor": 2,
+ "legendFormat": "System - Processes executing in kernel mode",
+ "refId": "A",
+ "step": 20
+ },
+ {
+ "expr": "sum by (mode)(irate(node_cpu_seconds_total{mode='user',instance=~\"$node:$port\",job=~\"$job\"}[5m])) * 100",
+ "format": "time_series",
+ "intervalFactor": 2,
+ "legendFormat": "User - Normal processes executing in user mode",
+ "refId": "B",
+ "step": 240
+ },
+ {
+ "expr": "sum by (mode)(irate(node_cpu_seconds_total{mode='nice',instance=~\"$node:$port\",job=~\"$job\"}[5m])) * 100",
+ "format": "time_series",
+ "intervalFactor": 2,
+ "legendFormat": "Nice - Niced processes executing in user mode",
+ "refId": "C",
+ "step": 240
+ },
+ {
+ "expr": "sum by (mode)(irate(node_cpu_seconds_total{mode='idle',instance=~\"$node:$port\",job=~\"$job\"}[5m])) * 100",
+ "format": "time_series",
+ "intervalFactor": 2,
+ "legendFormat": "Idle - Waiting for something to happen",
+ "refId": "D",
+ "step": 240
+ },
+ {
+ "expr": "sum by (mode)(irate(node_cpu_seconds_total{mode='iowait',instance=~\"$node:$port\",job=~\"$job\"}[5m])) * 100",
+ "format": "time_series",
+ "intervalFactor": 2,
+ "legendFormat": "Iowait - Waiting for I/O to complete",
+ "refId": "E",
+ "step": 240
+ },
+ {
+ "expr": "sum by (mode)(irate(node_cpu_seconds_total{mode='irq',instance=~\"$node:$port\",job=~\"$job\"}[5m])) * 100",
+ "format": "time_series",
+ "intervalFactor": 2,
+ "legendFormat": "Irq - Servicing interrupts",
+ "refId": "F",
+ "step": 240
+ },
+ {
+ "expr": "sum by (mode)(irate(node_cpu_seconds_total{mode='softirq',instance=~\"$node:$port\",job=~\"$job\"}[5m])) * 100",
+ "format": "time_series",
+ "intervalFactor": 2,
+ "legendFormat": "Softirq - Servicing softirqs",
+ "refId": "G",
+ "step": 240
+ },
+ {
+ "expr": "sum by (mode)(irate(node_cpu_seconds_total{mode='steal',instance=~\"$node:$port\",job=~\"$job\"}[5m])) * 100",
+ "format": "time_series",
+ "intervalFactor": 2,
+ "legendFormat": "Steal - Time spent in other operating systems when running in a virtualized environment",
+ "refId": "H",
+ "step": 240
+ },
+ {
+ "expr": "sum by (mode)(irate(node_cpu_seconds_total{mode='guest',instance=~\"$node:$port\",job=~\"$job\"}[5m])) * 100",
+ "format": "time_series",
+ "intervalFactor": 2,
+ "legendFormat": "Guest - Time spent running a virtual CPU for a guest operating system",
+ "refId": "I",
+ "step": 240
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "CPU",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "short",
+ "label": "percentage",
+ "logBase": 1,
+ "max": "100",
+ "min": "0",
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {
+ "Apps": "#629E51",
+ "Buffers": "#614D93",
+ "Cache": "#6D1F62",
+ "Cached": "#511749",
+ "Committed": "#508642",
+ "Free": "#0A437C",
+ "Hardware Corrupted - Amount of RAM that the kernel identified as corrupted / not working": "#CFFAFF",
+ "Inactive": "#584477",
+ "PageTables": "#0A50A1",
+ "Page_Tables": "#0A50A1",
+ "RAM_Free": "#E0F9D7",
+ "Slab": "#806EB7",
+ "Slab_Cache": "#E0752D",
+ "Swap": "#BF1B00",
+ "Swap - Swap memory usage": "#BF1B00",
+ "Swap_Cache": "#C15C17",
+ "Swap_Free": "#2F575E",
+ "Unused": "#EAB839",
+ "Unused - Free memory unassigned": "#052B51"
+ },
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "decimals": 2,
+ "description": "",
+ "fieldConfig": {
+ "defaults": {
+ "custom": {}
+ },
+ "overrides": []
+ },
+ "fill": 4,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 12,
+ "w": 12,
+ "x": 12,
+ "y": 11
+ },
+ "hiddenSeries": false,
+ "id": 24,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "max": true,
+ "min": true,
+ "rightSide": false,
+ "show": true,
+ "sideWidth": 350,
+ "sort": null,
+ "sortDesc": null,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "maxPerRow": 6,
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [
+ {
+ "alias": "/.*Hardware Corrupted - *./",
+ "stack": false
+ }
+ ],
+ "spaceLength": 10,
+ "stack": true,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "node_memory_MemTotal_bytes{instance=~\"$node:$port\",job=~\"$job\"} - node_memory_MemFree_bytes{instance=~\"$node:$port\",job=~\"$job\"} - node_memory_Buffers_bytes{instance=~\"$node:$port\",job=~\"$job\"} - node_memory_Cached_bytes{instance=~\"$node:$port\",job=~\"$job\"} - node_memory_Slab_bytes{instance=~\"$node:$port\",job=~\"$job\"} - node_memory_PageTables_bytes{instance=~\"$node:$port\",job=~\"$job\"} - node_memory_SwapCached_bytes{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "hide": false,
+ "intervalFactor": 2,
+ "legendFormat": "Apps - Memory used by user-space applications",
+ "refId": "A",
+ "step": 240
+ },
+ {
+ "expr": "node_memory_PageTables_bytes{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "hide": false,
+ "intervalFactor": 2,
+ "legendFormat": "PageTables - Memory used to map between virtual and physical memory addresses",
+ "refId": "B",
+ "step": 240
+ },
+ {
+ "expr": "node_memory_SwapCached_bytes{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "intervalFactor": 2,
+ "legendFormat": "SwapCache - Memory that keeps track of pages that have been fetched from swap but not yet been modified",
+ "refId": "C",
+ "step": 240
+ },
+ {
+ "expr": "node_memory_Slab_bytes{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "hide": false,
+ "intervalFactor": 2,
+ "legendFormat": "Slab - Memory used by the kernel to cache data structures for its own use (caches like inode, dentry, etc)",
+ "refId": "D",
+ "step": 240
+ },
+ {
+ "expr": "node_memory_Cached_bytes{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "hide": false,
+ "intervalFactor": 2,
+ "legendFormat": "Cache - Parked file data (file content) cache",
+ "refId": "E",
+ "step": 240
+ },
+ {
+ "expr": "node_memory_Buffers_bytes{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "hide": false,
+ "intervalFactor": 2,
+ "legendFormat": "Buffers - Block device (e.g. harddisk) cache",
+ "refId": "F",
+ "step": 240
+ },
+ {
+ "expr": "node_memory_MemFree_bytes{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "hide": false,
+ "intervalFactor": 2,
+ "legendFormat": "Unused - Free memory unassigned",
+ "refId": "G",
+ "step": 240
+ },
+ {
+ "expr": "(node_memory_SwapTotal_bytes{instance=~\"$node:$port\",job=~\"$job\"} - node_memory_SwapFree_bytes{instance=~\"$node:$port\",job=~\"$job\"})",
+ "format": "time_series",
+ "hide": false,
+ "intervalFactor": 2,
+ "legendFormat": "Swap - Swap space used",
+ "refId": "H",
+ "step": 240
+ },
+ {
+ "expr": "node_memory_HardwareCorrupted_bytes{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "hide": false,
+ "intervalFactor": 2,
+ "legendFormat": "Hardware Corrupted - Amount of RAM that the kernel identified as corrupted / not working",
+ "refId": "I",
+ "step": 240
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "Memory Stack",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "bytes",
+ "label": "bytes",
+ "logBase": 1,
+ "max": null,
+ "min": "0",
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {
+ "receive_packets_eth0": "#7EB26D",
+ "receive_packets_lo": "#E24D42",
+ "transmit_packets_eth0": "#7EB26D",
+ "transmit_packets_lo": "#E24D42"
+ },
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "fieldConfig": {
+ "defaults": {
+ "custom": {}
+ },
+ "overrides": []
+ },
+ "fill": 4,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 12,
+ "w": 12,
+ "x": 0,
+ "y": 23
+ },
+ "hiddenSeries": false,
+ "id": 84,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "max": true,
+ "min": true,
+ "rightSide": false,
+ "show": true,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [
+ {
+ "alias": "/.*Trans.*/",
+ "transform": "negative-Y"
+ }
+ ],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "irate(node_network_receive_bytes_total{instance=~\"$node:$port\",job=~\"$job\"}[5m])*8",
+ "format": "time_series",
+ "intervalFactor": 2,
+ "legendFormat": "{{device}} - Receive",
+ "refId": "A",
+ "step": 240
+ },
+ {
+ "expr": "irate(node_network_transmit_bytes_total{instance=~\"$node:$port\",job=~\"$job\"}[5m])*8",
+ "format": "time_series",
+ "intervalFactor": 2,
+ "legendFormat": "{{device}} - Transmit",
+ "refId": "B",
+ "step": 240
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "Network Traffic",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "bps",
+ "label": "bits out (-) / in (+)",
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "decimals": 3,
+ "description": "",
+ "fieldConfig": {
+ "defaults": {
+ "custom": {}
+ },
+ "overrides": []
+ },
+ "fill": 4,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 12,
+ "w": 12,
+ "x": 12,
+ "y": 23
+ },
+ "height": "",
+ "hiddenSeries": false,
+ "id": 156,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "max": true,
+ "min": true,
+ "rightSide": false,
+ "show": true,
+ "sort": "current",
+ "sortDesc": false,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "maxPerRow": 6,
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "node_filesystem_size_bytes{instance=~\"$node:$port\",job=~\"$job\",device!~'rootfs'} - node_filesystem_avail_bytes{instance=~\"$node:$port\",job=~\"$job\",device!~'rootfs'}",
+ "format": "time_series",
+ "intervalFactor": 2,
+ "legendFormat": "{{mountpoint}}",
+ "refId": "A",
+ "step": 240
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "Disk Space Used",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "bytes",
+ "label": "bytes",
+ "logBase": 1,
+ "max": null,
+ "min": "0",
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "Prometheus",
+ "description": "",
+ "fieldConfig": {
+ "defaults": {
+ "custom": {}
+ },
+ "overrides": []
+ },
+ "fill": 2,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 12,
+ "w": 12,
+ "x": 0,
+ "y": 35
+ },
+ "hiddenSeries": false,
+ "id": 229,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "hideZero": true,
+ "max": true,
+ "min": true,
+ "rightSide": false,
+ "show": true,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "maxPerRow": 6,
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [
+ {
+ "alias": "/.*Read.*/",
+ "transform": "negative-Y"
+ },
+ {
+ "alias": "/.*sda_.*/",
+ "color": "#7EB26D"
+ },
+ {
+ "alias": "/.*sdb_.*/",
+ "color": "#EAB839"
+ },
+ {
+ "alias": "/.*sdc_.*/",
+ "color": "#6ED0E0"
+ },
+ {
+ "alias": "/.*sdd_.*/",
+ "color": "#EF843C"
+ },
+ {
+ "alias": "/.*sde_.*/",
+ "color": "#E24D42"
+ },
+ {
+ "alias": "/.*sda1.*/",
+ "color": "#584477"
+ },
+ {
+ "alias": "/.*sda2_.*/",
+ "color": "#BA43A9"
+ },
+ {
+ "alias": "/.*sda3_.*/",
+ "color": "#F4D598"
+ },
+ {
+ "alias": "/.*sdb1.*/",
+ "color": "#0A50A1"
+ },
+ {
+ "alias": "/.*sdb2.*/",
+ "color": "#BF1B00"
+ },
+ {
+ "alias": "/.*sdb2.*/",
+ "color": "#BF1B00"
+ },
+ {
+ "alias": "/.*sdb3.*/",
+ "color": "#E0752D"
+ },
+ {
+ "alias": "/.*sdc1.*/",
+ "color": "#962D82"
+ },
+ {
+ "alias": "/.*sdc2.*/",
+ "color": "#614D93"
+ },
+ {
+ "alias": "/.*sdc3.*/",
+ "color": "#9AC48A"
+ },
+ {
+ "alias": "/.*sdd1.*/",
+ "color": "#65C5DB"
+ },
+ {
+ "alias": "/.*sdd2.*/",
+ "color": "#F9934E"
+ },
+ {
+ "alias": "/.*sdd3.*/",
+ "color": "#EA6460"
+ },
+ {
+ "alias": "/.*sde1.*/",
+ "color": "#E0F9D7"
+ },
+ {
+ "alias": "/.*sdd2.*/",
+ "color": "#FCEACA"
+ },
+ {
+ "alias": "/.*sde3.*/",
+ "color": "#F9E2D2"
+ }
+ ],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "irate(node_disk_reads_completed_total{instance=~\"$node:$port\"}[5m])",
+ "interval": "",
+ "intervalFactor": 4,
+ "legendFormat": "{{device}} - Reads completed",
+ "refId": "A",
+ "step": 480
+ },
+ {
+ "expr": "irate(node_disk_writes_completed_total{instance=~\"$node:$port\"}[5m])",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "{{device}} - Writes completed",
+ "refId": "B",
+ "step": 240
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "Disk IOps",
+ "tooltip": {
+ "shared": false,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "iops",
+ "label": "IO read (-) / write (+)",
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {
+ "io time": "#890F02"
+ },
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "decimals": 3,
+ "description": "",
+ "fieldConfig": {
+ "defaults": {
+ "custom": {}
+ },
+ "overrides": []
+ },
+ "fill": 4,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 12,
+ "w": 12,
+ "x": 12,
+ "y": 35
+ },
+ "hiddenSeries": false,
+ "id": 42,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "max": true,
+ "min": true,
+ "rightSide": false,
+ "show": true,
+ "sort": null,
+ "sortDesc": null,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "maxPerRow": 6,
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [
+ {
+ "alias": "/.*read*./",
+ "transform": "negative-Y"
+ },
+ {
+ "alias": "/.*sda.*/",
+ "color": "#7EB26D"
+ },
+ {
+ "alias": "/.*sdb.*/",
+ "color": "#EAB839"
+ },
+ {
+ "alias": "/.*sdc.*/",
+ "color": "#6ED0E0"
+ },
+ {
+ "alias": "/.*sdd.*/",
+ "color": "#EF843C"
+ },
+ {
+ "alias": "/.*sde.*/",
+ "color": "#E24D42"
+ }
+ ],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "irate(node_disk_read_bytes_total{instance=~\"$node:$port\"}[5m])",
+ "format": "time_series",
+ "hide": false,
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "{{device}} - Successfully read bytes",
+ "refId": "A",
+ "step": 240
+ },
+ {
+ "expr": "irate(node_disk_written_bytes_total{instance=~\"$node:$port\"}[5m])",
+ "format": "time_series",
+ "hide": false,
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "{{device}} - Successfully written bytes",
+ "refId": "B",
+ "step": 240
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "I/O Usage Read / Write",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": false,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "bytes",
+ "label": "bytes read (-) / write (+)",
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ },
+ {
+ "format": "ms",
+ "label": "",
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {
+ "io time": "#890F02"
+ },
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "decimals": 3,
+ "description": "",
+ "fieldConfig": {
+ "defaults": {
+ "custom": {}
+ },
+ "overrides": []
+ },
+ "fill": 4,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 12,
+ "w": 12,
+ "x": 0,
+ "y": 47
+ },
+ "hiddenSeries": false,
+ "id": 127,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "max": true,
+ "min": true,
+ "rightSide": false,
+ "show": true,
+ "sort": null,
+ "sortDesc": null,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "maxPerRow": 6,
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "irate(node_disk_io_time_seconds_total{instance=~\"$node:$port\"} [5m])",
+ "format": "time_series",
+ "hide": false,
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "{{device}} - Time spent doing I/Os",
+ "refId": "A",
+ "step": 240
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "I/O Usage Times",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": false,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "s",
+ "label": "time",
+ "logBase": 1,
+ "max": null,
+ "min": "0",
+ "show": true
+ },
+ {
+ "format": "s",
+ "label": "",
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "collapsed": true,
+ "datasource": "${DS_PROMETHEUS}",
+ "gridPos": {
+ "h": 1,
+ "w": 24,
+ "x": 0,
+ "y": 59
+ },
+ "id": 266,
+ "panels": [
+ {
+ "aliasColors": {
+ "Apps": "#629E51",
+ "Buffers": "#614D93",
+ "Cache": "#6D1F62",
+ "Cached": "#511749",
+ "Committed": "#508642",
+ "Free": "#0A437C",
+ "Hardware Corrupted - Amount of RAM that the kernel identified as corrupted / not working": "#CFFAFF",
+ "Inactive": "#584477",
+ "PageTables": "#0A50A1",
+ "Page_Tables": "#0A50A1",
+ "RAM_Free": "#E0F9D7",
+ "Slab": "#806EB7",
+ "Slab_Cache": "#E0752D",
+ "Swap": "#BF1B00",
+ "Swap_Cache": "#C15C17",
+ "Swap_Free": "#2F575E",
+ "Unused": "#EAB839"
+ },
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "decimals": 2,
+ "fieldConfig": {
+ "defaults": {
+ "custom": {}
+ },
+ "overrides": []
+ },
+ "fill": 2,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 0,
+ "y": 12
+ },
+ "hiddenSeries": false,
+ "id": 136,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "max": true,
+ "min": true,
+ "rightSide": false,
+ "show": true,
+ "sideWidth": 350,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "maxPerRow": 2,
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "stack": true,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "node_memory_Inactive_bytes{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "intervalFactor": 2,
+ "legendFormat": "Inactive - Memory which has been less recently used. It is more eligible to be reclaimed for other purposes",
+ "refId": "A",
+ "step": 4
+ },
+ {
+ "expr": "node_memory_Active_bytes{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "intervalFactor": 2,
+ "legendFormat": "Active - Memory that has been used more recently and usually not reclaimed unless absolutely necessary",
+ "refId": "B",
+ "step": 4
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "Memory Active / Inactive",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "cumulative"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "bytes",
+ "label": "bytes",
+ "logBase": 1,
+ "max": null,
+ "min": "0",
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {
+ "Apps": "#629E51",
+ "Buffers": "#614D93",
+ "Cache": "#6D1F62",
+ "Cached": "#511749",
+ "Committed": "#508642",
+ "Free": "#0A437C",
+ "Hardware Corrupted - Amount of RAM that the kernel identified as corrupted / not working": "#CFFAFF",
+ "Inactive": "#584477",
+ "PageTables": "#0A50A1",
+ "Page_Tables": "#0A50A1",
+ "RAM_Free": "#E0F9D7",
+ "Slab": "#806EB7",
+ "Slab_Cache": "#E0752D",
+ "Swap": "#BF1B00",
+ "Swap_Cache": "#C15C17",
+ "Swap_Free": "#2F575E",
+ "Unused": "#EAB839"
+ },
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "decimals": 2,
+ "fieldConfig": {
+ "defaults": {
+ "custom": {}
+ },
+ "overrides": []
+ },
+ "fill": 2,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 12,
+ "y": 12
+ },
+ "hiddenSeries": false,
+ "id": 135,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "max": true,
+ "min": true,
+ "rightSide": false,
+ "show": true,
+ "sideWidth": 350,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "maxPerRow": 6,
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [
+ {
+ "alias": "/.*Committed_AS - *./"
+ },
+ {
+ "alias": "/.*CommitLimit - *./",
+ "color": "#BF1B00",
+ "fill": 0
+ }
+ ],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "node_memory_Committed_AS_bytes{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "intervalFactor": 2,
+ "legendFormat": "Committed_AS - Amount of memory presently allocated on the system",
+ "refId": "A",
+ "step": 4
+ },
+ {
+ "expr": "node_memory_CommitLimit_bytes{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "intervalFactor": 2,
+ "legendFormat": "CommitLimit - Amount of memory currently available to be allocated on the system",
+ "refId": "B",
+ "step": 4
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "Memory Commited",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "cumulative"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "bytes",
+ "label": "bytes",
+ "logBase": 1,
+ "max": null,
+ "min": "0",
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {
+ "Apps": "#629E51",
+ "Buffers": "#614D93",
+ "Cache": "#6D1F62",
+ "Cached": "#511749",
+ "Committed": "#508642",
+ "Free": "#0A437C",
+ "Hardware Corrupted - Amount of RAM that the kernel identified as corrupted / not working": "#CFFAFF",
+ "Inactive": "#584477",
+ "PageTables": "#0A50A1",
+ "Page_Tables": "#0A50A1",
+ "RAM_Free": "#E0F9D7",
+ "Slab": "#806EB7",
+ "Slab_Cache": "#E0752D",
+ "Swap": "#BF1B00",
+ "Swap_Cache": "#C15C17",
+ "Swap_Free": "#2F575E",
+ "Unused": "#EAB839"
+ },
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "decimals": 2,
+ "fieldConfig": {
+ "defaults": {
+ "custom": {}
+ },
+ "overrides": []
+ },
+ "fill": 2,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 0,
+ "y": 22
+ },
+ "hiddenSeries": false,
+ "id": 191,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "max": true,
+ "min": true,
+ "rightSide": false,
+ "show": true,
+ "sideWidth": 350,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "maxPerRow": 6,
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "stack": true,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "node_memory_Inactive_file_bytes{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "hide": false,
+ "intervalFactor": 2,
+ "legendFormat": "Inactive_file - File-backed memory on inactive LRU list",
+ "refId": "A",
+ "step": 4
+ },
+ {
+ "expr": "node_memory_Inactive_anon_bytes{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "hide": false,
+ "intervalFactor": 2,
+ "legendFormat": "Inactive_anon - Anonymous and swap cache on inactive LRU list, including tmpfs (shmem)",
+ "refId": "B",
+ "step": 4
+ },
+ {
+ "expr": "node_memory_Active_file_bytes{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "hide": false,
+ "intervalFactor": 2,
+ "legendFormat": "Active_file - File-backed memory on active LRU list",
+ "refId": "C",
+ "step": 4
+ },
+ {
+ "expr": "node_memory_Active_anon_bytes{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "hide": false,
+ "intervalFactor": 2,
+ "legendFormat": "Active_anon - Anonymous and swap cache on active least-recently-used (LRU) list, including tmpfs",
+ "refId": "D",
+ "step": 4
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "Memory Active / Inactive Detail",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "cumulative"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "bytes",
+ "label": "bytes",
+ "logBase": 1,
+ "max": null,
+ "min": "0",
+ "show": true
+ },
+ {
+ "format": "bytes",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {
+ "Active": "#99440A",
+ "Buffers": "#58140C",
+ "Cache": "#6D1F62",
+ "Cached": "#511749",
+ "Committed": "#508642",
+ "Dirty": "#6ED0E0",
+ "Free": "#B7DBAB",
+ "Inactive": "#EA6460",
+ "Mapped": "#052B51",
+ "PageTables": "#0A50A1",
+ "Page_Tables": "#0A50A1",
+ "Slab_Cache": "#EAB839",
+ "Swap": "#BF1B00",
+ "Swap_Cache": "#C15C17",
+ "Total": "#511749",
+ "Total RAM": "#052B51",
+ "Total RAM + Swap": "#052B51",
+ "Total Swap": "#614D93",
+ "VmallocUsed": "#EA6460"
+ },
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "decimals": 2,
+ "fieldConfig": {
+ "defaults": {
+ "custom": {}
+ },
+ "overrides": []
+ },
+ "fill": 2,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 12,
+ "y": 22
+ },
+ "hiddenSeries": false,
+ "id": 130,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "max": true,
+ "min": true,
+ "rightSide": false,
+ "show": true,
+ "sideWidth": null,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "maxPerRow": 2,
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "node_memory_Writeback_bytes{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "intervalFactor": 2,
+ "legendFormat": "Writeback - Memory which is actively being written back to disk",
+ "refId": "A",
+ "step": 4
+ },
+ {
+ "expr": "node_memory_WritebackTmp_bytes{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "intervalFactor": 2,
+ "legendFormat": "WritebackTmp - Memory used by FUSE for temporary writeback buffers",
+ "refId": "B",
+ "step": 4
+ },
+ {
+ "expr": "node_memory_Dirty_bytes{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "intervalFactor": 2,
+ "legendFormat": "Dirty - Memory which is waiting to get written back to the disk",
+ "refId": "C",
+ "step": 4
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "Memory Writeback and Dirty",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "bytes",
+ "label": "bytes",
+ "logBase": 1,
+ "max": null,
+ "min": "0",
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {
+ "Apps": "#629E51",
+ "Buffers": "#614D93",
+ "Cache": "#6D1F62",
+ "Cached": "#511749",
+ "Committed": "#508642",
+ "Free": "#0A437C",
+ "Hardware Corrupted - Amount of RAM that the kernel identified as corrupted / not working": "#CFFAFF",
+ "Inactive": "#584477",
+ "PageTables": "#0A50A1",
+ "Page_Tables": "#0A50A1",
+ "RAM_Free": "#E0F9D7",
+ "Slab": "#806EB7",
+ "Slab_Cache": "#E0752D",
+ "Swap": "#BF1B00",
+ "Swap_Cache": "#C15C17",
+ "Swap_Free": "#2F575E",
+ "Unused": "#EAB839"
+ },
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "decimals": 2,
+ "fieldConfig": {
+ "defaults": {
+ "custom": {}
+ },
+ "overrides": []
+ },
+ "fill": 2,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 0,
+ "y": 32
+ },
+ "hiddenSeries": false,
+ "id": 138,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "max": true,
+ "min": true,
+ "rightSide": false,
+ "show": true,
+ "sideWidth": 350,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "maxPerRow": 6,
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [
+ {
+ "alias": "ShmemHugePages - Memory used by shared memory (shmem) and tmpfs allocated with huge pages",
+ "fill": 0
+ },
+ {
+ "alias": "ShmemHugePages - Memory used by shared memory (shmem) and tmpfs allocated with huge pages",
+ "fill": 0
+ }
+ ],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "node_memory_Mapped_bytes{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "intervalFactor": 2,
+ "legendFormat": "Mapped - Used memory in mapped pages files which have been mmaped, such as libraries",
+ "refId": "A",
+ "step": 4
+ },
+ {
+ "expr": "node_memory_Shmem_bytes{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "intervalFactor": 2,
+ "legendFormat": "Shmem - Used shared memory (shared between several processes, thus including RAM disks)",
+ "refId": "B",
+ "step": 4
+ },
+ {
+ "expr": "node_memory_ShmemHugePages_bytes{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "ShmemHugePages - Memory used by shared memory (shmem) and tmpfs allocated with huge pages",
+ "refId": "C",
+ "step": 4
+ },
+ {
+ "expr": "node_memory_ShmemPmdMapped_bytes{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "ShmemPmdMapped - Ammount of shared (shmem/tmpfs) memory backed by huge pages",
+ "refId": "D",
+ "step": 4
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "Memory Shared and Mapped",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "cumulative"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "bytes",
+ "label": "bytes",
+ "logBase": 1,
+ "max": null,
+ "min": "0",
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {
+ "Active": "#99440A",
+ "Buffers": "#58140C",
+ "Cache": "#6D1F62",
+ "Cached": "#511749",
+ "Committed": "#508642",
+ "Dirty": "#6ED0E0",
+ "Free": "#B7DBAB",
+ "Inactive": "#EA6460",
+ "Mapped": "#052B51",
+ "PageTables": "#0A50A1",
+ "Page_Tables": "#0A50A1",
+ "Slab_Cache": "#EAB839",
+ "Swap": "#BF1B00",
+ "Swap_Cache": "#C15C17",
+ "Total": "#511749",
+ "Total RAM": "#052B51",
+ "Total RAM + Swap": "#052B51",
+ "Total Swap": "#614D93",
+ "VmallocUsed": "#EA6460"
+ },
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "decimals": 2,
+ "fieldConfig": {
+ "defaults": {
+ "custom": {}
+ },
+ "overrides": []
+ },
+ "fill": 2,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 12,
+ "y": 32
+ },
+ "hiddenSeries": false,
+ "id": 131,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "max": true,
+ "min": true,
+ "rightSide": false,
+ "show": true,
+ "sideWidth": null,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "maxPerRow": 2,
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "stack": true,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "node_memory_SUnreclaim_bytes{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "intervalFactor": 2,
+ "legendFormat": "SUnreclaim - Part of Slab, that cannot be reclaimed on memory pressure",
+ "refId": "A",
+ "step": 4
+ },
+ {
+ "expr": "node_memory_SReclaimable_bytes{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "intervalFactor": 2,
+ "legendFormat": "SReclaimable - Part of Slab, that might be reclaimed, such as caches",
+ "refId": "B",
+ "step": 4
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "Memory Slab",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "bytes",
+ "label": "bytes",
+ "logBase": 1,
+ "max": null,
+ "min": "0",
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {
+ "Active": "#99440A",
+ "Buffers": "#58140C",
+ "Cache": "#6D1F62",
+ "Cached": "#511749",
+ "Committed": "#508642",
+ "Dirty": "#6ED0E0",
+ "Free": "#B7DBAB",
+ "Inactive": "#EA6460",
+ "Mapped": "#052B51",
+ "PageTables": "#0A50A1",
+ "Page_Tables": "#0A50A1",
+ "Slab_Cache": "#EAB839",
+ "Swap": "#BF1B00",
+ "Swap_Cache": "#C15C17",
+ "Total": "#511749",
+ "Total RAM": "#052B51",
+ "Total RAM + Swap": "#052B51",
+ "VmallocUsed": "#EA6460"
+ },
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "decimals": 2,
+ "fieldConfig": {
+ "defaults": {
+ "custom": {}
+ },
+ "overrides": []
+ },
+ "fill": 2,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 0,
+ "y": 42
+ },
+ "hiddenSeries": false,
+ "id": 70,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "max": true,
+ "min": true,
+ "rightSide": false,
+ "show": true,
+ "sideWidth": null,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "maxPerRow": 6,
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "node_memory_VmallocChunk_bytes{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "hide": false,
+ "intervalFactor": 2,
+ "legendFormat": "VmallocChunk - Largest contigious block of vmalloc area which is free",
+ "refId": "A",
+ "step": 4
+ },
+ {
+ "expr": "node_memory_VmallocTotal_bytes{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "hide": false,
+ "intervalFactor": 2,
+ "legendFormat": "VmallocTotal - Total size of vmalloc memory area",
+ "refId": "B",
+ "step": 4
+ },
+ {
+ "expr": "node_memory_VmallocUsed_bytes{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "hide": false,
+ "intervalFactor": 2,
+ "legendFormat": "VmallocUsed - Amount of vmalloc area which is used",
+ "refId": "C",
+ "step": 4
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "Memory Vmalloc",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "bytes",
+ "label": "bytes",
+ "logBase": 1,
+ "max": null,
+ "min": "0",
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {
+ "Apps": "#629E51",
+ "Buffers": "#614D93",
+ "Cache": "#6D1F62",
+ "Cached": "#511749",
+ "Committed": "#508642",
+ "Free": "#0A437C",
+ "Hardware Corrupted - Amount of RAM that the kernel identified as corrupted / not working": "#CFFAFF",
+ "Inactive": "#584477",
+ "PageTables": "#0A50A1",
+ "Page_Tables": "#0A50A1",
+ "RAM_Free": "#E0F9D7",
+ "Slab": "#806EB7",
+ "Slab_Cache": "#E0752D",
+ "Swap": "#BF1B00",
+ "Swap_Cache": "#C15C17",
+ "Swap_Free": "#2F575E",
+ "Unused": "#EAB839"
+ },
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "decimals": 2,
+ "fieldConfig": {
+ "defaults": {
+ "custom": {}
+ },
+ "overrides": []
+ },
+ "fill": 2,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 12,
+ "y": 42
+ },
+ "hiddenSeries": false,
+ "id": 159,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "max": true,
+ "min": true,
+ "rightSide": false,
+ "show": true,
+ "sideWidth": 350,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "maxPerRow": 6,
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "node_memory_Bounce_bytes{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "intervalFactor": 2,
+ "legendFormat": "Bounce - Memory used for block device bounce buffers",
+ "refId": "A",
+ "step": 4
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "Memory Bounce",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "cumulative"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "bytes",
+ "label": "bytes",
+ "logBase": 1,
+ "max": null,
+ "min": "0",
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {
+ "Active": "#99440A",
+ "Buffers": "#58140C",
+ "Cache": "#6D1F62",
+ "Cached": "#511749",
+ "Committed": "#508642",
+ "Dirty": "#6ED0E0",
+ "Free": "#B7DBAB",
+ "Inactive": "#EA6460",
+ "Mapped": "#052B51",
+ "PageTables": "#0A50A1",
+ "Page_Tables": "#0A50A1",
+ "Slab_Cache": "#EAB839",
+ "Swap": "#BF1B00",
+ "Swap_Cache": "#C15C17",
+ "Total": "#511749",
+ "Total RAM": "#052B51",
+ "Total RAM + Swap": "#052B51",
+ "VmallocUsed": "#EA6460"
+ },
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "decimals": 2,
+ "fieldConfig": {
+ "defaults": {
+ "custom": {}
+ },
+ "overrides": []
+ },
+ "fill": 2,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 0,
+ "y": 52
+ },
+ "hiddenSeries": false,
+ "id": 129,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "max": true,
+ "min": true,
+ "rightSide": false,
+ "show": true,
+ "sideWidth": null,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "maxPerRow": 6,
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [
+ {
+ "alias": "/.*Inactive *./",
+ "transform": "negative-Y"
+ }
+ ],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "node_memory_AnonHugePages_bytes{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "intervalFactor": 2,
+ "legendFormat": "AnonHugePages - Memory in anonymous huge pages",
+ "refId": "A",
+ "step": 4
+ },
+ {
+ "expr": "node_memory_AnonPages_bytes{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "intervalFactor": 2,
+ "legendFormat": "AnonPages - Memory in user pages not backed by files",
+ "refId": "B",
+ "step": 4
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "Memory Anonymous",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "bytes",
+ "label": "bytes",
+ "logBase": 1,
+ "max": null,
+ "min": "0",
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {
+ "Apps": "#629E51",
+ "Buffers": "#614D93",
+ "Cache": "#6D1F62",
+ "Cached": "#511749",
+ "Committed": "#508642",
+ "Free": "#0A437C",
+ "Hardware Corrupted - Amount of RAM that the kernel identified as corrupted / not working": "#CFFAFF",
+ "Inactive": "#584477",
+ "PageTables": "#0A50A1",
+ "Page_Tables": "#0A50A1",
+ "RAM_Free": "#E0F9D7",
+ "Slab": "#806EB7",
+ "Slab_Cache": "#E0752D",
+ "Swap": "#BF1B00",
+ "Swap_Cache": "#C15C17",
+ "Swap_Free": "#2F575E",
+ "Unused": "#EAB839"
+ },
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "decimals": 2,
+ "fieldConfig": {
+ "defaults": {
+ "custom": {}
+ },
+ "overrides": []
+ },
+ "fill": 2,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 12,
+ "y": 52
+ },
+ "hiddenSeries": false,
+ "id": 160,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "max": true,
+ "min": true,
+ "rightSide": false,
+ "show": true,
+ "sideWidth": 350,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "maxPerRow": 2,
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "node_memory_KernelStack_bytes{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "intervalFactor": 2,
+ "legendFormat": "KernelStack - Kernel memory stack. This is not reclaimable",
+ "refId": "A",
+ "step": 4
+ },
+ {
+ "expr": "node_memory_Percpu_bytes{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "PerCPU - Per CPU memory allocated dynamically by loadable modules",
+ "refId": "B",
+ "step": 4
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "Memory Kernel / CPU",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "cumulative"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "bytes",
+ "label": "bytes",
+ "logBase": 1,
+ "max": null,
+ "min": "0",
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {
+ "Active": "#99440A",
+ "Buffers": "#58140C",
+ "Cache": "#6D1F62",
+ "Cached": "#511749",
+ "Committed": "#508642",
+ "Dirty": "#6ED0E0",
+ "Free": "#B7DBAB",
+ "Inactive": "#EA6460",
+ "Mapped": "#052B51",
+ "PageTables": "#0A50A1",
+ "Page_Tables": "#0A50A1",
+ "Slab_Cache": "#EAB839",
+ "Swap": "#BF1B00",
+ "Swap_Cache": "#C15C17",
+ "Total": "#511749",
+ "Total RAM": "#806EB7",
+ "Total RAM + Swap": "#806EB7",
+ "VmallocUsed": "#EA6460"
+ },
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "decimals": 2,
+ "fieldConfig": {
+ "defaults": {
+ "custom": {}
+ },
+ "overrides": []
+ },
+ "fill": 2,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 0,
+ "y": 62
+ },
+ "hiddenSeries": false,
+ "id": 140,
+ "legend": {
+ "alignAsTable": true,
+ "avg": false,
+ "current": true,
+ "max": true,
+ "min": true,
+ "rightSide": false,
+ "show": true,
+ "sideWidth": null,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "maxPerRow": 6,
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "node_memory_HugePages_Free{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "intervalFactor": 2,
+ "legendFormat": "HugePages_Free - Huge pages in the pool that are not yet allocated",
+ "refId": "A",
+ "step": 4
+ },
+ {
+ "expr": "node_memory_HugePages_Rsvd{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "intervalFactor": 2,
+ "legendFormat": "HugePages_Rsvd - Huge pages for which a commitment to allocate from the pool has been made, but no allocation has yet been made",
+ "refId": "B",
+ "step": 4
+ },
+ {
+ "expr": "node_memory_HugePages_Surp{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "intervalFactor": 2,
+ "legendFormat": "HugePages_Surp - Huge pages in the pool above the value in /proc/sys/vm/nr_hugepages",
+ "refId": "C",
+ "step": 4
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "Memory HugePages Counter",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "short",
+ "label": "pages",
+ "logBase": 1,
+ "max": null,
+ "min": "0",
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": "",
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {
+ "Active": "#99440A",
+ "Buffers": "#58140C",
+ "Cache": "#6D1F62",
+ "Cached": "#511749",
+ "Committed": "#508642",
+ "Dirty": "#6ED0E0",
+ "Free": "#B7DBAB",
+ "Inactive": "#EA6460",
+ "Mapped": "#052B51",
+ "PageTables": "#0A50A1",
+ "Page_Tables": "#0A50A1",
+ "Slab_Cache": "#EAB839",
+ "Swap": "#BF1B00",
+ "Swap_Cache": "#C15C17",
+ "Total": "#511749",
+ "Total RAM": "#806EB7",
+ "Total RAM + Swap": "#806EB7",
+ "VmallocUsed": "#EA6460"
+ },
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "decimals": 2,
+ "fieldConfig": {
+ "defaults": {
+ "custom": {}
+ },
+ "overrides": []
+ },
+ "fill": 2,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 12,
+ "y": 62
+ },
+ "hiddenSeries": false,
+ "id": 71,
+ "legend": {
+ "alignAsTable": true,
+ "avg": false,
+ "current": true,
+ "max": true,
+ "min": true,
+ "rightSide": false,
+ "show": true,
+ "sideWidth": null,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "maxPerRow": 2,
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "node_memory_HugePages_Total{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "intervalFactor": 2,
+ "legendFormat": "HugePages - Total size of the pool of huge pages",
+ "refId": "A",
+ "step": 4
+ },
+ {
+ "expr": "node_memory_Hugepagesize_bytes{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "intervalFactor": 2,
+ "legendFormat": "Hugepagesize - Huge Page size",
+ "refId": "B",
+ "step": 4
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "Memory HugePages Size",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "bytes",
+ "label": "bytes",
+ "logBase": 1,
+ "max": null,
+ "min": "0",
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": "",
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {
+ "Active": "#99440A",
+ "Buffers": "#58140C",
+ "Cache": "#6D1F62",
+ "Cached": "#511749",
+ "Committed": "#508642",
+ "Dirty": "#6ED0E0",
+ "Free": "#B7DBAB",
+ "Inactive": "#EA6460",
+ "Mapped": "#052B51",
+ "PageTables": "#0A50A1",
+ "Page_Tables": "#0A50A1",
+ "Slab_Cache": "#EAB839",
+ "Swap": "#BF1B00",
+ "Swap_Cache": "#C15C17",
+ "Total": "#511749",
+ "Total RAM": "#052B51",
+ "Total RAM + Swap": "#052B51",
+ "VmallocUsed": "#EA6460"
+ },
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "decimals": 2,
+ "fieldConfig": {
+ "defaults": {
+ "custom": {}
+ },
+ "overrides": []
+ },
+ "fill": 2,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 0,
+ "y": 72
+ },
+ "hiddenSeries": false,
+ "id": 128,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": false,
+ "hideEmpty": false,
+ "hideZero": false,
+ "max": true,
+ "min": true,
+ "rightSide": false,
+ "show": true,
+ "sideWidth": null,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "maxPerRow": 6,
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "node_memory_DirectMap1G_bytes{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "intervalFactor": 2,
+ "legendFormat": "DirectMap1G - Amount of pages mapped as this size",
+ "refId": "A",
+ "step": 4
+ },
+ {
+ "expr": "node_memory_DirectMap2M_bytes{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "DirectMap2M - Amount of pages mapped as this size",
+ "refId": "B",
+ "step": 4
+ },
+ {
+ "expr": "node_memory_DirectMap4k_bytes{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "DirectMap4K - Amount of pages mapped as this size",
+ "refId": "C",
+ "step": 4
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "Memory DirectMap",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "bytes",
+ "label": "bytes",
+ "logBase": 1,
+ "max": null,
+ "min": "0",
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {
+ "Apps": "#629E51",
+ "Buffers": "#614D93",
+ "Cache": "#6D1F62",
+ "Cached": "#511749",
+ "Committed": "#508642",
+ "Free": "#0A437C",
+ "Hardware Corrupted - Amount of RAM that the kernel identified as corrupted / not working": "#CFFAFF",
+ "Inactive": "#584477",
+ "PageTables": "#0A50A1",
+ "Page_Tables": "#0A50A1",
+ "RAM_Free": "#E0F9D7",
+ "Slab": "#806EB7",
+ "Slab_Cache": "#E0752D",
+ "Swap": "#BF1B00",
+ "Swap_Cache": "#C15C17",
+ "Swap_Free": "#2F575E",
+ "Unused": "#EAB839"
+ },
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "decimals": 2,
+ "fieldConfig": {
+ "defaults": {
+ "custom": {}
+ },
+ "overrides": []
+ },
+ "fill": 2,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 12,
+ "y": 72
+ },
+ "hiddenSeries": false,
+ "id": 137,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "max": true,
+ "min": true,
+ "rightSide": false,
+ "show": true,
+ "sideWidth": 350,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "maxPerRow": 6,
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "node_memory_Unevictable_bytes{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "intervalFactor": 2,
+ "legendFormat": "Unevictable - Amount of unevictable memory that can't be swapped out for a variety of reasons",
+ "refId": "A",
+ "step": 4
+ },
+ {
+ "expr": "node_memory_Mlocked_bytes{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "intervalFactor": 2,
+ "legendFormat": "MLocked - Size of pages locked to memory using the mlock() system call",
+ "refId": "B",
+ "step": 4
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "Memory Unevictable and MLocked",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "cumulative"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "bytes",
+ "label": "bytes",
+ "logBase": 1,
+ "max": null,
+ "min": "0",
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {
+ "Active": "#99440A",
+ "Buffers": "#58140C",
+ "Cache": "#6D1F62",
+ "Cached": "#511749",
+ "Committed": "#508642",
+ "Dirty": "#6ED0E0",
+ "Free": "#B7DBAB",
+ "Inactive": "#EA6460",
+ "Mapped": "#052B51",
+ "PageTables": "#0A50A1",
+ "Page_Tables": "#0A50A1",
+ "Slab_Cache": "#EAB839",
+ "Swap": "#BF1B00",
+ "Swap_Cache": "#C15C17",
+ "Total": "#511749",
+ "Total RAM": "#052B51",
+ "Total RAM + Swap": "#052B51",
+ "Total Swap": "#614D93",
+ "VmallocUsed": "#EA6460"
+ },
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "decimals": 2,
+ "fieldConfig": {
+ "defaults": {
+ "custom": {}
+ },
+ "overrides": []
+ },
+ "fill": 2,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 0,
+ "y": 82
+ },
+ "hiddenSeries": false,
+ "id": 132,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "max": true,
+ "min": true,
+ "rightSide": false,
+ "show": true,
+ "sideWidth": null,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "maxPerRow": 6,
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "node_memory_NFS_Unstable_bytes{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "intervalFactor": 2,
+ "legendFormat": "NFS Unstable - Memory in NFS pages sent to the server, but not yet commited to the storage",
+ "refId": "A",
+ "step": 4
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "Memory NFS",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "bytes",
+ "label": "bytes",
+ "logBase": 1,
+ "max": null,
+ "min": "0",
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ }
+ ],
+ "repeat": null,
+ "title": "Memory Meminfo",
+ "type": "row"
+ },
+ {
+ "collapsed": true,
+ "datasource": "${DS_PROMETHEUS}",
+ "gridPos": {
+ "h": 1,
+ "w": 24,
+ "x": 0,
+ "y": 60
+ },
+ "id": 267,
+ "panels": [
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "fill": 2,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 0,
+ "y": 23
+ },
+ "hiddenSeries": false,
+ "id": 176,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "max": true,
+ "min": true,
+ "rightSide": false,
+ "show": true,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "maxPerRow": 6,
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [
+ {
+ "alias": "/.*out/",
+ "transform": "negative-Y"
+ }
+ ],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "irate(node_vmstat_pgpgin{instance=~\"$node:$port\",job=~\"$job\"}[5m])",
+ "format": "time_series",
+ "intervalFactor": 2,
+ "legendFormat": "Pagesin - Page in operations",
+ "refId": "A",
+ "step": 4
+ },
+ {
+ "expr": "irate(node_vmstat_pgpgout{instance=~\"$node:$port\",job=~\"$job\"}[5m])",
+ "format": "time_series",
+ "intervalFactor": 2,
+ "legendFormat": "Pagesout - Page out operations",
+ "refId": "B",
+ "step": 4
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "Memory Pages In / Out",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "short",
+ "label": "pages out (-) / in (+)",
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "fill": 2,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 12,
+ "y": 23
+ },
+ "hiddenSeries": false,
+ "id": 22,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "max": true,
+ "min": true,
+ "rightSide": false,
+ "show": true,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "maxPerRow": 6,
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [
+ {
+ "alias": "/.*out/",
+ "transform": "negative-Y"
+ }
+ ],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "irate(node_vmstat_pswpin{instance=~\"$node:$port\",job=~\"$job\"}[5m])",
+ "format": "time_series",
+ "intervalFactor": 2,
+ "legendFormat": "Pswpin - Pages swapped in",
+ "refId": "A",
+ "step": 4
+ },
+ {
+ "expr": "irate(node_vmstat_pswpout{instance=~\"$node:$port\",job=~\"$job\"}[5m])",
+ "format": "time_series",
+ "intervalFactor": 2,
+ "legendFormat": "Pswpout - Pages swapped out",
+ "refId": "B",
+ "step": 4
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "Memory Pages Swap In / Out",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "short",
+ "label": "pages out (-) / in (+)",
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {
+ "Apps": "#629E51",
+ "Buffers": "#614D93",
+ "Cache": "#6D1F62",
+ "Cached": "#511749",
+ "Committed": "#508642",
+ "Free": "#0A437C",
+ "Hardware Corrupted - Amount of RAM that the kernel identified as corrupted / not working": "#CFFAFF",
+ "Inactive": "#584477",
+ "PageTables": "#0A50A1",
+ "Page_Tables": "#0A50A1",
+ "RAM_Free": "#E0F9D7",
+ "Slab": "#806EB7",
+ "Slab_Cache": "#E0752D",
+ "Swap": "#BF1B00",
+ "Swap_Cache": "#C15C17",
+ "Swap_Free": "#2F575E",
+ "Unused": "#EAB839"
+ },
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "decimals": 2,
+ "fill": 2,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 0,
+ "y": 33
+ },
+ "hiddenSeries": false,
+ "id": 175,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "max": true,
+ "min": true,
+ "rightSide": false,
+ "show": true,
+ "sideWidth": 350,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "maxPerRow": 6,
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [
+ {
+ "alias": "Pgfault - Page major and minor fault operations",
+ "fill": 0,
+ "stack": false
+ }
+ ],
+ "spaceLength": 10,
+ "stack": true,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "irate(node_vmstat_pgfault{instance=~\"$node:$port\",job=~\"$job\"}[5m])",
+ "format": "time_series",
+ "intervalFactor": 2,
+ "legendFormat": "Pgfault - Page major and minor fault operations",
+ "refId": "A",
+ "step": 4
+ },
+ {
+ "expr": "irate(node_vmstat_pgmajfault{instance=~\"$node:$port\",job=~\"$job\"}[5m])",
+ "format": "time_series",
+ "intervalFactor": 2,
+ "legendFormat": "Pgmajfault - Major page fault operations",
+ "refId": "B",
+ "step": 4
+ },
+ {
+ "expr": "irate(node_vmstat_pgfault{instance=~\"$node:$port\",job=~\"$job\"}[5m]) - irate(node_vmstat_pgmajfault{instance=~\"$node:$port\",job=~\"$job\"}[5m])",
+ "format": "time_series",
+ "intervalFactor": 2,
+ "legendFormat": "Pgminfault - Minor page fault operations",
+ "refId": "C",
+ "step": 4
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "Memory Page Faults",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "cumulative"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "short",
+ "label": "faults",
+ "logBase": 1,
+ "max": null,
+ "min": "0",
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {
+ "Active": "#99440A",
+ "Buffers": "#58140C",
+ "Cache": "#6D1F62",
+ "Cached": "#511749",
+ "Committed": "#508642",
+ "Dirty": "#6ED0E0",
+ "Free": "#B7DBAB",
+ "Inactive": "#EA6460",
+ "Mapped": "#052B51",
+ "PageTables": "#0A50A1",
+ "Page_Tables": "#0A50A1",
+ "Slab_Cache": "#EAB839",
+ "Swap": "#BF1B00",
+ "Swap_Cache": "#C15C17",
+ "Total": "#511749",
+ "Total RAM": "#052B51",
+ "Total RAM + Swap": "#052B51",
+ "Total Swap": "#614D93",
+ "VmallocUsed": "#EA6460"
+ },
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "decimals": 2,
+ "fill": 2,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 12,
+ "y": 33
+ },
+ "hiddenSeries": false,
+ "id": 307,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "max": true,
+ "min": true,
+ "rightSide": false,
+ "show": true,
+ "sideWidth": null,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "maxPerRow": 6,
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "irate(node_vmstat_oom_kill{instance=~\"$node:$port\",job=~\"$job\"}[5m])",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "oom killer invocations ",
+ "refId": "A",
+ "step": 4
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "OOM Killer",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "short",
+ "label": "counter",
+ "logBase": 1,
+ "max": null,
+ "min": "0",
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ }
+ ],
+ "repeat": null,
+ "title": "Memory Vmstat",
+ "type": "row"
+ },
+ {
+ "collapsed": true,
+ "datasource": "${DS_PROMETHEUS}",
+ "gridPos": {
+ "h": 1,
+ "w": 24,
+ "x": 0,
+ "y": 61
+ },
+ "id": 293,
+ "panels": [
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "description": "",
+ "fill": 2,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 0,
+ "y": 24
+ },
+ "hiddenSeries": false,
+ "id": 260,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "max": true,
+ "min": true,
+ "show": true,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [
+ {
+ "alias": "/.*Variation*./",
+ "color": "#890F02"
+ }
+ ],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "node_timex_estimated_error_seconds{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "hide": false,
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "Estimated error in seconds",
+ "refId": "A",
+ "step": 240
+ },
+ {
+ "expr": "node_timex_offset_seconds{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "hide": false,
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "Time offset in between local system and reference clock",
+ "refId": "B",
+ "step": 240
+ },
+ {
+ "expr": "node_timex_maxerror_seconds{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "hide": false,
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "Maximum error in seconds",
+ "refId": "C",
+ "step": 240
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "Time Syncronized Drift",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "s",
+ "label": "seconds",
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": "counter",
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "description": "",
+ "fill": 2,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 12,
+ "y": 24
+ },
+ "hiddenSeries": false,
+ "id": 291,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "max": true,
+ "min": true,
+ "show": true,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "node_timex_loop_time_constant{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "Phase-locked loop time adjust",
+ "refId": "A",
+ "step": 240
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "Time PLL Adjust",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "short",
+ "label": "counter",
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "description": "",
+ "fill": 2,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 0,
+ "y": 34
+ },
+ "hiddenSeries": false,
+ "id": 168,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "max": true,
+ "min": true,
+ "show": true,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [
+ {
+ "alias": "/.*Variation*./",
+ "color": "#890F02"
+ }
+ ],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "node_timex_sync_status{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "Is clock synchronized to a reliable server (1 = yes, 0 = no)",
+ "refId": "A",
+ "step": 240
+ },
+ {
+ "expr": "node_timex_frequency_adjustment_ratio{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "Local clock frequency adjustment",
+ "refId": "B",
+ "step": 240
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "Time Syncronized Status",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "short",
+ "label": "counter",
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "description": "",
+ "fill": 2,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 12,
+ "y": 34
+ },
+ "hiddenSeries": false,
+ "id": 294,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "max": true,
+ "min": true,
+ "show": true,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "node_timex_tick_seconds{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "Seconds between clock ticks",
+ "refId": "A",
+ "step": 240
+ },
+ {
+ "expr": "node_timex_tai_offset_seconds{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "International Atomic Time (TAI) offset",
+ "refId": "B",
+ "step": 240
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "Time Misc",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "s",
+ "label": "seconds",
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ }
+ ],
+ "title": "System Timesync",
+ "type": "row"
+ },
+ {
+ "collapsed": true,
+ "datasource": "${DS_PROMETHEUS}",
+ "gridPos": {
+ "h": 1,
+ "w": 24,
+ "x": 0,
+ "y": 62
+ },
+ "id": 312,
+ "panels": [
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "fill": 2,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 0,
+ "y": 7
+ },
+ "hiddenSeries": false,
+ "id": 62,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "max": true,
+ "min": true,
+ "show": true,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "maxPerRow": 6,
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "node_procs_blocked{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "intervalFactor": 2,
+ "legendFormat": "Processes blocked waiting for I/O to complete",
+ "refId": "A",
+ "step": 240
+ },
+ {
+ "expr": "node_procs_running{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "intervalFactor": 2,
+ "legendFormat": "Processes in runnable state",
+ "refId": "B",
+ "step": 240
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "Processes Status",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "short",
+ "label": "counter",
+ "logBase": 1,
+ "max": null,
+ "min": "0",
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "fill": 2,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 12,
+ "y": 7
+ },
+ "hiddenSeries": false,
+ "id": 315,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "max": true,
+ "min": true,
+ "show": true,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "maxPerRow": 6,
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "stack": true,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "node_processes_state{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "{{ state }}",
+ "refId": "A",
+ "step": 240
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "Processes State",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "short",
+ "label": "counter",
+ "logBase": 1,
+ "max": null,
+ "min": "0",
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "fill": 2,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 0,
+ "y": 17
+ },
+ "hiddenSeries": false,
+ "id": 148,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "max": true,
+ "min": true,
+ "show": true,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "maxPerRow": 6,
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "irate(node_forks_total{instance=~\"$node:$port\",job=~\"$job\"}[5m])",
+ "format": "time_series",
+ "hide": false,
+ "intervalFactor": 2,
+ "legendFormat": "Processes forks second",
+ "refId": "A",
+ "step": 240
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "Processes Forks",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "short",
+ "label": "forks / sec",
+ "logBase": 1,
+ "max": null,
+ "min": "0",
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "fill": 2,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 12,
+ "y": 17
+ },
+ "hiddenSeries": false,
+ "id": 149,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "max": true,
+ "min": true,
+ "show": true,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [
+ {
+ "alias": "/.*Max.*/",
+ "fill": 0
+ }
+ ],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "irate(process_virtual_memory_bytes{instance=~\"$node:$port\",job=~\"$job\"}[5m])",
+ "hide": false,
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "Processes virtual memory size in bytes",
+ "refId": "A",
+ "step": 240
+ },
+ {
+ "expr": "process_resident_memory_max_bytes{instance=~\"$node:$port\",job=~\"$job\"}",
+ "hide": false,
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "Maximum amount of virtual memory available in bytes",
+ "refId": "B",
+ "step": 240
+ },
+ {
+ "expr": "irate(process_virtual_memory_bytes{instance=~\"$node:$port\",job=~\"$job\"}[5m])",
+ "hide": false,
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "Processes virtual memory size in bytes",
+ "refId": "C",
+ "step": 240
+ },
+ {
+ "expr": "irate(process_virtual_memory_max_bytes{instance=~\"$node:$port\",job=~\"$job\"}[5m])",
+ "hide": false,
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "Maximum amount of virtual memory available in bytes",
+ "refId": "D",
+ "step": 240
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "Processes Memory",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "decbytes",
+ "label": "bytes",
+ "logBase": 1,
+ "max": null,
+ "min": "0",
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "fill": 2,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 0,
+ "y": 27
+ },
+ "hiddenSeries": false,
+ "id": 313,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "max": true,
+ "min": true,
+ "show": true,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "maxPerRow": 6,
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [
+ {
+ "alias": "PIDs limit",
+ "color": "#F2495C",
+ "fill": 0
+ }
+ ],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "node_processes_pids{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "Number of PIDs",
+ "refId": "A",
+ "step": 240
+ },
+ {
+ "expr": "node_processes_max_processes{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "PIDs limit",
+ "refId": "B",
+ "step": 240
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "PIDs Number and Limit",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "short",
+ "label": "counter",
+ "logBase": 1,
+ "max": null,
+ "min": "0",
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "fill": 2,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 12,
+ "y": 27
+ },
+ "hiddenSeries": false,
+ "id": 305,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "max": true,
+ "min": true,
+ "show": true,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "maxPerRow": 6,
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [
+ {
+ "alias": "/.*waiting.*/",
+ "transform": "negative-Y"
+ }
+ ],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "irate(node_schedstat_running_seconds_total{instance=~\"$node:$port\",job=~\"$job\"}[5m])",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "CPU {{ cpu }} - seconds spent running a process",
+ "refId": "A",
+ "step": 240
+ },
+ {
+ "expr": "irate(node_schedstat_waiting_seconds_total{instance=~\"$node:$port\",job=~\"$job\"}[5m])",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "CPU {{ cpu }} - seconds spent by processing waiting for this CPU",
+ "refId": "B",
+ "step": 240
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "Process schdeule stats Running / Waiting",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "s",
+ "label": "seconds",
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "fill": 2,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 12,
+ "y": 37
+ },
+ "hiddenSeries": false,
+ "id": 314,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "max": true,
+ "min": true,
+ "show": true,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "maxPerRow": 6,
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [
+ {
+ "alias": "Threads limit",
+ "color": "#F2495C",
+ "fill": 0
+ }
+ ],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "node_processes_threads{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "Allocated threads",
+ "refId": "A",
+ "step": 240
+ },
+ {
+ "expr": "node_processes_max_threads{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "Threads limit",
+ "refId": "B",
+ "step": 240
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "Threads Number and Limit",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "short",
+ "label": "counter",
+ "logBase": 1,
+ "max": null,
+ "min": "0",
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ }
+ ],
+ "title": "System Processes",
+ "type": "row"
+ },
+ {
+ "collapsed": true,
+ "datasource": "${DS_PROMETHEUS}",
+ "gridPos": {
+ "h": 1,
+ "w": 24,
+ "x": 0,
+ "y": 63
+ },
+ "id": 269,
+ "panels": [
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "fieldConfig": {
+ "defaults": {
+ "custom": {}
+ },
+ "overrides": []
+ },
+ "fill": 2,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 0,
+ "y": 64
+ },
+ "hiddenSeries": false,
+ "id": 8,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "max": true,
+ "min": true,
+ "show": true,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "maxPerRow": 6,
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "repeat": null,
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "irate(node_context_switches_total{instance=~\"$node:$port\",job=~\"$job\"}[5m])",
+ "format": "time_series",
+ "intervalFactor": 2,
+ "legendFormat": "Context switches",
+ "refId": "A",
+ "step": 240
+ },
+ {
+ "expr": "irate(node_intr_total{instance=~\"$node:$port\",job=~\"$job\"}[5m])",
+ "format": "time_series",
+ "hide": false,
+ "intervalFactor": 2,
+ "legendFormat": "Interrupts",
+ "refId": "B",
+ "step": 240
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "Context Switches / Interrupts",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "short",
+ "label": "counter",
+ "logBase": 1,
+ "max": null,
+ "min": "0",
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "fieldConfig": {
+ "defaults": {
+ "custom": {}
+ },
+ "overrides": []
+ },
+ "fill": 2,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 12,
+ "y": 64
+ },
+ "hiddenSeries": false,
+ "id": 7,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "max": true,
+ "min": true,
+ "show": true,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "maxPerRow": 6,
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "repeat": null,
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "node_load1{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "intervalFactor": 4,
+ "legendFormat": "Load 1m",
+ "refId": "A",
+ "step": 480
+ },
+ {
+ "expr": "node_load5{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "intervalFactor": 4,
+ "legendFormat": "Load 5m",
+ "refId": "B",
+ "step": 480
+ },
+ {
+ "expr": "node_load15{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "intervalFactor": 4,
+ "legendFormat": "Load 15m",
+ "refId": "C",
+ "step": 480
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "System Load",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "short",
+ "label": "counter",
+ "logBase": 1,
+ "max": null,
+ "min": "0",
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "fieldConfig": {
+ "defaults": {
+ "custom": {}
+ },
+ "overrides": []
+ },
+ "fill": 2,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 0,
+ "y": 74
+ },
+ "hiddenSeries": false,
+ "id": 259,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "max": true,
+ "min": true,
+ "show": true,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [
+ {
+ "alias": "/.*Critical*./",
+ "color": "#E24D42",
+ "fill": 0
+ },
+ {
+ "alias": "/.*Max*./",
+ "color": "#EF843C",
+ "fill": 0
+ }
+ ],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "irate(node_interrupts_total{instance=~\"$node:$port\",job=~\"$job\"}[5m])",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "{{ type }} - {{ info }}",
+ "refId": "A",
+ "step": 240
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "Interrupts Detail",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "short",
+ "label": "counter",
+ "logBase": 1,
+ "max": null,
+ "min": "0",
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "fieldConfig": {
+ "defaults": {
+ "custom": {}
+ },
+ "overrides": []
+ },
+ "fill": 2,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 12,
+ "y": 74
+ },
+ "hiddenSeries": false,
+ "id": 306,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "max": true,
+ "min": true,
+ "show": true,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "maxPerRow": 6,
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "irate(node_schedstat_timeslices_total{instance=~\"$node:$port\",job=~\"$job\"}[5m])",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "CPU {{ cpu }}",
+ "refId": "A",
+ "step": 240
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "Schedule timeslices executed by each cpu",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "short",
+ "label": "counter",
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "fieldConfig": {
+ "defaults": {
+ "custom": {}
+ },
+ "overrides": []
+ },
+ "fill": 2,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 0,
+ "y": 84
+ },
+ "hiddenSeries": false,
+ "id": 151,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "max": true,
+ "min": true,
+ "show": true,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "maxPerRow": 6,
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "node_entropy_available_bits{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "intervalFactor": 2,
+ "legendFormat": "Entropy available to random number generators",
+ "refId": "A",
+ "step": 240
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "Entropy",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "short",
+ "label": "counter",
+ "logBase": 1,
+ "max": null,
+ "min": "0",
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "fieldConfig": {
+ "defaults": {
+ "custom": {}
+ },
+ "overrides": []
+ },
+ "fill": 2,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 12,
+ "y": 84
+ },
+ "hiddenSeries": false,
+ "id": 308,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "max": true,
+ "min": true,
+ "show": true,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "maxPerRow": 6,
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "irate(process_cpu_seconds_total{instance=~\"$node:$port\",job=~\"$job\"}[5m])",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "Time spent",
+ "refId": "A",
+ "step": 240
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "CPU time spent in user and system contexts",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "s",
+ "label": "seconds",
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "fieldConfig": {
+ "defaults": {
+ "custom": {}
+ },
+ "overrides": []
+ },
+ "fill": 2,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 0,
+ "y": 94
+ },
+ "hiddenSeries": false,
+ "id": 64,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "max": true,
+ "min": true,
+ "show": true,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [
+ {
+ "alias": "/.*Max*./",
+ "color": "#890F02",
+ "fill": 0
+ }
+ ],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "process_max_fds{instance=~\"$node:$port\",job=~\"$job\"}",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "Maximum open file descriptors",
+ "refId": "A",
+ "step": 240
+ },
+ {
+ "expr": "process_open_fds{instance=~\"$node:$port\",job=~\"$job\"}",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "Open file descriptors",
+ "refId": "B",
+ "step": 240
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "File Descriptors",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "short",
+ "label": "counter",
+ "logBase": 1,
+ "max": null,
+ "min": "0",
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ }
+ ],
+ "repeat": null,
+ "title": "System Misc",
+ "type": "row"
+ },
+ {
+ "collapsed": true,
+ "datasource": "${DS_PROMETHEUS}",
+ "gridPos": {
+ "h": 1,
+ "w": 24,
+ "x": 0,
+ "y": 64
+ },
+ "id": 304,
+ "panels": [
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "fieldConfig": {
+ "defaults": {
+ "custom": {}
+ },
+ "overrides": []
+ },
+ "fill": 2,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 0,
+ "y": 17
+ },
+ "hiddenSeries": false,
+ "id": 158,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "max": true,
+ "min": true,
+ "show": true,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [
+ {
+ "alias": "/.*Critical*./",
+ "color": "#E24D42",
+ "fill": 0
+ },
+ {
+ "alias": "/.*Max*./",
+ "color": "#EF843C",
+ "fill": 0
+ }
+ ],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "node_hwmon_temp_celsius{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "{{ chip }} {{ sensor }} temp",
+ "refId": "A",
+ "step": 240
+ },
+ {
+ "expr": "node_hwmon_temp_crit_alarm_celsius{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "hide": true,
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "{{ chip }} {{ sensor }} Critical Alarm",
+ "refId": "B",
+ "step": 240
+ },
+ {
+ "expr": "node_hwmon_temp_crit_celsius{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "{{ chip }} {{ sensor }} Critical",
+ "refId": "C",
+ "step": 240
+ },
+ {
+ "expr": "node_hwmon_temp_crit_hyst_celsius{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "hide": true,
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "{{ chip }} {{ sensor }} Critical Historical",
+ "refId": "D",
+ "step": 240
+ },
+ {
+ "expr": "node_hwmon_temp_max_celsius{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "hide": true,
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "{{ chip }} {{ sensor }} Max",
+ "refId": "E",
+ "step": 240
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "Hardware temperature monitor",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "celsius",
+ "label": "temperature",
+ "logBase": 1,
+ "max": null,
+ "min": "0",
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "fieldConfig": {
+ "defaults": {
+ "custom": {}
+ },
+ "overrides": []
+ },
+ "fill": 2,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 12,
+ "y": 17
+ },
+ "hiddenSeries": false,
+ "id": 300,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "max": true,
+ "min": true,
+ "show": true,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [
+ {
+ "alias": "/.*Max*./",
+ "color": "#EF843C",
+ "fill": 0
+ }
+ ],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "node_cooling_device_cur_state{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "hide": false,
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "Current {{ name }} in {{ type }}",
+ "refId": "A",
+ "step": 240
+ },
+ {
+ "expr": "node_cooling_device_max_state{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "Max {{ name }} in {{ type }}",
+ "refId": "B",
+ "step": 240
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "Throttle cooling device",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "short",
+ "label": "counter",
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "fieldConfig": {
+ "defaults": {
+ "custom": {}
+ },
+ "overrides": []
+ },
+ "fill": 2,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 0,
+ "y": 27
+ },
+ "hiddenSeries": false,
+ "id": 302,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "max": true,
+ "min": true,
+ "show": true,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "node_power_supply_online{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "hide": false,
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "{{ power_supply }} online",
+ "refId": "A",
+ "step": 240
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "Power supply",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "short",
+ "label": "counter",
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ }
+ ],
+ "title": "Hardware Misc",
+ "type": "row"
+ },
+ {
+ "collapsed": true,
+ "datasource": "${DS_PROMETHEUS}",
+ "gridPos": {
+ "h": 1,
+ "w": 24,
+ "x": 0,
+ "y": 65
+ },
+ "id": 296,
+ "panels": [
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "fill": 2,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 0,
+ "y": 10
+ },
+ "hiddenSeries": false,
+ "id": 297,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "max": true,
+ "min": true,
+ "show": true,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "irate(node_systemd_socket_accepted_connections_total{instance=~\"$node:$port\",job=~\"$job\"}[5m])",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "{{ name }} Connections",
+ "refId": "A",
+ "step": 240
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "Systemd Sockets",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "short",
+ "label": "counter",
+ "logBase": 1,
+ "max": null,
+ "min": "0",
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "fill": 2,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 12,
+ "y": 10
+ },
+ "hiddenSeries": false,
+ "id": 298,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "max": true,
+ "min": true,
+ "show": true,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [
+ {
+ "alias": "Failed",
+ "color": "#F2495C"
+ },
+ {
+ "alias": "Inactive",
+ "color": "#FF9830"
+ },
+ {
+ "alias": "Active",
+ "color": "#73BF69"
+ },
+ {
+ "alias": "Deactivating",
+ "color": "#FFCB7D"
+ },
+ {
+ "alias": "Activating",
+ "color": "#C8F2C2"
+ }
+ ],
+ "spaceLength": 10,
+ "stack": true,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "node_systemd_units{instance=~\"$node:$port\",job=~\"$job\",state=\"activating\"}",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "Activating",
+ "refId": "A",
+ "step": 240
+ },
+ {
+ "expr": "node_systemd_units{instance=~\"$node:$port\",job=~\"$job\",state=\"active\"}",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "Active",
+ "refId": "B",
+ "step": 240
+ },
+ {
+ "expr": "node_systemd_units{instance=~\"$node:$port\",job=~\"$job\",state=\"deactivating\"}",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "Deactivating",
+ "refId": "C",
+ "step": 240
+ },
+ {
+ "expr": "node_systemd_units{instance=~\"$node:$port\",job=~\"$job\",state=\"failed\"}",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "Failed",
+ "refId": "D",
+ "step": 240
+ },
+ {
+ "expr": "node_systemd_units{instance=~\"$node:$port\",job=~\"$job\",state=\"inactive\"}",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "Inactive",
+ "refId": "E",
+ "step": 240
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "Systemd Units State",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "short",
+ "label": "counter",
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ }
+ ],
+ "title": "Systemd",
+ "type": "row"
+ },
+ {
+ "collapsed": true,
+ "datasource": "${DS_PROMETHEUS}",
+ "gridPos": {
+ "h": 1,
+ "w": 24,
+ "x": 0,
+ "y": 66
+ },
+ "id": 270,
+ "panels": [
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "description": "",
+ "fill": 2,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 0,
+ "y": 29
+ },
+ "hiddenSeries": false,
+ "id": 9,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "hideZero": true,
+ "max": true,
+ "min": true,
+ "rightSide": false,
+ "show": true,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "maxPerRow": 6,
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "repeat": null,
+ "seriesOverrides": [
+ {
+ "alias": "/.*Read.*/",
+ "transform": "negative-Y"
+ },
+ {
+ "alias": "/.*sda_.*/",
+ "color": "#7EB26D"
+ },
+ {
+ "alias": "/.*sdb_.*/",
+ "color": "#EAB839"
+ },
+ {
+ "alias": "/.*sdc_.*/",
+ "color": "#6ED0E0"
+ },
+ {
+ "alias": "/.*sdd_.*/",
+ "color": "#EF843C"
+ },
+ {
+ "alias": "/.*sde_.*/",
+ "color": "#E24D42"
+ },
+ {
+ "alias": "/.*sda1.*/",
+ "color": "#584477"
+ },
+ {
+ "alias": "/.*sda2_.*/",
+ "color": "#BA43A9"
+ },
+ {
+ "alias": "/.*sda3_.*/",
+ "color": "#F4D598"
+ },
+ {
+ "alias": "/.*sdb1.*/",
+ "color": "#0A50A1"
+ },
+ {
+ "alias": "/.*sdb2.*/",
+ "color": "#BF1B00"
+ },
+ {
+ "alias": "/.*sdb3.*/",
+ "color": "#E0752D"
+ },
+ {
+ "alias": "/.*sdc1.*/",
+ "color": "#962D82"
+ },
+ {
+ "alias": "/.*sdc2.*/",
+ "color": "#614D93"
+ },
+ {
+ "alias": "/.*sdc3.*/",
+ "color": "#9AC48A"
+ },
+ {
+ "alias": "/.*sdd1.*/",
+ "color": "#65C5DB"
+ },
+ {
+ "alias": "/.*sdd2.*/",
+ "color": "#F9934E"
+ },
+ {
+ "alias": "/.*sdd3.*/",
+ "color": "#EA6460"
+ },
+ {
+ "alias": "/.*sde1.*/",
+ "color": "#E0F9D7"
+ },
+ {
+ "alias": "/.*sdd2.*/",
+ "color": "#FCEACA"
+ },
+ {
+ "alias": "/.*sde3.*/",
+ "color": "#F9E2D2"
+ }
+ ],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "irate(node_disk_reads_completed_total{instance=~\"$node:$port\",job=~\"$job\"}[5m])",
+ "intervalFactor": 4,
+ "legendFormat": "{{device}} - Reads completed",
+ "refId": "A",
+ "step": 8
+ },
+ {
+ "expr": "irate(node_disk_writes_completed_total{instance=~\"$node:$port\",job=~\"$job\"}[5m])",
+ "intervalFactor": 2,
+ "legendFormat": "{{device}} - Writes completed",
+ "refId": "B",
+ "step": 4
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "Disk IOps Completed",
+ "tooltip": {
+ "shared": false,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "iops",
+ "label": "IO read (-) / write (+)",
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "description": "",
+ "fill": 2,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 12,
+ "y": 29
+ },
+ "hiddenSeries": false,
+ "id": 33,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "hideZero": true,
+ "max": true,
+ "min": true,
+ "rightSide": false,
+ "show": true,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "maxPerRow": 6,
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [
+ {
+ "alias": "/.*Read.*/",
+ "transform": "negative-Y"
+ },
+ {
+ "alias": "/.*sda_.*/",
+ "color": "#7EB26D"
+ },
+ {
+ "alias": "/.*sdb_.*/",
+ "color": "#EAB839"
+ },
+ {
+ "alias": "/.*sdc_.*/",
+ "color": "#6ED0E0"
+ },
+ {
+ "alias": "/.*sdd_.*/",
+ "color": "#EF843C"
+ },
+ {
+ "alias": "/.*sde_.*/",
+ "color": "#E24D42"
+ },
+ {
+ "alias": "/.*sda1.*/",
+ "color": "#584477"
+ },
+ {
+ "alias": "/.*sda2_.*/",
+ "color": "#BA43A9"
+ },
+ {
+ "alias": "/.*sda3_.*/",
+ "color": "#F4D598"
+ },
+ {
+ "alias": "/.*sdb1.*/",
+ "color": "#0A50A1"
+ },
+ {
+ "alias": "/.*sdb2.*/",
+ "color": "#BF1B00"
+ },
+ {
+ "alias": "/.*sdb3.*/",
+ "color": "#E0752D"
+ },
+ {
+ "alias": "/.*sdc1.*/",
+ "color": "#962D82"
+ },
+ {
+ "alias": "/.*sdc2.*/",
+ "color": "#614D93"
+ },
+ {
+ "alias": "/.*sdc3.*/",
+ "color": "#9AC48A"
+ },
+ {
+ "alias": "/.*sdd1.*/",
+ "color": "#65C5DB"
+ },
+ {
+ "alias": "/.*sdd2.*/",
+ "color": "#F9934E"
+ },
+ {
+ "alias": "/.*sdd3.*/",
+ "color": "#EA6460"
+ },
+ {
+ "alias": "/.*sde1.*/",
+ "color": "#E0F9D7"
+ },
+ {
+ "alias": "/.*sdd2.*/",
+ "color": "#FCEACA"
+ },
+ {
+ "alias": "/.*sde3.*/",
+ "color": "#F9E2D2"
+ }
+ ],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "irate(node_disk_read_bytes_total{instance=~\"$node:$port\",job=~\"$job\"}[5m])",
+ "format": "time_series",
+ "intervalFactor": 4,
+ "legendFormat": "{{device}} - Read bytes",
+ "refId": "A",
+ "step": 8
+ },
+ {
+ "expr": "irate(node_disk_written_bytes_total{instance=~\"$node:$port\",job=~\"$job\"}[5m])",
+ "format": "time_series",
+ "intervalFactor": 2,
+ "legendFormat": "{{device}} - Written bytes",
+ "refId": "B",
+ "step": 4
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "Disk R/W Data",
+ "tooltip": {
+ "shared": false,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "Bps",
+ "label": "bytes read (-) / write (+)",
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "description": "",
+ "fill": 3,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 0,
+ "y": 39
+ },
+ "hiddenSeries": false,
+ "id": 37,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "hideZero": true,
+ "max": true,
+ "min": true,
+ "rightSide": false,
+ "show": true,
+ "sort": "current",
+ "sortDesc": true,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "maxPerRow": 6,
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [
+ {
+ "alias": "/.*Read.*/",
+ "transform": "negative-Y"
+ },
+ {
+ "alias": "/.*sda_.*/",
+ "color": "#7EB26D"
+ },
+ {
+ "alias": "/.*sdb_.*/",
+ "color": "#EAB839"
+ },
+ {
+ "alias": "/.*sdc_.*/",
+ "color": "#6ED0E0"
+ },
+ {
+ "alias": "/.*sdd_.*/",
+ "color": "#EF843C"
+ },
+ {
+ "alias": "/.*sde_.*/",
+ "color": "#E24D42"
+ },
+ {
+ "alias": "/.*sda1.*/",
+ "color": "#584477"
+ },
+ {
+ "alias": "/.*sda2_.*/",
+ "color": "#BA43A9"
+ },
+ {
+ "alias": "/.*sda3_.*/",
+ "color": "#F4D598"
+ },
+ {
+ "alias": "/.*sdb1.*/",
+ "color": "#0A50A1"
+ },
+ {
+ "alias": "/.*sdb2.*/",
+ "color": "#BF1B00"
+ },
+ {
+ "alias": "/.*sdb3.*/",
+ "color": "#E0752D"
+ },
+ {
+ "alias": "/.*sdc1.*/",
+ "color": "#962D82"
+ },
+ {
+ "alias": "/.*sdc2.*/",
+ "color": "#614D93"
+ },
+ {
+ "alias": "/.*sdc3.*/",
+ "color": "#9AC48A"
+ },
+ {
+ "alias": "/.*sdd1.*/",
+ "color": "#65C5DB"
+ },
+ {
+ "alias": "/.*sdd2.*/",
+ "color": "#F9934E"
+ },
+ {
+ "alias": "/.*sdd3.*/",
+ "color": "#EA6460"
+ },
+ {
+ "alias": "/.*sde1.*/",
+ "color": "#E0F9D7"
+ },
+ {
+ "alias": "/.*sdd2.*/",
+ "color": "#FCEACA"
+ },
+ {
+ "alias": "/.*sde3.*/",
+ "color": "#F9E2D2"
+ }
+ ],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "irate(node_disk_read_time_seconds_total{instance=~\"$node:$port\",job=~\"$job\"}[5m])",
+ "hide": false,
+ "intervalFactor": 4,
+ "legendFormat": "{{device}} - Read time",
+ "refId": "A",
+ "step": 8
+ },
+ {
+ "expr": "irate(node_disk_write_time_seconds_total{instance=~\"$node:$port\",job=~\"$job\"}[5m])",
+ "hide": false,
+ "intervalFactor": 2,
+ "legendFormat": "{{device}} - Write time",
+ "refId": "B",
+ "step": 4
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "Disk R/W Time",
+ "tooltip": {
+ "shared": false,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "s",
+ "label": "time. read (-) / write (+)",
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "description": "",
+ "fill": 2,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 12,
+ "y": 39
+ },
+ "hiddenSeries": false,
+ "id": 35,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "hideZero": true,
+ "max": true,
+ "min": true,
+ "rightSide": false,
+ "show": true,
+ "sort": "current",
+ "sortDesc": true,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "maxPerRow": 6,
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [
+ {
+ "alias": "/.*sda_.*/",
+ "color": "#7EB26D"
+ },
+ {
+ "alias": "/.*sdb_.*/",
+ "color": "#EAB839"
+ },
+ {
+ "alias": "/.*sdc_.*/",
+ "color": "#6ED0E0"
+ },
+ {
+ "alias": "/.*sdd_.*/",
+ "color": "#EF843C"
+ },
+ {
+ "alias": "/.*sde_.*/",
+ "color": "#E24D42"
+ },
+ {
+ "alias": "/.*sda1.*/",
+ "color": "#584477"
+ },
+ {
+ "alias": "/.*sda2_.*/",
+ "color": "#BA43A9"
+ },
+ {
+ "alias": "/.*sda3_.*/",
+ "color": "#F4D598"
+ },
+ {
+ "alias": "/.*sdb1.*/",
+ "color": "#0A50A1"
+ },
+ {
+ "alias": "/.*sdb2.*/",
+ "color": "#BF1B00"
+ },
+ {
+ "alias": "/.*sdb3.*/",
+ "color": "#E0752D"
+ },
+ {
+ "alias": "/.*sdc1.*/",
+ "color": "#962D82"
+ },
+ {
+ "alias": "/.*sdc2.*/",
+ "color": "#614D93"
+ },
+ {
+ "alias": "/.*sdc3.*/",
+ "color": "#9AC48A"
+ },
+ {
+ "alias": "/.*sdd1.*/",
+ "color": "#65C5DB"
+ },
+ {
+ "alias": "/.*sdd2.*/",
+ "color": "#F9934E"
+ },
+ {
+ "alias": "/.*sdd3.*/",
+ "color": "#EA6460"
+ },
+ {
+ "alias": "/.*sde1.*/",
+ "color": "#E0F9D7"
+ },
+ {
+ "alias": "/.*sdd2.*/",
+ "color": "#FCEACA"
+ },
+ {
+ "alias": "/.*sde3.*/",
+ "color": "#F9E2D2"
+ }
+ ],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "irate(node_disk_io_time_weighted_seconds_total{instance=~\"$node:$port\",job=~\"$job\"}[5m])",
+ "intervalFactor": 4,
+ "legendFormat": "{{device}} - IO time weighted",
+ "refId": "A",
+ "step": 8
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "Disk IOs Weighted",
+ "tooltip": {
+ "shared": false,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "s",
+ "label": "time",
+ "logBase": 1,
+ "max": null,
+ "min": "0",
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "description": "",
+ "fill": 2,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 0,
+ "y": 49
+ },
+ "hiddenSeries": false,
+ "id": 133,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "hideZero": true,
+ "max": true,
+ "min": true,
+ "rightSide": false,
+ "show": true,
+ "sort": "current",
+ "sortDesc": true,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "maxPerRow": 6,
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [
+ {
+ "alias": "/.*Read.*/",
+ "transform": "negative-Y"
+ },
+ {
+ "alias": "/.*sda_.*/",
+ "color": "#7EB26D"
+ },
+ {
+ "alias": "/.*sdb_.*/",
+ "color": "#EAB839"
+ },
+ {
+ "alias": "/.*sdc_.*/",
+ "color": "#6ED0E0"
+ },
+ {
+ "alias": "/.*sdd_.*/",
+ "color": "#EF843C"
+ },
+ {
+ "alias": "/.*sde_.*/",
+ "color": "#E24D42"
+ },
+ {
+ "alias": "/.*sda1.*/",
+ "color": "#584477"
+ },
+ {
+ "alias": "/.*sda2_.*/",
+ "color": "#BA43A9"
+ },
+ {
+ "alias": "/.*sda3_.*/",
+ "color": "#F4D598"
+ },
+ {
+ "alias": "/.*sdb1.*/",
+ "color": "#0A50A1"
+ },
+ {
+ "alias": "/.*sdb2.*/",
+ "color": "#BF1B00"
+ },
+ {
+ "alias": "/.*sdb3.*/",
+ "color": "#E0752D"
+ },
+ {
+ "alias": "/.*sdc1.*/",
+ "color": "#962D82"
+ },
+ {
+ "alias": "/.*sdc2.*/",
+ "color": "#614D93"
+ },
+ {
+ "alias": "/.*sdc3.*/",
+ "color": "#9AC48A"
+ },
+ {
+ "alias": "/.*sdd1.*/",
+ "color": "#65C5DB"
+ },
+ {
+ "alias": "/.*sdd2.*/",
+ "color": "#F9934E"
+ },
+ {
+ "alias": "/.*sdd3.*/",
+ "color": "#EA6460"
+ },
+ {
+ "alias": "/.*sde1.*/",
+ "color": "#E0F9D7"
+ },
+ {
+ "alias": "/.*sdd2.*/",
+ "color": "#FCEACA"
+ },
+ {
+ "alias": "/.*sde3.*/",
+ "color": "#F9E2D2"
+ }
+ ],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "irate(node_disk_reads_merged_total{instance=~\"$node:$port\",job=~\"$job\"}[5m])",
+ "intervalFactor": 2,
+ "legendFormat": "{{device}} - Read merged",
+ "refId": "A",
+ "step": 4
+ },
+ {
+ "expr": "irate(node_disk_writes_merged_total{instance=~\"$node:$port\",job=~\"$job\"}[5m])",
+ "intervalFactor": 2,
+ "legendFormat": "{{device}} - Write merged",
+ "refId": "B",
+ "step": 4
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "Disk R/W Merged",
+ "tooltip": {
+ "shared": false,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "iops",
+ "label": "I/Os",
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "description": "",
+ "fill": 3,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 12,
+ "y": 49
+ },
+ "hiddenSeries": false,
+ "id": 36,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "hideZero": true,
+ "max": true,
+ "min": true,
+ "rightSide": false,
+ "show": true,
+ "sort": "current",
+ "sortDesc": true,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "maxPerRow": 6,
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [
+ {
+ "alias": "/.*sda_.*/",
+ "color": "#7EB26D"
+ },
+ {
+ "alias": "/.*sdb_.*/",
+ "color": "#EAB839"
+ },
+ {
+ "alias": "/.*sdc_.*/",
+ "color": "#6ED0E0"
+ },
+ {
+ "alias": "/.*sdd_.*/",
+ "color": "#EF843C"
+ },
+ {
+ "alias": "/.*sde_.*/",
+ "color": "#E24D42"
+ },
+ {
+ "alias": "/.*sda1.*/",
+ "color": "#584477"
+ },
+ {
+ "alias": "/.*sda2_.*/",
+ "color": "#BA43A9"
+ },
+ {
+ "alias": "/.*sda3_.*/",
+ "color": "#F4D598"
+ },
+ {
+ "alias": "/.*sdb1.*/",
+ "color": "#0A50A1"
+ },
+ {
+ "alias": "/.*sdb2.*/",
+ "color": "#BF1B00"
+ },
+ {
+ "alias": "/.*sdb3.*/",
+ "color": "#E0752D"
+ },
+ {
+ "alias": "/.*sdc1.*/",
+ "color": "#962D82"
+ },
+ {
+ "alias": "/.*sdc2.*/",
+ "color": "#614D93"
+ },
+ {
+ "alias": "/.*sdc3.*/",
+ "color": "#9AC48A"
+ },
+ {
+ "alias": "/.*sdd1.*/",
+ "color": "#65C5DB"
+ },
+ {
+ "alias": "/.*sdd2.*/",
+ "color": "#F9934E"
+ },
+ {
+ "alias": "/.*sdd3.*/",
+ "color": "#EA6460"
+ },
+ {
+ "alias": "/.*sde1.*/",
+ "color": "#E0F9D7"
+ },
+ {
+ "alias": "/.*sdd2.*/",
+ "color": "#FCEACA"
+ },
+ {
+ "alias": "/.*sde3.*/",
+ "color": "#F9E2D2"
+ }
+ ],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "irate(node_disk_io_time_seconds_total{instance=~\"$node:$port\",job=~\"$job\"}[5m])",
+ "intervalFactor": 4,
+ "legendFormat": "{{device}} - IO time",
+ "refId": "A",
+ "step": 8
+ },
+ {
+ "expr": "irate(node_disk_discard_time_seconds_total{instance=~\"$node:$port\",job=~\"$job\"}[5m])",
+ "interval": "",
+ "intervalFactor": 4,
+ "legendFormat": "{{device}} - discard time",
+ "refId": "B",
+ "step": 8
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "Time Spent Doing I/Os",
+ "tooltip": {
+ "shared": false,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "s",
+ "label": "time",
+ "logBase": 1,
+ "max": null,
+ "min": "0",
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "description": "",
+ "fill": 2,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 0,
+ "y": 59
+ },
+ "hiddenSeries": false,
+ "id": 34,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "hideZero": true,
+ "max": true,
+ "min": true,
+ "rightSide": false,
+ "show": true,
+ "sort": "current",
+ "sortDesc": true,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "maxPerRow": 6,
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [
+ {
+ "alias": "/.*sda_.*/",
+ "color": "#7EB26D"
+ },
+ {
+ "alias": "/.*sdb_.*/",
+ "color": "#EAB839"
+ },
+ {
+ "alias": "/.*sdc_.*/",
+ "color": "#6ED0E0"
+ },
+ {
+ "alias": "/.*sdd_.*/",
+ "color": "#EF843C"
+ },
+ {
+ "alias": "/.*sde_.*/",
+ "color": "#E24D42"
+ },
+ {
+ "alias": "/.*sda1.*/",
+ "color": "#584477"
+ },
+ {
+ "alias": "/.*sda2_.*/",
+ "color": "#BA43A9"
+ },
+ {
+ "alias": "/.*sda3_.*/",
+ "color": "#F4D598"
+ },
+ {
+ "alias": "/.*sdb1.*/",
+ "color": "#0A50A1"
+ },
+ {
+ "alias": "/.*sdb2.*/",
+ "color": "#BF1B00"
+ },
+ {
+ "alias": "/.*sdb3.*/",
+ "color": "#E0752D"
+ },
+ {
+ "alias": "/.*sdc1.*/",
+ "color": "#962D82"
+ },
+ {
+ "alias": "/.*sdc2.*/",
+ "color": "#614D93"
+ },
+ {
+ "alias": "/.*sdc3.*/",
+ "color": "#9AC48A"
+ },
+ {
+ "alias": "/.*sdd1.*/",
+ "color": "#65C5DB"
+ },
+ {
+ "alias": "/.*sdd2.*/",
+ "color": "#F9934E"
+ },
+ {
+ "alias": "/.*sdd3.*/",
+ "color": "#EA6460"
+ },
+ {
+ "alias": "/.*sde1.*/",
+ "color": "#E0F9D7"
+ },
+ {
+ "alias": "/.*sdd2.*/",
+ "color": "#FCEACA"
+ },
+ {
+ "alias": "/.*sde3.*/",
+ "color": "#F9E2D2"
+ }
+ ],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "node_disk_io_now{instance=~\"$node:$port\",job=~\"$job\"}",
+ "interval": "",
+ "intervalFactor": 4,
+ "legendFormat": "{{device}} - IO now",
+ "refId": "A",
+ "step": 8
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "Disk IOs Current in Progress",
+ "tooltip": {
+ "shared": false,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "iops",
+ "label": "I/Os",
+ "logBase": 1,
+ "max": null,
+ "min": "0",
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "description": "",
+ "fill": 2,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 12,
+ "y": 59
+ },
+ "hiddenSeries": false,
+ "id": 301,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "hideZero": true,
+ "max": true,
+ "min": true,
+ "rightSide": false,
+ "show": true,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "maxPerRow": 6,
+ "nullPointMode": "null as zero",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [
+ {
+ "alias": "/.*sda_.*/",
+ "color": "#7EB26D"
+ },
+ {
+ "alias": "/.*sdb_.*/",
+ "color": "#EAB839"
+ },
+ {
+ "alias": "/.*sdc_.*/",
+ "color": "#6ED0E0"
+ },
+ {
+ "alias": "/.*sdd_.*/",
+ "color": "#EF843C"
+ },
+ {
+ "alias": "/.*sde_.*/",
+ "color": "#E24D42"
+ },
+ {
+ "alias": "/.*sda1.*/",
+ "color": "#584477"
+ },
+ {
+ "alias": "/.*sda2_.*/",
+ "color": "#BA43A9"
+ },
+ {
+ "alias": "/.*sda3_.*/",
+ "color": "#F4D598"
+ },
+ {
+ "alias": "/.*sdb1.*/",
+ "color": "#0A50A1"
+ },
+ {
+ "alias": "/.*sdb2.*/",
+ "color": "#BF1B00"
+ },
+ {
+ "alias": "/.*sdb3.*/",
+ "color": "#E0752D"
+ },
+ {
+ "alias": "/.*sdc1.*/",
+ "color": "#962D82"
+ },
+ {
+ "alias": "/.*sdc2.*/",
+ "color": "#614D93"
+ },
+ {
+ "alias": "/.*sdc3.*/",
+ "color": "#9AC48A"
+ },
+ {
+ "alias": "/.*sdd1.*/",
+ "color": "#65C5DB"
+ },
+ {
+ "alias": "/.*sdd2.*/",
+ "color": "#F9934E"
+ },
+ {
+ "alias": "/.*sdd3.*/",
+ "color": "#EA6460"
+ },
+ {
+ "alias": "/.*sde1.*/",
+ "color": "#E0F9D7"
+ },
+ {
+ "alias": "/.*sdd2.*/",
+ "color": "#FCEACA"
+ },
+ {
+ "alias": "/.*sde3.*/",
+ "color": "#F9E2D2"
+ }
+ ],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "irate(node_disk_discards_completed_total{instance=~\"$node:$port\",job=~\"$job\"}[5m])",
+ "interval": "",
+ "intervalFactor": 4,
+ "legendFormat": "{{device}} - Discards completed",
+ "refId": "A",
+ "step": 8
+ },
+ {
+ "expr": "irate(node_disk_discards_merged_total{instance=~\"$node:$port\",job=~\"$job\"}[5m])",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "{{device}} - Discards merged",
+ "refId": "B",
+ "step": 4
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "Disk IOps Discards completed / merged",
+ "tooltip": {
+ "shared": false,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "iops",
+ "label": "IOs",
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ }
+ ],
+ "repeat": null,
+ "title": "Storage Disk",
+ "type": "row"
+ },
+ {
+ "collapsed": true,
+ "datasource": "${DS_PROMETHEUS}",
+ "gridPos": {
+ "h": 1,
+ "w": 24,
+ "x": 0,
+ "y": 67
+ },
+ "id": 271,
+ "panels": [
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "decimals": 3,
+ "description": "",
+ "fill": 2,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 0,
+ "y": 29
+ },
+ "hiddenSeries": false,
+ "id": 43,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "max": true,
+ "min": true,
+ "rightSide": false,
+ "show": true,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "maxPerRow": 6,
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "node_filesystem_avail_bytes{instance=~\"$node:$port\",job=~\"$job\",device!~'rootfs'}",
+ "format": "time_series",
+ "hide": false,
+ "intervalFactor": 2,
+ "legendFormat": "{{mountpoint}} - Available",
+ "metric": "",
+ "refId": "A",
+ "step": 4
+ },
+ {
+ "expr": "node_filesystem_free_bytes{instance=~\"$node:$port\",job=~\"$job\",device!~'rootfs'}",
+ "format": "time_series",
+ "hide": true,
+ "intervalFactor": 2,
+ "legendFormat": "{{mountpoint}} - Free",
+ "refId": "B",
+ "step": 2
+ },
+ {
+ "expr": "node_filesystem_size_bytes{instance=~\"$node:$port\",job=~\"$job\",device!~'rootfs'}",
+ "format": "time_series",
+ "hide": true,
+ "intervalFactor": 2,
+ "legendFormat": "{{mountpoint}} - Size",
+ "refId": "C",
+ "step": 2
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "Filesystem space available",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "bytes",
+ "label": "bytes",
+ "logBase": 1,
+ "max": null,
+ "min": "0",
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "description": "",
+ "fill": 2,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 12,
+ "y": 29
+ },
+ "hiddenSeries": false,
+ "id": 41,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "hideZero": true,
+ "max": true,
+ "min": true,
+ "rightSide": false,
+ "show": true,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "node_filesystem_files_free{instance=~\"$node:$port\",job=~\"$job\",device!~'rootfs'}",
+ "format": "time_series",
+ "hide": false,
+ "intervalFactor": 2,
+ "legendFormat": "{{mountpoint}} - Free file nodes",
+ "refId": "A",
+ "step": 4
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "File Nodes Free",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "short",
+ "label": "file nodes",
+ "logBase": 1,
+ "max": null,
+ "min": "0",
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "description": "",
+ "fill": 2,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 0,
+ "y": 39
+ },
+ "hiddenSeries": false,
+ "id": 28,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "max": true,
+ "min": true,
+ "show": true,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "maxPerRow": 6,
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "node_filefd_maximum{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "intervalFactor": 4,
+ "legendFormat": "Max open files",
+ "refId": "A",
+ "step": 8
+ },
+ {
+ "expr": "node_filefd_allocated{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "intervalFactor": 2,
+ "legendFormat": "Open files",
+ "refId": "B",
+ "step": 4
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "File Descriptor",
+ "tooltip": {
+ "shared": false,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "short",
+ "label": "files",
+ "logBase": 1,
+ "max": null,
+ "min": "0",
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "description": "",
+ "fill": 2,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 12,
+ "y": 39
+ },
+ "hiddenSeries": false,
+ "id": 219,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "hideZero": true,
+ "max": true,
+ "min": true,
+ "rightSide": false,
+ "show": true,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "node_filesystem_files{instance=~\"$node:$port\",job=~\"$job\",device!~'rootfs'}",
+ "format": "time_series",
+ "hide": false,
+ "intervalFactor": 2,
+ "legendFormat": "{{mountpoint}} - File nodes total",
+ "refId": "A",
+ "step": 4
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "File Nodes Size",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "short",
+ "label": "file Nodes",
+ "logBase": 1,
+ "max": null,
+ "min": "0",
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {
+ "/ ReadOnly": "#890F02"
+ },
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "decimals": null,
+ "description": "",
+ "fill": 2,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 0,
+ "y": 49
+ },
+ "hiddenSeries": false,
+ "id": 44,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "hideEmpty": true,
+ "hideZero": true,
+ "max": true,
+ "min": true,
+ "rightSide": false,
+ "show": true,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "maxPerRow": 6,
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "stack": true,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "node_filesystem_readonly{instance=~\"$node:$port\",job=~\"$job\",device!~'rootfs'}",
+ "format": "time_series",
+ "intervalFactor": 2,
+ "legendFormat": "{{mountpoint}} - ReadOnly",
+ "refId": "A",
+ "step": 4
+ },
+ {
+ "expr": "node_filesystem_device_error{instance=~\"$node:$port\",job=~\"$job\",device!~'rootfs'}",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "{{mountpoint}} - Device error",
+ "refId": "B",
+ "step": 4
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "Filesystem in ReadOnly / Error",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "short",
+ "label": "counter",
+ "logBase": 1,
+ "max": "1",
+ "min": "0",
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ }
+ ],
+ "repeat": null,
+ "title": "Storage Filesystem",
+ "type": "row"
+ },
+ {
+ "collapsed": true,
+ "datasource": "${DS_PROMETHEUS}",
+ "gridPos": {
+ "h": 1,
+ "w": 24,
+ "x": 0,
+ "y": 68
+ },
+ "id": 272,
+ "panels": [
+ {
+ "aliasColors": {
+ "receive_packets_eth0": "#7EB26D",
+ "receive_packets_lo": "#E24D42",
+ "transmit_packets_eth0": "#7EB26D",
+ "transmit_packets_lo": "#E24D42"
+ },
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "fieldConfig": {
+ "defaults": {
+ "custom": {}
+ },
+ "overrides": []
+ },
+ "fill": 2,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 0,
+ "y": 109
+ },
+ "hiddenSeries": false,
+ "id": 60,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "max": true,
+ "min": true,
+ "rightSide": false,
+ "show": true,
+ "sideWidth": 300,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [
+ {
+ "alias": "/.*Trans.*/",
+ "transform": "negative-Y"
+ }
+ ],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "irate(node_network_receive_packets_total{instance=~\"$node:$port\",job=~\"$job\"}[5m])",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "{{device}} - Receive",
+ "refId": "A",
+ "step": 4
+ },
+ {
+ "expr": "irate(node_network_transmit_packets_total{instance=~\"$node:$port\",job=~\"$job\"}[5m])",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "{{device}} - Transmit",
+ "refId": "B",
+ "step": 4
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "Network Traffic by Packets",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "pps",
+ "label": "packets out (-) / in (+)",
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "fieldConfig": {
+ "defaults": {
+ "custom": {}
+ },
+ "overrides": []
+ },
+ "fill": 2,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 12,
+ "y": 109
+ },
+ "hiddenSeries": false,
+ "id": 142,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "hideEmpty": false,
+ "hideZero": false,
+ "max": true,
+ "min": true,
+ "rightSide": false,
+ "show": true,
+ "sideWidth": 300,
+ "sort": "current",
+ "sortDesc": true,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [
+ {
+ "alias": "/.*Trans.*/",
+ "transform": "negative-Y"
+ }
+ ],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "irate(node_network_receive_errs_total{instance=~\"$node:$port\",job=~\"$job\"}[5m])",
+ "format": "time_series",
+ "intervalFactor": 2,
+ "legendFormat": "{{device}} - Receive errors",
+ "refId": "A",
+ "step": 4
+ },
+ {
+ "expr": "irate(node_network_transmit_errs_total{instance=~\"$node:$port\",job=~\"$job\"}[5m])",
+ "format": "time_series",
+ "intervalFactor": 2,
+ "legendFormat": "{{device}} - Rransmit errors",
+ "refId": "B",
+ "step": 4
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "Network Traffic Errors",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "pps",
+ "label": "packets out (-) / in (+)",
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "fieldConfig": {
+ "defaults": {
+ "custom": {}
+ },
+ "overrides": []
+ },
+ "fill": 2,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 0,
+ "y": 119
+ },
+ "hiddenSeries": false,
+ "id": 143,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "hideEmpty": false,
+ "hideZero": false,
+ "max": true,
+ "min": true,
+ "rightSide": false,
+ "show": true,
+ "sideWidth": 300,
+ "sort": "current",
+ "sortDesc": true,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [
+ {
+ "alias": "/.*Trans.*/",
+ "transform": "negative-Y"
+ }
+ ],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "irate(node_network_receive_drop_total{instance=~\"$node:$port\",job=~\"$job\"}[5m])",
+ "format": "time_series",
+ "intervalFactor": 2,
+ "legendFormat": "{{device}} - Receive drop",
+ "refId": "A",
+ "step": 4
+ },
+ {
+ "expr": "irate(node_network_transmit_drop_total{instance=~\"$node:$port\",job=~\"$job\"}[5m])",
+ "format": "time_series",
+ "intervalFactor": 2,
+ "legendFormat": "{{device}} - Transmit drop",
+ "refId": "B",
+ "step": 4
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "Network Traffic Drop",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "pps",
+ "label": "packets out (-) / in (+)",
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "fieldConfig": {
+ "defaults": {
+ "custom": {}
+ },
+ "overrides": []
+ },
+ "fill": 2,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 12,
+ "y": 119
+ },
+ "hiddenSeries": false,
+ "id": 141,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "hideEmpty": false,
+ "hideZero": false,
+ "max": true,
+ "min": true,
+ "rightSide": false,
+ "show": true,
+ "sideWidth": 300,
+ "sort": "current",
+ "sortDesc": true,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [
+ {
+ "alias": "/.*Trans.*/",
+ "transform": "negative-Y"
+ }
+ ],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "irate(node_network_receive_compressed_total{instance=~\"$node:$port\",job=~\"$job\"}[5m])",
+ "format": "time_series",
+ "intervalFactor": 2,
+ "legendFormat": "{{device}} - Receive compressed",
+ "refId": "A",
+ "step": 4
+ },
+ {
+ "expr": "irate(node_network_transmit_compressed_total{instance=~\"$node:$port\",job=~\"$job\"}[5m])",
+ "format": "time_series",
+ "intervalFactor": 2,
+ "legendFormat": "{{device}} - Transmit compressed",
+ "refId": "B",
+ "step": 4
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "Network Traffic Compressed",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "pps",
+ "label": "packets out (-) / in (+)",
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "fieldConfig": {
+ "defaults": {
+ "custom": {}
+ },
+ "overrides": []
+ },
+ "fill": 2,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 0,
+ "y": 129
+ },
+ "hiddenSeries": false,
+ "id": 146,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "hideEmpty": false,
+ "hideZero": false,
+ "max": true,
+ "min": true,
+ "rightSide": false,
+ "show": true,
+ "sideWidth": 300,
+ "sort": "current",
+ "sortDesc": true,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [
+ {
+ "alias": "/.*Trans.*/",
+ "transform": "negative-Y"
+ }
+ ],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "irate(node_network_receive_multicast_total{instance=~\"$node:$port\",job=~\"$job\"}[5m])",
+ "format": "time_series",
+ "intervalFactor": 2,
+ "legendFormat": "{{device}} - Receive multicast",
+ "refId": "A",
+ "step": 4
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "Network Traffic Multicast",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "pps",
+ "label": "packets out (-) / in (+)",
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "fieldConfig": {
+ "defaults": {
+ "custom": {}
+ },
+ "overrides": []
+ },
+ "fill": 2,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 12,
+ "y": 129
+ },
+ "hiddenSeries": false,
+ "id": 144,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "hideEmpty": false,
+ "hideZero": false,
+ "max": true,
+ "min": true,
+ "rightSide": false,
+ "show": true,
+ "sideWidth": 300,
+ "sort": "current",
+ "sortDesc": true,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [
+ {
+ "alias": "/.*Trans.*/",
+ "transform": "negative-Y"
+ }
+ ],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "irate(node_network_receive_fifo_total{instance=~\"$node:$port\",job=~\"$job\"}[5m])",
+ "format": "time_series",
+ "intervalFactor": 2,
+ "legendFormat": "{{device}} - Receive fifo",
+ "refId": "A",
+ "step": 4
+ },
+ {
+ "expr": "irate(node_network_transmit_fifo_total{instance=~\"$node:$port\",job=~\"$job\"}[5m])",
+ "format": "time_series",
+ "intervalFactor": 2,
+ "legendFormat": "{{device}} - Transmit fifo",
+ "refId": "B",
+ "step": 4
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "Network Traffic Fifo",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "pps",
+ "label": "packets out (-) / in (+)",
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "fieldConfig": {
+ "defaults": {
+ "custom": {}
+ },
+ "overrides": []
+ },
+ "fill": 2,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 0,
+ "y": 139
+ },
+ "hiddenSeries": false,
+ "id": 145,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "hideEmpty": false,
+ "hideZero": false,
+ "max": true,
+ "min": true,
+ "rightSide": false,
+ "show": true,
+ "sideWidth": 300,
+ "sort": "current",
+ "sortDesc": true,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [
+ {
+ "alias": "/.*Trans.*/",
+ "transform": "negative-Y"
+ }
+ ],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "irate(node_network_receive_frame_total{instance=~\"$node:$port\",job=~\"$job\"}[5m])",
+ "format": "time_series",
+ "hide": false,
+ "intervalFactor": 2,
+ "legendFormat": "{{device}} - Receive frame",
+ "refId": "A",
+ "step": 4
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "Network Traffic Frame",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "pps",
+ "label": "packets out (-) / in (+)",
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "fieldConfig": {
+ "defaults": {
+ "custom": {}
+ },
+ "overrides": []
+ },
+ "fill": 2,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 12,
+ "y": 139
+ },
+ "hiddenSeries": false,
+ "id": 231,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "hideEmpty": false,
+ "hideZero": false,
+ "max": true,
+ "min": true,
+ "rightSide": false,
+ "show": true,
+ "sideWidth": 300,
+ "sort": "current",
+ "sortDesc": true,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "irate(node_network_transmit_carrier_total{instance=~\"$node:$port\",job=~\"$job\"}[5m])",
+ "format": "time_series",
+ "intervalFactor": 2,
+ "legendFormat": "{{device}} - Statistic transmit_carrier",
+ "refId": "A",
+ "step": 4
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "Network Traffic Carrier",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "short",
+ "label": "counter",
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "fieldConfig": {
+ "defaults": {
+ "custom": {}
+ },
+ "overrides": []
+ },
+ "fill": 2,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 0,
+ "y": 149
+ },
+ "hiddenSeries": false,
+ "id": 232,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "hideEmpty": false,
+ "hideZero": false,
+ "max": true,
+ "min": true,
+ "rightSide": false,
+ "show": true,
+ "sideWidth": 300,
+ "sort": "current",
+ "sortDesc": true,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [
+ {
+ "alias": "/.*Trans.*/",
+ "transform": "negative-Y"
+ }
+ ],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "irate(node_network_transmit_colls_total{instance=~\"$node:$port\",job=~\"$job\"}[5m])",
+ "format": "time_series",
+ "intervalFactor": 2,
+ "legendFormat": "{{device}} - Transmit colls",
+ "refId": "A",
+ "step": 4
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "Network Traffic Colls",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "short",
+ "label": "counter",
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "fieldConfig": {
+ "defaults": {
+ "custom": {}
+ },
+ "overrides": []
+ },
+ "fill": 2,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 12,
+ "y": 149
+ },
+ "hiddenSeries": false,
+ "id": 61,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "max": true,
+ "min": true,
+ "rightSide": false,
+ "show": true,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [
+ {
+ "alias": "NF conntrack limit",
+ "color": "#890F02",
+ "fill": 0
+ }
+ ],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "node_nf_conntrack_entries{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "intervalFactor": 2,
+ "legendFormat": "NF conntrack entries",
+ "refId": "A",
+ "step": 4
+ },
+ {
+ "expr": "node_nf_conntrack_entries_limit{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "intervalFactor": 2,
+ "legendFormat": "NF conntrack limit",
+ "refId": "B",
+ "step": 4
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "NF Contrack",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "short",
+ "label": "entries",
+ "logBase": 1,
+ "max": null,
+ "min": "0",
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "fieldConfig": {
+ "defaults": {
+ "custom": {}
+ },
+ "overrides": []
+ },
+ "fill": 2,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 0,
+ "y": 159
+ },
+ "hiddenSeries": false,
+ "id": 230,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "max": true,
+ "min": true,
+ "rightSide": false,
+ "show": true,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "node_arp_entries{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "intervalFactor": 2,
+ "legendFormat": "{{ device }} - ARP entries",
+ "refId": "A",
+ "step": 4
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "ARP Entries",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "short",
+ "label": "Entries",
+ "logBase": 1,
+ "max": null,
+ "min": "0",
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "fieldConfig": {
+ "defaults": {
+ "custom": {}
+ },
+ "overrides": []
+ },
+ "fill": 2,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 12,
+ "y": 159
+ },
+ "hiddenSeries": false,
+ "id": 288,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "max": true,
+ "min": true,
+ "rightSide": false,
+ "show": true,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 1,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "node_network_mtu_bytes{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "intervalFactor": 2,
+ "legendFormat": "{{ device }} - Bytes",
+ "refId": "A",
+ "step": 4
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "MTU",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "decimals": 0,
+ "format": "bytes",
+ "label": "bytes",
+ "logBase": 1,
+ "max": null,
+ "min": "0",
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "fieldConfig": {
+ "defaults": {
+ "custom": {}
+ },
+ "overrides": []
+ },
+ "fill": 2,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 0,
+ "y": 169
+ },
+ "hiddenSeries": false,
+ "id": 280,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "max": true,
+ "min": true,
+ "rightSide": false,
+ "show": true,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 1,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "node_network_speed_bytes{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "intervalFactor": 2,
+ "legendFormat": "{{ device }} - Speed",
+ "refId": "A",
+ "step": 4
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "Speed",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "decimals": 0,
+ "format": "bytes",
+ "label": "bytes",
+ "logBase": 1,
+ "max": null,
+ "min": "0",
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "fieldConfig": {
+ "defaults": {
+ "custom": {}
+ },
+ "overrides": []
+ },
+ "fill": 2,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 12,
+ "y": 169
+ },
+ "hiddenSeries": false,
+ "id": 289,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "max": true,
+ "min": true,
+ "rightSide": false,
+ "show": true,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 1,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "node_network_transmit_queue_length{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "intervalFactor": 2,
+ "legendFormat": "{{ device }} - Interface transmit queue length",
+ "refId": "A",
+ "step": 4
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "Queue Length",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "decimals": 0,
+ "format": "none",
+ "label": "packets",
+ "logBase": 1,
+ "max": null,
+ "min": "0",
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "fieldConfig": {
+ "defaults": {
+ "custom": {}
+ },
+ "overrides": []
+ },
+ "fill": 2,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 0,
+ "y": 179
+ },
+ "hiddenSeries": false,
+ "id": 290,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "hideEmpty": false,
+ "hideZero": false,
+ "max": true,
+ "min": true,
+ "rightSide": false,
+ "show": true,
+ "sideWidth": 300,
+ "sort": "current",
+ "sortDesc": true,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [
+ {
+ "alias": "/.*Dropped.*/",
+ "transform": "negative-Y"
+ }
+ ],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "irate(node_softnet_processed_total{instance=~\"$node:$port\",job=~\"$job\"}[5m])",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "CPU {{cpu}} - Processed",
+ "refId": "A",
+ "step": 4
+ },
+ {
+ "expr": "irate(node_softnet_dropped_total{instance=~\"$node:$port\",job=~\"$job\"}[5m])",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "CPU {{cpu}} - Dropped",
+ "refId": "B",
+ "step": 4
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "Softnet Packets",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "short",
+ "label": "packetes drop (-) / process (+)",
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "fieldConfig": {
+ "defaults": {
+ "custom": {}
+ },
+ "overrides": []
+ },
+ "fill": 2,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 12,
+ "y": 179
+ },
+ "hiddenSeries": false,
+ "id": 310,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "hideEmpty": false,
+ "hideZero": false,
+ "max": true,
+ "min": true,
+ "rightSide": false,
+ "show": true,
+ "sideWidth": 300,
+ "sort": "current",
+ "sortDesc": true,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "irate(node_softnet_times_squeezed_total{instance=~\"$node:$port\",job=~\"$job\"}[5m])",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "CPU {{cpu}} - Squeezed",
+ "refId": "A",
+ "step": 4
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "Softnet Out of Quota",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "short",
+ "label": "counter",
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "fieldConfig": {
+ "defaults": {
+ "custom": {}
+ },
+ "overrides": []
+ },
+ "fill": 2,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 0,
+ "y": 189
+ },
+ "hiddenSeries": false,
+ "id": 309,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "hideEmpty": false,
+ "hideZero": false,
+ "max": true,
+ "min": true,
+ "rightSide": false,
+ "show": true,
+ "sideWidth": 300,
+ "sort": "current",
+ "sortDesc": true,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "node_network_up{operstate=\"up\",instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "intervalFactor": 2,
+ "legendFormat": "{{interface}} - Operational state UP",
+ "refId": "A",
+ "step": 4
+ },
+ {
+ "expr": "node_network_carrier{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "instant": false,
+ "legendFormat": "{{device}} - Physical link state",
+ "refId": "B"
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "Network Operational Status",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "short",
+ "label": "counter",
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ }
+ ],
+ "repeat": null,
+ "title": "Network Traffic",
+ "type": "row"
+ },
+ {
+ "collapsed": true,
+ "datasource": "${DS_PROMETHEUS}",
+ "gridPos": {
+ "h": 1,
+ "w": 24,
+ "x": 0,
+ "y": 69
+ },
+ "id": 273,
+ "panels": [
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "fill": 2,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 0,
+ "y": 13
+ },
+ "hiddenSeries": false,
+ "id": 63,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "hideEmpty": false,
+ "hideZero": false,
+ "max": true,
+ "min": true,
+ "rightSide": false,
+ "show": true,
+ "sideWidth": 300,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "node_sockstat_TCP_alloc{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "TCP_alloc - Allocated sockets",
+ "refId": "A",
+ "step": 240
+ },
+ {
+ "expr": "node_sockstat_TCP_inuse{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "TCP_inuse - Tcp sockets currently in use",
+ "refId": "B",
+ "step": 240
+ },
+ {
+ "expr": "node_sockstat_TCP_mem{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "hide": true,
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "TCP_mem - Used memory for tcp",
+ "refId": "C",
+ "step": 240
+ },
+ {
+ "expr": "node_sockstat_TCP_orphan{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "TCP_orphan - Orphan sockets",
+ "refId": "D",
+ "step": 240
+ },
+ {
+ "expr": "node_sockstat_TCP_tw{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "TCP_tw - Sockets wating close",
+ "refId": "E",
+ "step": 240
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "Sockstat TCP",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "short",
+ "label": "counter",
+ "logBase": 1,
+ "max": null,
+ "min": "0",
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "fill": 2,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 12,
+ "y": 13
+ },
+ "hiddenSeries": false,
+ "id": 124,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "hideEmpty": false,
+ "hideZero": false,
+ "max": true,
+ "min": true,
+ "rightSide": false,
+ "show": true,
+ "sideWidth": 300,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "node_sockstat_UDPLITE_inuse{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "UDPLITE_inuse - Udplite sockets currently in use",
+ "refId": "A",
+ "step": 240
+ },
+ {
+ "expr": "node_sockstat_UDP_inuse{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "UDP_inuse - Udp sockets currently in use",
+ "refId": "B",
+ "step": 240
+ },
+ {
+ "expr": "node_sockstat_UDP_mem{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "UDP_mem - Used memory for udp",
+ "refId": "C",
+ "step": 240
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "Sockstat UDP",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "short",
+ "label": "counter",
+ "logBase": 1,
+ "max": null,
+ "min": "0",
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "fill": 2,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 0,
+ "y": 23
+ },
+ "hiddenSeries": false,
+ "id": 126,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "hideEmpty": false,
+ "hideZero": false,
+ "max": true,
+ "min": true,
+ "rightSide": false,
+ "show": true,
+ "sideWidth": 300,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "node_sockstat_sockets_used{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "Sockets_used - Sockets currently in use",
+ "refId": "A",
+ "step": 240
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "Sockstat Used",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "short",
+ "label": "sockets",
+ "logBase": 1,
+ "max": null,
+ "min": "0",
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "fill": 2,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 12,
+ "y": 23
+ },
+ "hiddenSeries": false,
+ "id": 220,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "hideEmpty": false,
+ "hideZero": false,
+ "max": true,
+ "min": true,
+ "rightSide": false,
+ "show": true,
+ "sideWidth": 300,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "node_sockstat_TCP_mem_bytes{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "mem_bytes - TCP sockets in that state",
+ "refId": "A",
+ "step": 240
+ },
+ {
+ "expr": "node_sockstat_UDP_mem_bytes{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "mem_bytes - UDP sockets in that state",
+ "refId": "B",
+ "step": 240
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "Sockstat Memory Size",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "bytes",
+ "label": "bytes",
+ "logBase": 1,
+ "max": null,
+ "min": "0",
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "fill": 2,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 0,
+ "y": 33
+ },
+ "hiddenSeries": false,
+ "id": 125,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "hideEmpty": false,
+ "hideZero": false,
+ "max": true,
+ "min": true,
+ "rightSide": false,
+ "show": true,
+ "sideWidth": 300,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "node_sockstat_FRAG_inuse{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "FRAG_inuse - Frag sockets currently in use",
+ "refId": "A",
+ "step": 240
+ },
+ {
+ "expr": "node_sockstat_FRAG_memory{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "FRAG_memory - Used memory for frag",
+ "refId": "B",
+ "step": 240
+ },
+ {
+ "expr": "node_sockstat_RAW_inuse{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "RAW_inuse - Raw sockets currently in use",
+ "refId": "C",
+ "step": 240
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "Sockstat FRAG / RAW",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "short",
+ "label": "counter",
+ "logBase": 1,
+ "max": null,
+ "min": "0",
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ }
+ ],
+ "repeat": null,
+ "title": "Network Sockstat",
+ "type": "row"
+ },
+ {
+ "collapsed": true,
+ "datasource": "${DS_PROMETHEUS}",
+ "gridPos": {
+ "h": 1,
+ "w": 24,
+ "x": 0,
+ "y": 70
+ },
+ "id": 274,
+ "panels": [
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "fill": 2,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 0,
+ "y": 32
+ },
+ "height": "",
+ "hiddenSeries": false,
+ "id": 221,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "hideEmpty": false,
+ "hideZero": false,
+ "max": true,
+ "min": true,
+ "rightSide": false,
+ "show": true,
+ "sideWidth": 300,
+ "sort": "current",
+ "sortDesc": true,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "maxPerRow": 12,
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [
+ {
+ "alias": "/.*Out.*/",
+ "transform": "negative-Y"
+ }
+ ],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "irate(node_netstat_IpExt_InOctets{instance=~\"$node:$port\",job=~\"$job\"}[5m])",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "InOctets - Received octets",
+ "refId": "A",
+ "step": 4
+ },
+ {
+ "expr": "irate(node_netstat_IpExt_OutOctets{instance=~\"$node:$port\",job=~\"$job\"}[5m])",
+ "format": "time_series",
+ "intervalFactor": 2,
+ "legendFormat": "OutOctets - Sent octets",
+ "refId": "B",
+ "step": 4
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "Netstat IP In / Out Octets",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "short",
+ "label": "octects out (-) / in (+)",
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "fill": 2,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 12,
+ "y": 32
+ },
+ "height": "",
+ "hiddenSeries": false,
+ "id": 81,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "hideEmpty": false,
+ "hideZero": false,
+ "max": true,
+ "min": true,
+ "rightSide": false,
+ "show": true,
+ "sideWidth": 300,
+ "sort": "current",
+ "sortDesc": true,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "irate(node_netstat_Ip_Forwarding{instance=~\"$node:$port\",job=~\"$job\"}[5m])",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "Forwarding - IP forwarding",
+ "refId": "A",
+ "step": 4
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "Netstat IP Forwarding",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "short",
+ "label": "datagrams",
+ "logBase": 1,
+ "max": null,
+ "min": "0",
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "decimals": null,
+ "fill": 2,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 0,
+ "y": 42
+ },
+ "height": "",
+ "hiddenSeries": false,
+ "id": 115,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "hideZero": false,
+ "max": true,
+ "min": true,
+ "rightSide": false,
+ "show": true,
+ "sort": "current",
+ "sortDesc": true,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "maxPerRow": 12,
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [
+ {
+ "alias": "/.*Out.*/",
+ "transform": "negative-Y"
+ }
+ ],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "irate(node_netstat_Icmp_InMsgs{instance=~\"$node:$port\",job=~\"$job\"}[5m])",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "InMsgs - Messages which the entity received. Note that this counter includes all those counted by icmpInErrors",
+ "refId": "A",
+ "step": 4
+ },
+ {
+ "expr": "irate(node_netstat_Icmp_OutMsgs{instance=~\"$node:$port\",job=~\"$job\"}[5m])",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "OutMsgs - Messages which this entity attempted to send. Note that this counter includes all those counted by icmpOutErrors",
+ "refId": "B",
+ "step": 4
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "ICMP In / Out",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "short",
+ "label": "messages out (-) / in (+)",
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "decimals": null,
+ "fill": 2,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 12,
+ "y": 42
+ },
+ "height": "",
+ "hiddenSeries": false,
+ "id": 50,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "hideZero": false,
+ "max": true,
+ "min": true,
+ "rightSide": false,
+ "show": true,
+ "sort": "current",
+ "sortDesc": true,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "maxPerRow": 12,
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [
+ {
+ "alias": "/.*Out.*/",
+ "transform": "negative-Y"
+ }
+ ],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "irate(node_netstat_Icmp_InErrors{instance=~\"$node:$port\",job=~\"$job\"}[5m])",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "InErrors - Messages which the entity received but determined as having ICMP-specific errors (bad ICMP checksums, bad length, etc.)",
+ "refId": "A",
+ "step": 4
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "ICMP Errors",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "short",
+ "label": "messages out (-) / in (+)",
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "decimals": null,
+ "fill": 2,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 0,
+ "y": 52
+ },
+ "height": "",
+ "hiddenSeries": false,
+ "id": 55,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "hideZero": false,
+ "max": true,
+ "min": true,
+ "rightSide": false,
+ "show": true,
+ "sort": "current",
+ "sortDesc": true,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "maxPerRow": 12,
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [
+ {
+ "alias": "/.*Out.*/",
+ "transform": "negative-Y"
+ },
+ {
+ "alias": "/.*Snd.*/",
+ "transform": "negative-Y"
+ }
+ ],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "irate(node_netstat_Udp_InDatagrams{instance=~\"$node:$port\",job=~\"$job\"}[5m])",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "InDatagrams - Datagrams received",
+ "refId": "A",
+ "step": 4
+ },
+ {
+ "expr": "irate(node_netstat_Udp_OutDatagrams{instance=~\"$node:$port\",job=~\"$job\"}[5m])",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "OutDatagrams - Datagrams sent",
+ "refId": "B",
+ "step": 4
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "UDP In / Out",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "short",
+ "label": "datagrams out (-) / in (+)",
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "fill": 2,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 12,
+ "y": 52
+ },
+ "height": "",
+ "hiddenSeries": false,
+ "id": 109,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "hideZero": false,
+ "max": true,
+ "min": true,
+ "rightSide": false,
+ "show": true,
+ "sort": "current",
+ "sortDesc": true,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "maxPerRow": 12,
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "irate(node_netstat_Udp_InErrors{instance=~\"$node:$port\",job=~\"$job\"}[5m])",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "InErrors - UDP Datagrams that could not be delivered to an application",
+ "refId": "A",
+ "step": 4
+ },
+ {
+ "expr": "irate(node_netstat_Udp_NoPorts{instance=~\"$node:$port\",job=~\"$job\"}[5m])",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "NoPorts - UDP Datagrams received on a port with no listener",
+ "refId": "B",
+ "step": 4
+ },
+ {
+ "expr": "irate(node_netstat_UdpLite_InErrors{instance=~\"$node:$port\",job=~\"$job\"}[5m])",
+ "interval": "",
+ "legendFormat": "InErrors Lite - UDPLite Datagrams that could not be delivered to an application",
+ "refId": "C"
+ },
+ {
+ "expr": "irate(node_netstat_Udp_RcvbufErrors{instance=~\"$node:$port\",job=~\"$job\"}[5m])",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "RcvbufErrors - UDP buffer errors received",
+ "refId": "D",
+ "step": 4
+ },
+ {
+ "expr": "irate(node_netstat_Udp_SndbufErrors{instance=~\"$node:$port\",job=~\"$job\"}[5m])",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "SndbufErrors - UDP buffer errors send",
+ "refId": "E",
+ "step": 4
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "UDP Errors",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "short",
+ "label": "datagrams",
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "decimals": null,
+ "fill": 2,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 0,
+ "y": 62
+ },
+ "height": "",
+ "hiddenSeries": false,
+ "id": 299,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "hideZero": false,
+ "max": true,
+ "min": true,
+ "rightSide": false,
+ "show": true,
+ "sort": "current",
+ "sortDesc": true,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "maxPerRow": 12,
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [
+ {
+ "alias": "/.*Out.*/",
+ "transform": "negative-Y"
+ },
+ {
+ "alias": "/.*Snd.*/",
+ "transform": "negative-Y"
+ }
+ ],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "irate(node_netstat_Tcp_InSegs{instance=~\"$node:$port\",job=~\"$job\"}[5m])",
+ "format": "time_series",
+ "instant": false,
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "InSegs - Segments received, including those received in error. This count includes segments received on currently established connections",
+ "refId": "A",
+ "step": 4
+ },
+ {
+ "expr": "irate(node_netstat_Tcp_OutSegs{instance=~\"$node:$port\",job=~\"$job\"}[5m])",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "OutSegs - Segments sent, including those on current connections but excluding those containing only retransmitted octets",
+ "refId": "B",
+ "step": 4
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "TCP In / Out",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "short",
+ "label": "datagrams out (-) / in (+)",
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "description": "",
+ "fill": 2,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 12,
+ "y": 62
+ },
+ "height": "",
+ "hiddenSeries": false,
+ "id": 104,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "hideEmpty": false,
+ "hideZero": false,
+ "max": true,
+ "min": true,
+ "rightSide": false,
+ "show": true,
+ "sort": "current",
+ "sortDesc": true,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "maxPerRow": 12,
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "irate(node_netstat_TcpExt_ListenOverflows{instance=~\"$node:$port\",job=~\"$job\"}[5m])",
+ "format": "time_series",
+ "hide": false,
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "ListenOverflows - Times the listen queue of a socket overflowed",
+ "refId": "A",
+ "step": 4
+ },
+ {
+ "expr": "irate(node_netstat_TcpExt_ListenDrops{instance=~\"$node:$port\",job=~\"$job\"}[5m])",
+ "format": "time_series",
+ "hide": false,
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "ListenDrops - SYNs to LISTEN sockets ignored",
+ "refId": "B",
+ "step": 4
+ },
+ {
+ "expr": "irate(node_netstat_TcpExt_TCPSynRetrans{instance=~\"$node:$port\",job=~\"$job\"}[5m])",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "TCPSynRetrans - SYN-SYN/ACK retransmits to break down retransmissions in SYN, fast/timeout retransmits",
+ "refId": "C",
+ "step": 4
+ },
+ {
+ "expr": "irate(node_netstat_Tcp_RetransSegs{instance=~\"$node:$port\",job=~\"$job\"}[5m])",
+ "interval": "",
+ "legendFormat": "RetransSegs - Segments retransmitted - that is, the number of TCP segments transmitted containing one or more previously transmitted octets",
+ "refId": "D"
+ },
+ {
+ "expr": "irate(node_netstat_Tcp_InErrs{instance=~\"$node:$port\",job=~\"$job\"}[5m])",
+ "interval": "",
+ "legendFormat": "InErrs - Segments received in error (e.g., bad TCP checksums)",
+ "refId": "E"
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "TCP Errors",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "short",
+ "label": "counter",
+ "logBase": 1,
+ "max": null,
+ "min": "0",
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "fill": 2,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 0,
+ "y": 72
+ },
+ "height": "",
+ "hiddenSeries": false,
+ "id": 85,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "hideZero": false,
+ "max": true,
+ "min": true,
+ "rightSide": false,
+ "show": true,
+ "sortDesc": true,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "maxPerRow": 12,
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [
+ {
+ "alias": "/.*MaxConn *./",
+ "color": "#890F02",
+ "fill": 0
+ }
+ ],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "node_netstat_Tcp_CurrEstab{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "hide": false,
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "CurrEstab - TCP connections for which the current state is either ESTABLISHED or CLOSE- WAIT",
+ "refId": "A",
+ "step": 4
+ },
+ {
+ "expr": "node_netstat_Tcp_MaxConn{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "hide": false,
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "MaxConn - Limit on the total number of TCP connections the entity can support (Dinamic is \"-1\")",
+ "refId": "B",
+ "step": 4
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "TCP Connections",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "short",
+ "label": "connections",
+ "logBase": 1,
+ "max": null,
+ "min": "0",
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "description": "",
+ "fill": 2,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 12,
+ "y": 72
+ },
+ "height": "",
+ "hiddenSeries": false,
+ "id": 91,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "hideEmpty": false,
+ "hideZero": false,
+ "max": true,
+ "min": true,
+ "rightSide": false,
+ "show": true,
+ "sort": "current",
+ "sortDesc": true,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "maxPerRow": 12,
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [
+ {
+ "alias": "/.*Sent.*/",
+ "transform": "negative-Y"
+ }
+ ],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "irate(node_netstat_TcpExt_SyncookiesFailed{instance=~\"$node:$port\",job=~\"$job\"}[5m])",
+ "format": "time_series",
+ "hide": false,
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "SyncookiesFailed - Invalid SYN cookies received",
+ "refId": "A",
+ "step": 4
+ },
+ {
+ "expr": "irate(node_netstat_TcpExt_SyncookiesRecv{instance=~\"$node:$port\",job=~\"$job\"}[5m])",
+ "format": "time_series",
+ "hide": false,
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "SyncookiesRecv - SYN cookies received",
+ "refId": "B",
+ "step": 4
+ },
+ {
+ "expr": "irate(node_netstat_TcpExt_SyncookiesSent{instance=~\"$node:$port\",job=~\"$job\"}[5m])",
+ "format": "time_series",
+ "hide": false,
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "SyncookiesSent - SYN cookies sent",
+ "refId": "C",
+ "step": 4
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "TCP SynCookie",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "short",
+ "label": "counter out (-) / in (+)",
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "fill": 2,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 0,
+ "y": 82
+ },
+ "height": "",
+ "hiddenSeries": false,
+ "id": 82,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "hideZero": false,
+ "max": true,
+ "min": true,
+ "rightSide": false,
+ "show": true,
+ "sortDesc": true,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "maxPerRow": 12,
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "irate(node_netstat_Tcp_ActiveOpens{instance=~\"$node:$port\",job=~\"$job\"}[5m])",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "ActiveOpens - TCP connections that have made a direct transition to the SYN-SENT state from the CLOSED state",
+ "refId": "A",
+ "step": 4
+ },
+ {
+ "expr": "irate(node_netstat_Tcp_PassiveOpens{instance=~\"$node:$port\",job=~\"$job\"}[5m])",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "PassiveOpens - TCP connections that have made a direct transition to the SYN-RCVD state from the LISTEN state",
+ "refId": "B",
+ "step": 4
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "TCP Direct Transition",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "short",
+ "label": "connections",
+ "logBase": 1,
+ "max": null,
+ "min": "0",
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ }
+ ],
+ "repeat": null,
+ "title": "Network Netstat",
+ "type": "row"
+ },
+ {
+ "collapsed": true,
+ "datasource": "${DS_PROMETHEUS}",
+ "gridPos": {
+ "h": 1,
+ "w": 24,
+ "x": 0,
+ "y": 71
+ },
+ "id": 279,
+ "panels": [
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "description": "",
+ "fieldConfig": {
+ "defaults": {
+ "custom": {}
+ },
+ "overrides": []
+ },
+ "fill": 2,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 0,
+ "y": 154
+ },
+ "hiddenSeries": false,
+ "id": 40,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "max": true,
+ "min": true,
+ "show": true,
+ "sort": "current",
+ "sortDesc": true,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "stack": true,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "node_scrape_collector_duration_seconds{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "hide": false,
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "{{collector}} - Scrape duration",
+ "refId": "A",
+ "step": 4
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "Node Exporter Scrape Time",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "s",
+ "label": "seconds",
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "description": "",
+ "fieldConfig": {
+ "defaults": {
+ "custom": {}
+ },
+ "overrides": []
+ },
+ "fill": 2,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 12,
+ "y": 154
+ },
+ "hiddenSeries": false,
+ "id": 157,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "max": true,
+ "min": true,
+ "show": true,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [
+ {
+ "alias": "/.*error.*/",
+ "color": "#F2495C",
+ "transform": "negative-Y"
+ }
+ ],
+ "spaceLength": 10,
+ "stack": true,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "node_scrape_collector_success{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "hide": false,
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "{{collector}} - Scrape success",
+ "refId": "A",
+ "step": 4
+ },
+ {
+ "expr": "node_textfile_scrape_error{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "hide": false,
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "{{collector}} - Scrape textfile error (1 = true)",
+ "refId": "B",
+ "step": 4
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "Node Exporter Scrape",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "short",
+ "label": "counter",
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ }
+ ],
+ "repeat": null,
+ "title": "Node Exporter",
+ "type": "row"
+ }
+ ],
+ "refresh": false,
+ "schemaVersion": 25,
+ "style": "dark",
+ "tags": [
+ "linux"
+ ],
+ "templating": {
+ "list": [
+ {
+ "current": {
+ "selected": false,
+ "text": "Prometheus",
+ "value": "Prometheus"
+ },
+ "hide": 0,
+ "includeAll": false,
+ "label": "datasource",
+ "multi": false,
+ "name": "DS_PROMETHEUS",
+ "options": [],
+ "query": "prometheus",
+ "queryValue": "",
+ "refresh": 1,
+ "regex": "",
+ "skipUrlSync": false,
+ "type": "datasource"
+ },
+ {
+ "allValue": null,
+ "current": {
+ "selected": false,
+ "text": "kafka",
+ "value": "kafka"
+ },
+ "datasource": "${DS_PROMETHEUS}",
+ "definition": "",
+ "hide": 0,
+ "includeAll": false,
+ "label": "Job",
+ "multi": false,
+ "name": "job",
+ "options": [],
+ "query": "label_values(node_uname_info, job)",
+ "refresh": 1,
+ "regex": "",
+ "skipUrlSync": false,
+ "sort": 1,
+ "tagValuesQuery": "",
+ "tags": [],
+ "tagsQuery": "",
+ "type": "query",
+ "useTags": false
+ },
+ {
+ "allValue": null,
+ "current": {
+ "selected": false,
+ "text": "ip-10-0-0-15.us-west-2.compute.internal",
+ "value": "ip-10-0-0-15.us-west-2.compute.internal"
+ },
+ "datasource": "${DS_PROMETHEUS}",
+ "definition": "",
+ "hide": 0,
+ "includeAll": false,
+ "label": "Host:",
+ "multi": false,
+ "name": "name",
+ "options": [],
+ "query": "label_values(node_uname_info{job=~\"$job\"}, nodename)",
+ "refresh": 1,
+ "regex": "",
+ "skipUrlSync": false,
+ "sort": 1,
+ "tagValuesQuery": "",
+ "tags": [],
+ "tagsQuery": "",
+ "type": "query",
+ "useTags": false
+ },
+ {
+ "allValue": null,
+ "current": {
+ "selected": false,
+ "text": "10.0.0.15",
+ "value": "10.0.0.15"
+ },
+ "datasource": "${DS_PROMETHEUS}",
+ "definition": "",
+ "hide": 2,
+ "includeAll": false,
+ "label": "Host:",
+ "multi": false,
+ "name": "node",
+ "options": [],
+ "query": "label_values(node_uname_info{nodename=\"$name\"}, instance)",
+ "refresh": 1,
+ "regex": "/([^:]+):.*/",
+ "skipUrlSync": false,
+ "sort": 1,
+ "tagValuesQuery": "",
+ "tags": [],
+ "tagsQuery": "",
+ "type": "query",
+ "useTags": false
+ },
+ {
+ "allValue": null,
+ "current": {
+ "selected": false,
+ "text": "9100",
+ "value": "9100"
+ },
+ "datasource": "${DS_PROMETHEUS}",
+ "definition": "",
+ "hide": 0,
+ "includeAll": false,
+ "label": "Port",
+ "multi": false,
+ "name": "port",
+ "options": [],
+ "query": "label_values(node_uname_info{instance=~\"$node:(.*)\"}, instance)",
+ "refresh": 1,
+ "regex": "/[^:]+:(.*)/",
+ "skipUrlSync": false,
+ "sort": 3,
+ "tagValuesQuery": "",
+ "tags": [],
+ "tagsQuery": "",
+ "type": "query",
+ "useTags": false
+ }
+ ]
+ },
+ "time": {
+ "from": "now-3h",
+ "to": "now"
+ },
+ "timepicker": {
+ "refresh_intervals": [
+ "10s",
+ "30s",
+ "1m",
+ "5m",
+ "15m",
+ "30m",
+ "1h",
+ "2h",
+ "1d"
+ ],
+ "time_options": [
+ "5m",
+ "15m",
+ "1h",
+ "6h",
+ "12h",
+ "24h",
+ "2d",
+ "7d",
+ "30d"
+ ]
+ },
+ "timezone": "browser",
+ "title": "Node Exporter Full",
+ "uid": "rYdddlPWk",
+ "version": 4
+}
\ No newline at end of file
diff --git a/driver-kafka/deploy/monitoring/dashboards/zookeeper.json b/driver-kafka/deploy/monitoring/dashboards/zookeeper.json
new file mode 100644
index 00000000..d8a4ab43
--- /dev/null
+++ b/driver-kafka/deploy/monitoring/dashboards/zookeeper.json
@@ -0,0 +1,953 @@
+{
+ "annotations": {
+ "list": []
+ },
+ "editable": true,
+ "gnetId": null,
+ "graphTooltip": 0,
+ "hideControls": false,
+ "id": 3,
+ "links": [],
+ "rows": [
+ {
+ "collapse": false,
+ "height": 236,
+ "panels": [
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": null,
+ "decimals": 1,
+ "description": "All write operations (create / setData / ...)",
+ "fill": 0,
+ "id": 1,
+ "legend": {
+ "avg": false,
+ "current": false,
+ "max": false,
+ "min": false,
+ "show": false,
+ "total": false,
+ "values": false
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "nullPointMode": "null",
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "span": 6,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "irate(zookeeper_server_requests_latency_ms_count{cluster=~\"$cluster\", kubernetes_pod_name=~\"$server\", type=\"write\"}[30s])",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "{{kubernetes_pod_name}}",
+ "metric": "zookeeper_server_requests_latency_ms_count",
+ "refId": "A",
+ "step": 10
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeShift": null,
+ "title": "Write Requests",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "short",
+ "label": "writes / s",
+ "logBase": 1,
+ "max": null,
+ "min": "0",
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ }
+ ]
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": null,
+ "description": "All read operations (getData / getChildren / ...)",
+ "fill": 0,
+ "id": 2,
+ "legend": {
+ "avg": false,
+ "current": false,
+ "max": false,
+ "min": false,
+ "show": false,
+ "total": false,
+ "values": false
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "nullPointMode": "null",
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "span": 6,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "irate(zookeeper_server_requests_latency_ms_count{cluster=~\"$cluster\", kubernetes_pod_name=~\"$server\", type=\"read\"}[30s])",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "{{kubernetes_pod_name}}",
+ "metric": "zookeeper_server_requests_latency_ms_count",
+ "refId": "A",
+ "step": 10
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeShift": null,
+ "title": "Read Requests",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "short",
+ "label": "reads / s",
+ "logBase": 1,
+ "max": null,
+ "min": "0",
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ }
+ ]
+ }
+ ],
+ "repeat": null,
+ "repeatIteration": null,
+ "repeatRowId": null,
+ "showTitle": false,
+ "title": "Dashboard Row",
+ "titleSize": "h6"
+ },
+ {
+ "collapse": false,
+ "height": 250,
+ "panels": [
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": null,
+ "description": "All write operations (create / setData / ...)",
+ "fill": 0,
+ "id": 4,
+ "legend": {
+ "avg": false,
+ "current": false,
+ "max": false,
+ "min": false,
+ "show": false,
+ "total": false,
+ "values": false
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "nullPointMode": "null as zero",
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "span": 6,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "zookeeper_server_requests_latency_ms{cluster=~\"$cluster\", kubernetes_pod_name=~\"$server\", type=\"write\", quantile=\"0.99\"}",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "{{kubernetes_pod_name}}",
+ "metric": "zookeeper_server_requests_latency_ms_count",
+ "refId": "A",
+ "step": 10
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeShift": null,
+ "title": "Write Latency - 99pct",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "ms",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": "0",
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ }
+ ]
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": null,
+ "decimals": 1,
+ "description": "All write operations (create / setData / ...)",
+ "fill": 0,
+ "id": 5,
+ "legend": {
+ "avg": false,
+ "current": false,
+ "max": false,
+ "min": false,
+ "show": false,
+ "total": false,
+ "values": false
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "nullPointMode": "null",
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "span": 6,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "zookeeper_server_requests_latency_ms{cluster=~\"$cluster\", kubernetes_pod_name=~\"$server\", type=\"read\", quantile=\"0.99\"}",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "{{kubernetes_pod_name}}",
+ "metric": "zookeeper_server_requests_latency_ms_count",
+ "refId": "A",
+ "step": 10
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeShift": null,
+ "title": "Read Latency - 99pct",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "ms",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": "0",
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ }
+ ]
+ }
+ ],
+ "repeat": null,
+ "repeatIteration": null,
+ "repeatRowId": null,
+ "showTitle": false,
+ "title": "Dashboard Row",
+ "titleSize": "h6"
+ },
+ {
+ "collapse": false,
+ "height": 250,
+ "panels": [
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": null,
+ "decimals": 1,
+ "description": "",
+ "fill": 1,
+ "id": 6,
+ "legend": {
+ "avg": false,
+ "current": false,
+ "max": false,
+ "min": false,
+ "show": true,
+ "total": false,
+ "values": false
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "nullPointMode": "null",
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "span": 12,
+ "stack": true,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "sum(irate(zookeeper_server_requests{cluster=~\"$cluster\", kubernetes_pod_name=~\"$server\"}[30s])) by (type)",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "{{type}}",
+ "metric": "zookeeper_server_requests_latency_ms_count",
+ "refId": "A",
+ "step": 4
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeShift": null,
+ "title": "Request Types",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "short",
+ "label": "",
+ "logBase": 1,
+ "max": null,
+ "min": "0",
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ }
+ ]
+ }
+ ],
+ "repeat": null,
+ "repeatIteration": null,
+ "repeatRowId": null,
+ "showTitle": false,
+ "title": "Dashboard Row",
+ "titleSize": "h6"
+ },
+ {
+ "collapse": false,
+ "height": 250,
+ "panels": [
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": null,
+ "description": "",
+ "fill": 0,
+ "id": 7,
+ "legend": {
+ "avg": false,
+ "current": false,
+ "max": false,
+ "min": false,
+ "show": false,
+ "total": false,
+ "values": false
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "nullPointMode": "null",
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "span": 6,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "zookeeper_server_znode_count{cluster=~\"$cluster\", kubernetes_pod_name=~\"$server\"}",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "{{cluster}} - {{kubernetes_pod_name}}",
+ "metric": "zookeeper_server_requests_latency_ms_count",
+ "refId": "A",
+ "step": 10
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeShift": null,
+ "title": "Z-Nodes count",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "short",
+ "label": "",
+ "logBase": 1,
+ "max": null,
+ "min": "0",
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ }
+ ]
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": null,
+ "description": "Size of the data stored in a ZK server",
+ "fill": 0,
+ "id": 8,
+ "legend": {
+ "avg": false,
+ "current": false,
+ "max": false,
+ "min": false,
+ "show": false,
+ "total": false,
+ "values": false
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "nullPointMode": "null",
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "span": 6,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "zookeeper_server_data_size_bytes{cluster=~\"$cluster\", kubernetes_pod_name=~\"$server\"}",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "{{cluster}} - {{kubernetes_pod_name}}",
+ "metric": "zookeeper_server_requests_latency_ms_count",
+ "refId": "A",
+ "step": 10
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeShift": null,
+ "title": "Data set size",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "decbytes",
+ "label": "",
+ "logBase": 1,
+ "max": null,
+ "min": "0",
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ }
+ ]
+ }
+ ],
+ "repeat": null,
+ "repeatIteration": null,
+ "repeatRowId": null,
+ "showTitle": false,
+ "title": "Dashboard Row",
+ "titleSize": "h6"
+ },
+ {
+ "collapse": false,
+ "height": 250,
+ "panels": [
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": null,
+ "description": "Number of opened connections",
+ "fill": 1,
+ "id": 9,
+ "legend": {
+ "avg": false,
+ "current": false,
+ "max": false,
+ "min": false,
+ "show": false,
+ "total": false,
+ "values": false
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "nullPointMode": "null",
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "span": 4,
+ "stack": true,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "zookeeper_server_connections{cluster=~\"$cluster\", kubernetes_pod_name=~\"$server\"}",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "{{cluster}} - {{kubernetes_pod_name}}",
+ "metric": "zookeeper_server_requests_latency_ms_count",
+ "refId": "A",
+ "step": 10
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeShift": null,
+ "title": "Connections",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "short",
+ "label": "",
+ "logBase": 1,
+ "max": null,
+ "min": "0",
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ }
+ ]
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": null,
+ "description": "",
+ "fill": 1,
+ "id": 10,
+ "legend": {
+ "avg": false,
+ "current": false,
+ "max": false,
+ "min": false,
+ "show": false,
+ "total": false,
+ "values": false
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "nullPointMode": "null",
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "span": 4,
+ "stack": true,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "zookeeper_server_watches_count{cluster=~\"$cluster\", kubernetes_pod_name=~\"$server\"}",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "{{cluster}} - {{kubernetes_pod_name}}",
+ "metric": "zookeeper_server_requests_latency_ms_count",
+ "refId": "A",
+ "step": 10
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeShift": null,
+ "title": "Number of watches",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "short",
+ "label": "",
+ "logBase": 1,
+ "max": null,
+ "min": "0",
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ }
+ ]
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": null,
+ "description": "",
+ "fill": 0,
+ "id": 11,
+ "legend": {
+ "avg": false,
+ "current": false,
+ "max": false,
+ "min": false,
+ "show": false,
+ "total": false,
+ "values": false
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "nullPointMode": "null",
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "span": 4,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "zookeeper_server_ephemerals_count{cluster=~\"$cluster\", kubernetes_pod_name=~\"$server\"}",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "{{cluster}} - {{kubernetes_pod_name}}",
+ "metric": "zookeeper_server_requests_latency_ms_count",
+ "refId": "A",
+ "step": 10
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeShift": null,
+ "title": "Number of ephemerals",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "short",
+ "label": "",
+ "logBase": 1,
+ "max": null,
+ "min": "0",
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ }
+ ]
+ }
+ ],
+ "repeat": null,
+ "repeatIteration": null,
+ "repeatRowId": null,
+ "showTitle": false,
+ "title": "Dashboard Row",
+ "titleSize": "h6"
+ }
+ ],
+ "schemaVersion": 14,
+ "style": "dark",
+ "tags": [
+ "zookeeper"
+ ],
+ "templating": {
+ "list": [
+ {
+ "allValue": null,
+ "current": {
+ "text": "us-central",
+ "value": "us-central"
+ },
+ "datasource": "Prometheus",
+ "hide": 0,
+ "includeAll": false,
+ "label": "Cluster",
+ "multi": false,
+ "name": "cluster",
+ "options": [],
+ "query": "{cluster=~\".+\"}",
+ "refresh": 1,
+ "regex": "/.*[^_]cluster=\\\"([^\\\"]+)\\\".*/",
+ "sort": 1,
+ "tagValuesQuery": "",
+ "tags": [],
+ "tagsQuery": "",
+ "type": "query",
+ "useTags": false
+ },
+ {
+ "allValue": null,
+ "current": {
+ "text": "All",
+ "value": "$__all"
+ },
+ "datasource": "Prometheus",
+ "hide": 0,
+ "includeAll": true,
+ "label": "ZK Server",
+ "multi": false,
+ "name": "server",
+ "options": [],
+ "query": "zookeeper_server_requests_latency_ms_count{cluster=~\"$cluster\", kubernetes_pod_name=~\".+\"}",
+ "refresh": 2,
+ "regex": "/.*[^_]kubernetes_pod_name=\\\"([^\\\"]+)\\\".*/",
+ "sort": 1,
+ "tagValuesQuery": "",
+ "tags": [],
+ "tagsQuery": "",
+ "type": "query",
+ "useTags": false
+ }
+ ]
+ },
+ "time": {
+ "from": "now-1h",
+ "to": "now"
+ },
+ "timepicker": {
+ "refresh_intervals": [
+ "5s",
+ "10s",
+ "30s",
+ "1m",
+ "5m",
+ "15m",
+ "30m",
+ "1h",
+ "2h",
+ "1d"
+ ],
+ "time_options": [
+ "5m",
+ "15m",
+ "1h",
+ "6h",
+ "12h",
+ "24h",
+ "2d",
+ "7d",
+ "30d"
+ ]
+ },
+ "timezone": "browser",
+ "title": "ZooKeeper",
+ "version": 11
+}
\ No newline at end of file
diff --git a/driver-kafka/deploy/monitoring/grafana.ini b/driver-kafka/deploy/monitoring/grafana.ini
new file mode 100644
index 00000000..e28ca88b
--- /dev/null
+++ b/driver-kafka/deploy/monitoring/grafana.ini
@@ -0,0 +1,39 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements. See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership. The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied. See the License for the
+# specific language governing permissions and limitations
+# under the License.
+#
+
+[users]
+# disable user signup / registration
+allow_sign_up = false
+
+# Default UI theme ("dark" or "light")
+default_theme = dark
+
+[auth.anonymous]
+# enable anonymous access
+enabled = true
+
+# specify organization name that should be used for unauthenticated users
+org_name = Main Org.
+
+# specify role for unauthenticated users
+org_role = Viewer
+
+[dashboards]
+enabled = true
+path = /var/lib/grafana/dashboards
\ No newline at end of file
diff --git a/driver-kafka/deploy/monitoring/jmx_exporter/client.yml b/driver-kafka/deploy/monitoring/jmx_exporter/client.yml
new file mode 100644
index 00000000..792f8af8
--- /dev/null
+++ b/driver-kafka/deploy/monitoring/jmx_exporter/client.yml
@@ -0,0 +1,22 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements. See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership. The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied. See the License for the
+# specific language governing permissions and limitations
+# under the License.
+#
+
+lowercaseOutputLabelNames: false
+lowercaseOutputName: true
+## Export everything
diff --git a/driver-kafka/deploy/monitoring/jmx_exporter/jmx_prometheus_javaagent-0.13.0.jar b/driver-kafka/deploy/monitoring/jmx_exporter/jmx_prometheus_javaagent-0.13.0.jar
new file mode 100644
index 00000000..c55a92e5
Binary files /dev/null and b/driver-kafka/deploy/monitoring/jmx_exporter/jmx_prometheus_javaagent-0.13.0.jar differ
diff --git a/driver-kafka/deploy/monitoring/jmx_exporter/metrics.yml b/driver-kafka/deploy/monitoring/jmx_exporter/metrics.yml
new file mode 100644
index 00000000..cbde974b
--- /dev/null
+++ b/driver-kafka/deploy/monitoring/jmx_exporter/metrics.yml
@@ -0,0 +1,122 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements. See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership. The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied. See the License for the
+# specific language governing permissions and limitations
+# under the License.
+#
+
+lowercaseOutputName: true
+
+rules:
+ # Special cases and very specific rules
+ - pattern: kafka.server<>Value
+ name: kafka_server_$1_$2
+ type: GAUGE
+ labels:
+ clientId: "$3"
+ topic: "$4"
+ partition: "$5"
+ - pattern: kafka.server<>Value
+ name: kafka_server_$1_$2
+ type: GAUGE
+ labels:
+ clientId: "$3"
+ broker: "$4:$5"
+ - pattern: kafka.coordinator.(\w+)<>Value
+ name: kafka_coordinator_$1_$2_$3
+ type: GAUGE
+
+ # Generic per-second counters with 0-2 key/value pairs
+ - pattern: kafka.(\w+)<>Count
+ name: kafka_$1_$2_$3_total
+ type: COUNTER
+ labels:
+ "$4": "$5"
+ "$6": "$7"
+ - pattern: kafka.(\w+)<>Count
+ name: kafka_$1_$2_$3_total
+ type: COUNTER
+ labels:
+ "$4": "$5"
+ - pattern: kafka.(\w+)<>Count
+ name: kafka_$1_$2_$3_total
+ type: COUNTER
+
+ - pattern: kafka.server<>([a-z-]+)
+ name: kafka_server_quota_$3
+ type: GAUGE
+ labels:
+ resource: "$1"
+ clientId: "$2"
+
+ - pattern: kafka.server<>([a-z-]+)
+ name: kafka_server_quota_$4
+ type: GAUGE
+ labels:
+ resource: "$1"
+ user: "$2"
+ clientId: "$3"
+
+ # Generic gauges with 0-2 key/value pairs
+ - pattern: kafka.(\w+)<>Value
+ name: kafka_$1_$2_$3
+ type: GAUGE
+ labels:
+ "$4": "$5"
+ "$6": "$7"
+ - pattern: kafka.(\w+)<>Value
+ name: kafka_$1_$2_$3
+ type: GAUGE
+ labels:
+ "$4": "$5"
+ - pattern: kafka.(\w+)<>Value
+ name: kafka_$1_$2_$3
+ type: GAUGE
+
+ # Emulate Prometheus 'Summary' metrics for the exported 'Histogram's.
+ #
+ # Note that these are missing the '_sum' metric!
+ - pattern: kafka.(\w+)<>Count
+ name: kafka_$1_$2_$3_count
+ type: COUNTER
+ labels:
+ "$4": "$5"
+ "$6": "$7"
+ - pattern: kafka.(\w+)<>(\d+)thPercentile
+ name: kafka_$1_$2_$3
+ type: GAUGE
+ labels:
+ "$4": "$5"
+ "$6": "$7"
+ quantile: "0.$8"
+ - pattern: kafka.(\w+)<>Count
+ name: kafka_$1_$2_$3_count
+ type: COUNTER
+ labels:
+ "$4": "$5"
+ - pattern: kafka.(\w+)<>(\d+)thPercentile
+ name: kafka_$1_$2_$3
+ type: GAUGE
+ labels:
+ "$4": "$5"
+ quantile: "0.$6"
+ - pattern: kafka.(\w+)<>Count
+ name: kafka_$1_$2_$3_count
+ type: COUNTER
+ - pattern: kafka.(\w+)<>(\d+)thPercentile
+ name: kafka_$1_$2_$3
+ type: GAUGE
+ labels:
+ quantile: "0.$4"
diff --git a/driver-kafka/deploy/monitoring/jmx_exporter/zookeeper.yml b/driver-kafka/deploy/monitoring/jmx_exporter/zookeeper.yml
new file mode 100644
index 00000000..22b82b72
--- /dev/null
+++ b/driver-kafka/deploy/monitoring/jmx_exporter/zookeeper.yml
@@ -0,0 +1,123 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements. See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership. The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied. See the License for the
+# specific language governing permissions and limitations
+# under the License.
+#
+
+lowercaseOutputLabelNames: false
+lowercaseOutputName: true
+## Export everything
+# rules:
+# # Special cases and very specific rules
+# - pattern: kafka.server<>Value
+# name: kafka_server_$1_$2
+# type: GAUGE
+# labels:
+# clientId: "$3"
+# topic: "$4"
+# partition: "$5"
+# - pattern: kafka.server<>Value
+# name: kafka_server_$1_$2
+# type: GAUGE
+# labels:
+# clientId: "$3"
+# broker: "$4:$5"
+# - pattern: kafka.coordinator.(\w+)<>Value
+# name: kafka_coordinator_$1_$2_$3
+# type: GAUGE
+
+# # Generic per-second counters with 0-2 key/value pairs
+# - pattern: kafka.(\w+)<>Count
+# name: kafka_$1_$2_$3_total
+# type: COUNTER
+# labels:
+# "$4": "$5"
+# "$6": "$7"
+# - pattern: kafka.(\w+)<>Count
+# name: kafka_$1_$2_$3_total
+# type: COUNTER
+# labels:
+# "$4": "$5"
+# - pattern: kafka.(\w+)<>Count
+# name: kafka_$1_$2_$3_total
+# type: COUNTER
+
+# - pattern: kafka.server<>([a-z-]+)
+# name: kafka_server_quota_$3
+# type: GAUGE
+# labels:
+# resource: "$1"
+# clientId: "$2"
+
+# - pattern: kafka.server<>([a-z-]+)
+# name: kafka_server_quota_$4
+# type: GAUGE
+# labels:
+# resource: "$1"
+# user: "$2"
+# clientId: "$3"
+
+# # Generic gauges with 0-2 key/value pairs
+# - pattern: kafka.(\w+)<>Value
+# name: kafka_$1_$2_$3
+# type: GAUGE
+# labels:
+# "$4": "$5"
+# "$6": "$7"
+# - pattern: kafka.(\w+)<>Value
+# name: kafka_$1_$2_$3
+# type: GAUGE
+# labels:
+# "$4": "$5"
+# - pattern: kafka.(\w+)<>Value
+# name: kafka_$1_$2_$3
+# type: GAUGE
+
+# # Emulate Prometheus 'Summary' metrics for the exported 'Histogram's.
+# #
+# # Note that these are missing the '_sum' metric!
+# - pattern: kafka.(\w+)<>Count
+# name: kafka_$1_$2_$3_count
+# type: COUNTER
+# labels:
+# "$4": "$5"
+# "$6": "$7"
+# - pattern: kafka.(\w+)<>(\d+)thPercentile
+# name: kafka_$1_$2_$3
+# type: GAUGE
+# labels:
+# "$4": "$5"
+# "$6": "$7"
+# quantile: "0.$8"
+# - pattern: kafka.(\w+)<>Count
+# name: kafka_$1_$2_$3_count
+# type: COUNTER
+# labels:
+# "$4": "$5"
+# - pattern: kafka.(\w+)<>(\d+)thPercentile
+# name: kafka_$1_$2_$3
+# type: GAUGE
+# labels:
+# "$4": "$5"
+# quantile: "0.$6"
+# - pattern: kafka.(\w+)<>Count
+# name: kafka_$1_$2_$3_count
+# type: COUNTER
+# - pattern: kafka.(\w+)<>(\d+)thPercentile
+# name: kafka_$1_$2_$3
+# type: GAUGE
+# labels:
+# quantile: "0.$4"
diff --git a/driver-kafka/deploy/monitoring/start.sh b/driver-kafka/deploy/monitoring/start.sh
new file mode 100755
index 00000000..8cc5a499
--- /dev/null
+++ b/driver-kafka/deploy/monitoring/start.sh
@@ -0,0 +1,39 @@
+#!/bin/bash
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements. See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership. The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied. See the License for the
+# specific language governing permissions and limitations
+# under the License.
+#
+
+
+echo 'Starting Grafana...'
+
+/run.sh "$@" &
+
+AddDataSource() {
+ curl 'http://admin:admin@localhost:3000/api/datasources' \
+ -X POST \
+ -H 'Content-Type: application/json;charset=UTF-8' \
+ --data-binary \
+ "{\"name\":\"Prometheus\",\"type\":\"prometheus\",\"url\":\"$PROMETHEUS_URL\",\"access\":\"proxy\",\"isDefault\":true}"
+}
+
+until AddDataSource; do
+ echo 'Configuring Grafana...'
+ sleep 1
+done
+ echo 'Done!'
+wait
\ No newline at end of file
diff --git a/driver-kafka/deploy/provision-kafka-aws.tf b/driver-kafka/deploy/provision-kafka-aws.tf
index aebf657a..1f56b17a 100644
--- a/driver-kafka/deploy/provision-kafka-aws.tf
+++ b/driver-kafka/deploy/provision-kafka-aws.tf
@@ -1,6 +1,7 @@
provider "aws" {
region = "${var.region}"
version = "~> 2.7"
+ profile = var.profile
}
provider "random" {
@@ -30,6 +31,8 @@ variable "region" {}
variable "ami" {}
+variable "profile" {}
+
variable "instance_types" {
type = "map"
}
@@ -42,7 +45,7 @@ variable "num_instances" {
resource "aws_vpc" "benchmark_vpc" {
cidr_block = "10.0.0.0/16"
- tags {
+ tags = {
Name = "Kafka-Benchmark-VPC-${random_id.hash.hex}"
}
}
@@ -64,6 +67,12 @@ resource "aws_subnet" "benchmark_subnet" {
vpc_id = "${aws_vpc.benchmark_vpc.id}"
cidr_block = "10.0.0.0/24"
map_public_ip_on_launch = true
+ availability_zone = "us-west-2b"
+}
+
+# Get public IP of this machine
+data "http" "myip" {
+ url = "http://ipv4.icanhazip.com"
}
resource "aws_security_group" "benchmark_security_group" {
@@ -71,21 +80,43 @@ resource "aws_security_group" "benchmark_security_group" {
vpc_id = "${aws_vpc.benchmark_vpc.id}"
# SSH access from anywhere
+ # ingress {
+ # from_port = 22
+ # to_port = 22
+ # protocol = "tcp"
+ # cidr_blocks = ["0.0.0.0/0"]
+ # }
+
+ # All ports open within the VPC
ingress {
- from_port = 22
- to_port = 22
+ from_port = 0
+ to_port = 65535
protocol = "tcp"
- cidr_blocks = ["0.0.0.0/0"]
+ cidr_blocks = ["10.0.0.0/16"]
}
- # All ports open within the VPC
+ # All ports open to this machine
ingress {
from_port = 0
to_port = 65535
protocol = "tcp"
- cidr_blocks = ["10.0.0.0/16"]
+ cidr_blocks = ["${chomp(data.http.myip.body)}/32"]
}
+ # Prometheus/Dashboard access
+ # ingress {
+ # from_port = 9090
+ # to_port = 9090
+ # protocol = "tcp"
+ # cidr_blocks = ["0.0.0.0/0"]
+ # }
+ ingress {
+ from_port = 3000
+ to_port = 3000
+ protocol = "tcp"
+ cidr_blocks = ["0.0.0.0/0"]
+ }
+
# outbound internet access
egress {
from_port = 0
@@ -94,7 +125,7 @@ resource "aws_security_group" "benchmark_security_group" {
cidr_blocks = ["0.0.0.0/0"]
}
- tags {
+ tags = {
Name = "Benchmark-Security-Group-${random_id.hash.hex}"
}
}
@@ -111,8 +142,15 @@ resource "aws_instance" "zookeeper" {
subnet_id = "${aws_subnet.benchmark_subnet.id}"
vpc_security_group_ids = ["${aws_security_group.benchmark_security_group.id}"]
count = "${var.num_instances["zookeeper"]}"
+ monitoring = true
+
+ root_block_device {
+ volume_size = 100
+ volume_type = "io1"
+ iops = 1000
+ }
- tags {
+ tags = {
Name = "zk-${count.index}"
}
}
@@ -124,8 +162,9 @@ resource "aws_instance" "kafka" {
subnet_id = "${aws_subnet.benchmark_subnet.id}"
vpc_security_group_ids = ["${aws_security_group.benchmark_security_group.id}"]
count = "${var.num_instances["kafka"]}"
+ monitoring = true
- tags {
+ tags = {
Name = "kafka-${count.index}"
}
}
@@ -137,12 +176,58 @@ resource "aws_instance" "client" {
subnet_id = "${aws_subnet.benchmark_subnet.id}"
vpc_security_group_ids = ["${aws_security_group.benchmark_security_group.id}"]
count = "${var.num_instances["client"]}"
+ monitoring = true
- tags {
+ tags = {
Name = "kafka-client-${count.index}"
}
}
-output "client_ssh_host" {
- value = "${aws_instance.client.0.public_ip}"
+resource "aws_instance" "prometheus" {
+ ami = "${var.ami}"
+ instance_type = "${var.instance_types["prometheus"]}"
+ key_name = "${aws_key_pair.auth.id}"
+ subnet_id = "${aws_subnet.benchmark_subnet.id}"
+ vpc_security_group_ids = ["${aws_security_group.benchmark_security_group.id}"]
+ count = "${var.num_instances["prometheus"]}"
+
+ root_block_device {
+ volume_size = 20
+ }
+
+ tags = {
+ Name = "prometheus-${count.index}"
+ }
+}
+
+output "clients" {
+ value = {
+ for instance in aws_instance.client :
+ instance.public_ip => instance.private_ip
+ }
}
+
+output "brokers" {
+ value = {
+ for instance in aws_instance.kafka :
+ instance.public_ip => instance.private_ip
+ }
+}
+
+output "zookeeper" {
+ value = {
+ for instance in aws_instance.zookeeper :
+ instance.public_ip => instance.private_ip
+ }
+}
+
+output "prometheus_host" {
+ value = {
+ for instance in aws_instance.prometheus :
+ instance.public_ip => instance.private_ip
+ }
+}
+
+#output "client_ssh_host" {
+# value = "${aws_instance.client.0.public_ip}"
+#}
diff --git a/driver-kafka/deploy/templates/chrony.conf b/driver-kafka/deploy/templates/chrony.conf
index 00fb403e..0e68c66a 100644
--- a/driver-kafka/deploy/templates/chrony.conf
+++ b/driver-kafka/deploy/templates/chrony.conf
@@ -1,3 +1,22 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements. See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership. The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied. See the License for the
+# specific language governing permissions and limitations
+# under the License.
+#
+
# Use public servers from the pool.ntp.org project.
# Please consider joining the pool (http://www.pool.ntp.org/join.html).
server 169.254.169.123 prefer iburst
diff --git a/driver-kafka/deploy/templates/kafka-dashboard.service b/driver-kafka/deploy/templates/kafka-dashboard.service
new file mode 100644
index 00000000..8b00e459
--- /dev/null
+++ b/driver-kafka/deploy/templates/kafka-dashboard.service
@@ -0,0 +1,16 @@
+[Unit]
+Description=Kafka Dashboard
+After=docker.service
+Requires=docker.service
+After=prometheus.service
+Requires=prometheus.service
+
+[Service]
+ExecStartPre=/usr/bin/docker pull aloknnikhil/omb-kafka-dashboard:latest
+ExecStart=/usr/bin/docker run --restart=always --name=systemd_kafka_dashboard -p3000:3000 -e PROMETHEUS_URL=http://{{ hostvars[groups['prometheus'][0]].private_ip }}:9090/ aloknnikhil/omb-kafka-dashboard:latest
+ExecStop=/usr/bin/docker stop systemd_kafka_dashboard
+ExecStopPost=/usr/bin/docker rm -f systemd_kafka_dashboard
+ExecReload=/usr/bin/docker restart systemd_kafka_dashboard
+
+[Install]
+WantedBy=multi-user.target
diff --git a/driver-kafka/deploy/templates/kafka.service b/driver-kafka/deploy/templates/kafka.service
index c5afac56..3ed876d3 100644
--- a/driver-kafka/deploy/templates/kafka.service
+++ b/driver-kafka/deploy/templates/kafka.service
@@ -4,6 +4,9 @@ After=network.target
[Service]
ExecStart=/opt/kafka/bin/kafka-server-start.sh config/server.properties
+Environment=JMX_PORT=9999
+Environment="KAFKA_OPTS=-javaagent:/opt/kafka/jmx_prometheus_javaagent-0.13.0.jar=8080:/opt/kafka/metrics.yml"
+Environment="KAFKA_JMX_OPTS=-Dcom.sun.management.jmxremote=true -Dcom.sun.management.jmxremote.authenticate=false -Dcom.sun.management.jmxremote.ssl=false -Djava.rmi.server.hostname=prod-kafka-001 -Djava.net.preferIPv4Stack=true"
Environment='KAFKA_HEAP_OPTS=-Xms6g -Xmx6g -XX:MetaspaceSize=96m'
Environment='KAFKA_JVM_PERFORMANCE_OPTS=-server -XX:+UseG1GC -XX:MaxGCPauseMillis=20 -XX:InitiatingHeapOccupancyPercent=35 -XX:G1HeapRegionSize=16M -XX:MinMetaspaceFreeRatio=50 -XX:MaxMetaspaceFreeRatio=80 -Djava.awt.headless=true"
WorkingDirectory=/opt/kafka
diff --git a/driver-kafka/deploy/templates/prometheus.service b/driver-kafka/deploy/templates/prometheus.service
new file mode 100644
index 00000000..8f1effac
--- /dev/null
+++ b/driver-kafka/deploy/templates/prometheus.service
@@ -0,0 +1,13 @@
+[Unit]
+Description=Prometheus server
+After=network.target
+
+[Service]
+WorkingDirectory=/opt/prometheus
+ExecStart=/opt/prometheus/prometheus \
+ --web.enable-admin-api \
+ --storage.tsdb.path=/opt/prometheus/data \
+ --config.file=/opt/prometheus/prometheus.yml
+
+[Install]
+WantedBy=multi-user.target
\ No newline at end of file
diff --git a/driver-kafka/deploy/templates/prometheus.yml b/driver-kafka/deploy/templates/prometheus.yml
new file mode 100644
index 00000000..13462e8c
--- /dev/null
+++ b/driver-kafka/deploy/templates/prometheus.yml
@@ -0,0 +1,83 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements. See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership. The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied. See the License for the
+# specific language governing permissions and limitations
+# under the License.
+#
+
+# my global config
+global:
+ scrape_interval: 15s # Set the scrape interval to every 15 seconds. Default is every 1 minute.
+ evaluation_interval: 15s # Evaluate rules every 15 seconds. The default is every 1 minute.
+ # scrape_timeout is set to the global default (10s).
+
+# Alertmanager configuration
+alerting:
+ alertmanagers:
+ - static_configs:
+ - targets:
+ # - alertmanager:9093
+
+# Load rules once and periodically evaluate them according to the global 'evaluation_interval'.
+rule_files:
+ # - "first_rules.yml"
+ # - "second_rules.yml"
+
+# A scrape configuration containing exactly one endpoint to scrape:
+# Here it's Prometheus itself.
+scrape_configs:
+ # The job name is added as a label `job=` to any timeseries scraped from this config.
+ - job_name: 'kafka'
+
+ # metrics_path defaults to '/metrics'
+ # scheme defaults to 'http'.
+
+ static_configs:
+ - targets:
+{% for worker in groups['kafka'] %}
+ - {{ hostvars[worker].private_ip }}:8080
+{% endfor %}
+ labels:
+ group: 'broker'
+ - targets:
+{% for worker in groups['client'] %}
+ - {{ hostvars[worker].private_ip }}:8081
+{% endfor %}
+ labels:
+ group: 'client'
+ - targets:
+{% for worker in groups['kafka'] %}
+ - {{ hostvars[worker].private_ip }}:9100
+{% endfor %}
+ labels:
+ group: 'broker-os'
+ - targets:
+{% for worker in groups['zookeeper'] %}
+ - {{ hostvars[worker].private_ip }}:7070
+{% endfor %}
+ labels:
+ group: 'zookeeper-jmx'
+ - targets:
+{% for worker in groups['zookeeper'] %}
+ - {{ hostvars[worker].private_ip }}:8000
+{% endfor %}
+ labels:
+ group: 'zookeeper'
+ - targets:
+{% for worker in groups['client'] %}
+ - {{ hostvars[worker].private_ip }}:9090
+{% endfor %}
+ labels:
+ group: 'client - jmx'
diff --git a/driver-kafka/deploy/templates/server.properties b/driver-kafka/deploy/templates/server.properties
index 1f2e4686..d5ec0f61 100644
--- a/driver-kafka/deploy/templates/server.properties
+++ b/driver-kafka/deploy/templates/server.properties
@@ -25,3 +25,14 @@ advertised.listeners=PLAINTEXT://{{ privateIp }}:9092
log.dirs=/mnt/data-1,/mnt/data-2
zookeeper.connect={{ zookeeperServers }}
+
+num.replica.fetchers=16
+
+message.max.bytes=10485760
+
+replica.fetch.max.bytes=10485760
+
+num.network.threads=16
+
+num.io.threads=16
+
diff --git a/driver-kafka/deploy/templates/zookeeper.service b/driver-kafka/deploy/templates/zookeeper.service
index 3a2706ef..13dbf8e9 100644
--- a/driver-kafka/deploy/templates/zookeeper.service
+++ b/driver-kafka/deploy/templates/zookeeper.service
@@ -3,6 +3,7 @@ Description=ZooKeeper
After=network.target
[Service]
+Environment="EXTRA_ARGS=-javaagent:/opt/kafka/jmx_prometheus_javaagent-0.13.0.jar=7070:/opt/kafka/zookeeper.yml"
ExecStart=/opt/kafka/bin/zookeeper-server-start.sh config/zookeeper.properties
WorkingDirectory=/opt/kafka
RestartSec=1s
diff --git a/driver-kafka/deploy/terraform.tfvars b/driver-kafka/deploy/terraform.tfvars
index da2c6451..d6a9d9c4 100644
--- a/driver-kafka/deploy/terraform.tfvars
+++ b/driver-kafka/deploy/terraform.tfvars
@@ -1,15 +1,18 @@
public_key_path = "~/.ssh/kafka_aws.pub"
region = "us-west-2"
-ami = "ami-9fa343e7" // RHEL-7.4
+ami = "ami-01e78c5619c5e68b4"
+profile = "default"
instance_types = {
- "kafka" = "i3.4xlarge"
- "zookeeper" = "t2.small"
- "client" = "c5.2xlarge"
+ "kafka" = "i3en.6xlarge"
+ "zookeeper" = "c5.xlarge"
+ "client" = "m5n.8xlarge"
+ "prometheus" = "c5.2xlarge"
}
num_instances = {
- "client" = 4
- "kafka" = 3
- "zookeeper" = 3
+ "client" = 2
+ "kafka" = 3
+ "zookeeper" = 3
+ "prometheus" = 1
}
diff --git a/driver-kafka/kafka.yaml b/driver-kafka/kafka-0.yaml
similarity index 98%
rename from driver-kafka/kafka.yaml
rename to driver-kafka/kafka-0.yaml
index 3ac29203..7b9082f2 100644
--- a/driver-kafka/kafka.yaml
+++ b/driver-kafka/kafka-0.yaml
@@ -17,12 +17,12 @@
# under the License.
#
-
name: Kafka
driverClass: io.openmessaging.benchmark.driver.kafka.KafkaBenchmarkDriver
# Kafka client-specific configuration
replicationFactor: 3
+reset: true
topicConfig: |
min.insync.replicas=2
@@ -31,7 +31,7 @@ commonConfig: |
bootstrap.servers=localhost:9092
producerConfig: |
- acks=all
+ acks=0
linger.ms=1
batch.size=131072
diff --git a/driver-kafka/kafka-1.yaml b/driver-kafka/kafka-1.yaml
new file mode 100644
index 00000000..5abf8242
--- /dev/null
+++ b/driver-kafka/kafka-1.yaml
@@ -0,0 +1,40 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements. See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership. The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied. See the License for the
+# specific language governing permissions and limitations
+# under the License.
+#
+
+name: Kafka
+driverClass: io.openmessaging.benchmark.driver.kafka.KafkaBenchmarkDriver
+
+# Kafka client-specific configuration
+replicationFactor: 3
+reset: true
+
+topicConfig: |
+ min.insync.replicas=2
+
+commonConfig: |
+ bootstrap.servers=localhost:9092
+
+producerConfig: |
+ acks=1
+ linger.ms=1
+ batch.size=131072
+
+consumerConfig: |
+ auto.offset.reset=earliest
+ enable.auto.commit=false
diff --git a/driver-kafka/kafka-all.yaml b/driver-kafka/kafka-all.yaml
new file mode 100644
index 00000000..c78f99c0
--- /dev/null
+++ b/driver-kafka/kafka-all.yaml
@@ -0,0 +1,41 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements. See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership. The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied. See the License for the
+# specific language governing permissions and limitations
+# under the License.
+#
+
+name: Kafka
+driverClass: io.openmessaging.benchmark.driver.kafka.KafkaBenchmarkDriver
+
+# Kafka client-specific configuration
+replicationFactor: 3
+reset: true
+
+topicConfig: |
+ min.insync.replicas=2
+
+commonConfig: |
+ bootstrap.servers=localhost:9092
+
+producerConfig: |
+ acks=all
+ linger.ms=100
+ batch.size=1048576
+
+consumerConfig: |
+ auto.offset.reset=earliest
+ enable.auto.commit=false
+ max.partition.fetch.bytes=10485760
diff --git a/driver-kafka/kafka-exactly-once.yaml b/driver-kafka/kafka-exactly-once.yaml
index d1fd54ac..a7407661 100644
--- a/driver-kafka/kafka-exactly-once.yaml
+++ b/driver-kafka/kafka-exactly-once.yaml
@@ -17,12 +17,12 @@
# under the License.
#
-
name: Kafka-exactly-once
driverClass: io.openmessaging.benchmark.driver.kafka.KafkaBenchmarkDriver
# Kafka client-specific configuration
replicationFactor: 3
+reset: true
topicConfig: |
min.insync.replicas=2
diff --git a/driver-kafka/kafka-group-all.yaml b/driver-kafka/kafka-group-all.yaml
new file mode 100644
index 00000000..b8944d36
--- /dev/null
+++ b/driver-kafka/kafka-group-all.yaml
@@ -0,0 +1,42 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements. See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership. The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied. See the License for the
+# specific language governing permissions and limitations
+# under the License.
+#
+
+name: Kafka
+driverClass: io.openmessaging.benchmark.driver.kafka.KafkaBenchmarkDriver
+
+# Kafka client-specific configuration
+replicationFactor: 3
+reset: true
+
+topicConfig: |
+ min.insync.replicas=2
+
+commonConfig: |
+ bootstrap.servers=localhost:9092
+ request.timeout.ms=120000
+producerConfig: |
+ acks=all
+ linger.ms=10
+ batch.size=1048576
+
+consumerConfig: |
+ group.id=benchGroup
+ auto.offset.reset=earliest
+ enable.auto.commit=false
+ max.partition.fetch.bytes=1048576
diff --git a/driver-kafka/kafka-sync-0.yaml b/driver-kafka/kafka-sync-0.yaml
new file mode 100644
index 00000000..a4e03180
--- /dev/null
+++ b/driver-kafka/kafka-sync-0.yaml
@@ -0,0 +1,44 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements. See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership. The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied. See the License for the
+# specific language governing permissions and limitations
+# under the License.
+#
+
+# Kafka driver profile that is configuring to sync all
+# published messages to disk to achieve durability
+name: Kafka
+driverClass: io.openmessaging.benchmark.driver.kafka.KafkaBenchmarkDriver
+
+# Kafka client-specific configuration
+replicationFactor: 3
+reset: true
+
+topicConfig: |
+ min.insync.replicas=2
+ flush.messages=1
+ flush.ms=0
+
+commonConfig: |
+ bootstrap.servers=localhost:9092
+
+producerConfig: |
+ acks=0
+ linger.ms=1
+ batch.size=131072
+
+consumerConfig: |
+ auto.offset.reset=earliest
+ enable.auto.commit=false
diff --git a/driver-kafka/kafka-sync-1.yaml b/driver-kafka/kafka-sync-1.yaml
new file mode 100644
index 00000000..3bf6d7d1
--- /dev/null
+++ b/driver-kafka/kafka-sync-1.yaml
@@ -0,0 +1,44 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements. See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership. The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied. See the License for the
+# specific language governing permissions and limitations
+# under the License.
+#
+
+# Kafka driver profile that is configuring to sync all
+# published messages to disk to achieve durability
+name: Kafka
+driverClass: io.openmessaging.benchmark.driver.kafka.KafkaBenchmarkDriver
+
+# Kafka client-specific configuration
+replicationFactor: 3
+reset: true
+
+topicConfig: |
+ min.insync.replicas=2
+ flush.messages=1
+ flush.ms=0
+
+commonConfig: |
+ bootstrap.servers=localhost:9092
+
+producerConfig: |
+ acks=1
+ linger.ms=1
+ batch.size=131072
+
+consumerConfig: |
+ auto.offset.reset=earliest
+ enable.auto.commit=false
diff --git a/driver-kafka/kafka-sync.yaml b/driver-kafka/kafka-sync-all.yaml
similarity index 99%
rename from driver-kafka/kafka-sync.yaml
rename to driver-kafka/kafka-sync-all.yaml
index c53a4a51..a4fb4955 100644
--- a/driver-kafka/kafka-sync.yaml
+++ b/driver-kafka/kafka-sync-all.yaml
@@ -24,6 +24,7 @@ driverClass: io.openmessaging.benchmark.driver.kafka.KafkaBenchmarkDriver
# Kafka client-specific configuration
replicationFactor: 3
+reset: true
topicConfig: |
min.insync.replicas=2
diff --git a/driver-kafka/kafka-sync-group-all-1ms.yaml b/driver-kafka/kafka-sync-group-all-1ms.yaml
new file mode 100644
index 00000000..fcda5030
--- /dev/null
+++ b/driver-kafka/kafka-sync-group-all-1ms.yaml
@@ -0,0 +1,41 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements. See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership. The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied. See the License for the
+# specific language governing permissions and limitations
+# under the License.
+#
+
+name: Kafka
+driverClass: io.openmessaging.benchmark.driver.kafka.KafkaBenchmarkDriver
+# Kafka client-specific configuration
+replicationFactor: 3
+reset: true
+
+topicConfig: |
+ min.insync.replicas=2
+ flush.messages=1
+ flush.ms=0
+commonConfig: |
+ bootstrap.servers=localhost:9092
+ request.timeout.ms=120000
+producerConfig: |
+ acks=all
+ linger.ms=1
+ batch.size=131072
+consumerConfig: |
+ group.id=benchGroup
+ auto.offset.reset=earliest
+ enable.auto.commit=false
+ max.partition.fetch.bytes=131072
diff --git a/driver-kafka/kafka-sync-group-all.yaml b/driver-kafka/kafka-sync-group-all.yaml
new file mode 100644
index 00000000..dbd91136
--- /dev/null
+++ b/driver-kafka/kafka-sync-group-all.yaml
@@ -0,0 +1,42 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements. See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership. The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied. See the License for the
+# specific language governing permissions and limitations
+# under the License.
+#
+
+name: Kafka
+driverClass: io.openmessaging.benchmark.driver.kafka.KafkaBenchmarkDriver
+# Kafka client-specific configuration
+replicationFactor: 3
+reset: true
+
+# Configuration (flush.messages; flush.ms) enabled only for tests that fsync every message
+topicConfig: |
+ min.insync.replicas=2
+ flush.messages=1
+ flush.ms=0
+commonConfig: |
+ bootstrap.servers=localhost:9092
+ request.timeout.ms=120000
+producerConfig: |
+ acks=all
+ linger.ms=10
+ batch.size=1048576
+consumerConfig: |
+ group.id=benchGroup
+ auto.offset.reset=earliest
+ enable.auto.commit=false
+ max.partition.fetch.bytes=1048576
diff --git a/driver-kafka/pom.xml b/driver-kafka/pom.xml
index 8127f9c6..9c044ae8 100644
--- a/driver-kafka/pom.xml
+++ b/driver-kafka/pom.xml
@@ -29,6 +29,21 @@
driver-kafka
+
+
+
+
+
+ Kafka Public Repo
+ https://repository.apache.org/content/groups/public/
+
+
+
${project.groupId}
@@ -39,8 +54,17 @@
org.apache.kafka
kafka-clients
- 1.0.0
+ 2.6.0
+
+
+
\ No newline at end of file
diff --git a/driver-kafka/src/main/java/io/openmessaging/benchmark/driver/kafka/Config.java b/driver-kafka/src/main/java/io/openmessaging/benchmark/driver/kafka/Config.java
index d67452f0..da59b963 100644
--- a/driver-kafka/src/main/java/io/openmessaging/benchmark/driver/kafka/Config.java
+++ b/driver-kafka/src/main/java/io/openmessaging/benchmark/driver/kafka/Config.java
@@ -21,6 +21,8 @@
public class Config {
public short replicationFactor;
+ public boolean reset;
+
public String topicConfig;
public String commonConfig;
diff --git a/driver-kafka/src/main/java/io/openmessaging/benchmark/driver/kafka/KafkaBenchmarkConsumer.java b/driver-kafka/src/main/java/io/openmessaging/benchmark/driver/kafka/KafkaBenchmarkConsumer.java
index 9ad7e1d6..c5a4f13f 100644
--- a/driver-kafka/src/main/java/io/openmessaging/benchmark/driver/kafka/KafkaBenchmarkConsumer.java
+++ b/driver-kafka/src/main/java/io/openmessaging/benchmark/driver/kafka/KafkaBenchmarkConsumer.java
@@ -18,11 +18,13 @@
*/
package io.openmessaging.benchmark.driver.kafka;
+import java.time.Duration;
import java.util.HashMap;
import java.util.Map;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
import org.apache.kafka.clients.consumer.ConsumerRecord;
import org.apache.kafka.clients.consumer.ConsumerRecords;
@@ -52,20 +54,19 @@ public KafkaBenchmarkConsumer(KafkaConsumer consumer, ConsumerCa
this.consumerTask = this.executor.submit(() -> {
while (!closing) {
try {
- ConsumerRecords records = consumer.poll(100);
-
+ ConsumerRecords records = consumer.poll(Duration.ofMillis(100));
Map offsetMap = new HashMap<>();
for (ConsumerRecord record : records) {
- callback.messageReceived(record.value(), record.timestamp());
-
+ callback.messageReceived(record.value(), TimeUnit.MILLISECONDS.toNanos(record.timestamp()));
offsetMap.put(new TopicPartition(record.topic(), record.partition()),
- new OffsetAndMetadata(record.offset()));
+ new OffsetAndMetadata(record.offset()));
}
- if (!offsetMap.isEmpty()) {
- consumer.commitSync(offsetMap);
+ if (!records.isEmpty()) {
+ // Async commit all messages polled so far
+ consumer.commitAsync(offsetMap, null);
}
- }catch(Exception e){
+ } catch (Exception e) {
log.error("exception occur while consuming message", e);
}
}
diff --git a/driver-kafka/src/main/java/io/openmessaging/benchmark/driver/kafka/KafkaBenchmarkDriver.java b/driver-kafka/src/main/java/io/openmessaging/benchmark/driver/kafka/KafkaBenchmarkDriver.java
index 40854c66..8d708c31 100644
--- a/driver-kafka/src/main/java/io/openmessaging/benchmark/driver/kafka/KafkaBenchmarkDriver.java
+++ b/driver-kafka/src/main/java/io/openmessaging/benchmark/driver/kafka/KafkaBenchmarkDriver.java
@@ -21,18 +21,14 @@
import java.io.File;
import java.io.IOException;
import java.io.StringReader;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Properties;
+import java.util.*;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.ExecutionException;
import org.apache.bookkeeper.stats.StatsLogger;
import org.apache.kafka.clients.admin.AdminClient;
+import org.apache.kafka.clients.admin.DeleteTopicsResult;
+import org.apache.kafka.clients.admin.ListTopicsResult;
import org.apache.kafka.clients.admin.NewTopic;
import org.apache.kafka.clients.consumer.ConsumerConfig;
import org.apache.kafka.clients.consumer.KafkaConsumer;
@@ -56,7 +52,7 @@ public class KafkaBenchmarkDriver implements BenchmarkDriver {
private Config config;
- private KafkaProducer producer;
+ private List producers = Collections.synchronizedList(new ArrayList<>());
private List consumers = Collections.synchronizedList(new ArrayList<>());
private Properties topicProperties;
@@ -89,7 +85,19 @@ public void initialize(File configurationFile, StatsLogger statsLogger) throws I
admin = AdminClient.create(commonProperties);
- producer = new KafkaProducer<>(producerProperties);
+ if (config.reset) {
+ // List existing topics
+ ListTopicsResult result = admin.listTopics();
+ try {
+ Set topics = result.names().get();
+ // Delete all existing topics
+ DeleteTopicsResult deletes = admin.deleteTopics(topics);
+ deletes.all().get();
+ } catch (InterruptedException | ExecutionException e) {
+ e.printStackTrace();
+ throw new IOException(e);
+ }
+ }
}
@Override
@@ -111,33 +119,58 @@ public CompletableFuture createTopic(String topic, int partitions) {
});
}
+ @Override
+ public CompletableFuture notifyTopicCreation(String topic, int partitions) {
+ // No-op
+ return CompletableFuture.completedFuture(null);
+ }
+
@Override
public CompletableFuture createProducer(String topic) {
- return CompletableFuture.completedFuture(new KafkaBenchmarkProducer(producer, topic));
+ KafkaProducer kafkaProducer = new KafkaProducer<>(producerProperties);
+ BenchmarkProducer benchmarkProducer = new KafkaBenchmarkProducer(kafkaProducer, topic);
+ try {
+ // Add to producer list to close later
+ producers.add(benchmarkProducer);
+ return CompletableFuture.completedFuture(benchmarkProducer);
+ } catch (Throwable t) {
+ kafkaProducer.close();
+ CompletableFuture future = new CompletableFuture<>();
+ future.completeExceptionally(t);
+ return future;
+ }
}
@Override
public CompletableFuture createConsumer(String topic, String subscriptionName,
- ConsumerCallback consumerCallback) {
+ Optional partition, ConsumerCallback consumerCallback) {
Properties properties = new Properties();
consumerProperties.forEach((key, value) -> properties.put(key, value));
properties.put(ConsumerConfig.GROUP_ID_CONFIG, subscriptionName);
- KafkaConsumer consumer = new KafkaConsumer<>(properties);
+ KafkaConsumer kafkaConsumer = new KafkaConsumer<>(properties);
try {
- consumer.subscribe(Arrays.asList(topic));
- return CompletableFuture.completedFuture(new KafkaBenchmarkConsumer(consumer, consumerCallback));
+ // Subscribe
+ kafkaConsumer.subscribe(Arrays.asList(topic));
+
+ // Start polling
+ BenchmarkConsumer benchmarkConsumer = new KafkaBenchmarkConsumer(kafkaConsumer, consumerCallback);
+
+ // Add to consumer list to close later
+ consumers.add(benchmarkConsumer);
+ return CompletableFuture.completedFuture(benchmarkConsumer);
} catch (Throwable t) {
- consumer.close();
+ kafkaConsumer.close();
CompletableFuture future = new CompletableFuture<>();
future.completeExceptionally(t);
return future;
}
-
}
@Override
public void close() throws Exception {
- producer.close();
+ for (BenchmarkProducer producer : producers) {
+ producer.close();
+ }
for (BenchmarkConsumer consumer : consumers) {
consumer.close();
diff --git a/driver-msk/README.md b/driver-msk/README.md
new file mode 100644
index 00000000..6ec3eba6
--- /dev/null
+++ b/driver-msk/README.md
@@ -0,0 +1,3 @@
+# Apache Kafka benchmarks
+
+For instructions on running the OpenMessaging bencmarks for Kafka, see the [official documentation](http://openmessaging.cloud/docs/benchmarks/kafka).
diff --git a/driver-msk/deploy/alicloud/ansible.cfg b/driver-msk/deploy/alicloud/ansible.cfg
new file mode 100644
index 00000000..5e22902d
--- /dev/null
+++ b/driver-msk/deploy/alicloud/ansible.cfg
@@ -0,0 +1,8 @@
+[defaults]
+host_key_checking=false
+private_key_file=benchmark_message_alicloud.pem
+
+[privilege_escalation]
+become=true
+become_method='sudo'
+become_user='root'
diff --git a/driver-msk/deploy/alicloud/deploy.yaml b/driver-msk/deploy/alicloud/deploy.yaml
new file mode 100644
index 00000000..8cd0ed1d
--- /dev/null
+++ b/driver-msk/deploy/alicloud/deploy.yaml
@@ -0,0 +1,184 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements. See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership. The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied. See the License for the
+# specific language governing permissions and limitations
+# under the License.
+#
+
+- name: Format and mount disks for Kafka hosts
+ hosts: kafka
+ connection: ssh
+ become: true
+ tasks:
+ - command: >
+ tuned-adm profile latency-performance
+ - name: Format disks
+ filesystem:
+ fstype: xfs
+ dev: '{{ item }}'
+ with_items:
+ - '/dev/vdb'
+ - '/dev/vdc'
+ - name: Mount disks
+ mount:
+ path: "{{ item.path }}"
+ src: "{{ item.src }}"
+ fstype: xfs
+ opts: defaults,noatime,nodiscard
+ state: mounted
+ with_items:
+ - { path: "/mnt/data-1", src: "/dev/vdb" }
+ - { path: "/mnt/data-2", src: "/dev/vdc" }
+
+- name: Kafka setup
+ hosts: all
+ connection: ssh
+ become: true
+ tasks:
+ - name: Install RPM packages
+ yum: pkg={{ item }} state=latest
+ with_items:
+ - wget
+ - java
+ - sysstat
+ - vim
+ - file: path=/opt/kafka state=absent
+ - file: path=/opt/kafka state=directory
+ - set_fact:
+ zookeeperServers: "{{ groups['zookeeper'] | map('extract', hostvars, ['ansible_default_ipv4', 'address']) | map('regex_replace', '^(.*)$', '\\1:2181') | join(',') }}"
+ boostrapServers: "{{ groups['kafka'] | map('extract', hostvars, ['private_ip']) | map('regex_replace', '^(.*)$', '\\1:9092') | join(',') }}"
+ kafkaVersion: "2.0.0"
+ - debug:
+ msg: "zookeeper servers: {{ zookeeperServers }}\nboostrap servers: {{ boostrapServers }}"
+ - name: Download Kafka package
+ unarchive:
+ src: http://mirrors.ocf.berkeley.edu/apache/kafka/{{ kafkaVersion }}/kafka_2.11-{{ kafkaVersion }}.tgz
+ remote_src: yes
+ dest: /opt/kafka
+ extra_opts: ["--strip-components=1"]
+ - set_fact:
+ privateIp: "{{ hostvars[inventory_hostname]['ansible_default_ipv4']['address'] }}"
+ - name: Change locale to en_US.utf-8
+ shell: |
+ echo 'LANG=en_US.utf-8
+ LC_ALL=en_US.utf-8' > /etc/environment
+
+- name: Setup ZooKeeper
+ hosts: zookeeper
+ connection: ssh
+ become: true
+ tasks:
+ - set_fact:
+ zid: "{{ groups['zookeeper'].index(inventory_hostname) }}"
+ - file:
+ path: "/opt/kafka/data/zookeeper"
+ state: directory
+ - template:
+ src: "../templates/zookeeper.properties"
+ dest: "/opt/kafka/config/zookeeper.properties"
+ - template:
+ src: ../templates/myid
+ dest: "/opt/kafka/data/zookeeper/myid"
+ - template:
+ src: "../templates/zookeeper.service"
+ dest: "/etc/systemd/system/zookeeper.service"
+ - systemd:
+ state: restarted
+ daemon_reload: yes
+ name: "zookeeper"
+
+- name: Setup Kafka
+ hosts: kafka
+ connection: ssh
+ become: true
+ tasks:
+ - set_fact:
+ brokerId: "{{ groups['kafka'].index(inventory_hostname) }}"
+ - name: Set up broker
+ template:
+ src: "../templates/server.properties"
+ dest: "/opt/kafka/config/server.properties"
+ - template:
+ src: "../templates/kafka.service"
+ dest: "/etc/systemd/system/kafka.service"
+ - systemd:
+ state: restarted
+ daemon_reload: yes
+ name: "kafka"
+
+- name: Setup Benchmark client
+ hosts: client
+ connection: ssh
+ become: true
+ tasks:
+ - file: path=/opt/benchmark state=absent
+ - name: Copy benchmark code
+ unarchive:
+ src: ../../../package/target/openmessaging-benchmark-0.0.1-SNAPSHOT-bin.tar.gz
+ dest: /opt
+ - shell: mv /opt/openmessaging-benchmark-0.0.1-SNAPSHOT /opt/benchmark
+ - shell: tuned-adm profile latency-performance
+
+ - name: Get list of driver config files
+ raw: ls -1 /opt/benchmark/driver-kafka/*.yaml
+ register: drivers_list
+
+ - name: Configure URL
+ lineinfile:
+ dest: /opt/benchmark/driver-kafka/kafka.yaml
+ regexp: '^ bootstrap.servers='
+ line: ' bootstrap.servers={{ boostrapServers }}'
+ with_items: '{{ drivers_list.stdout_lines }}'
+ - name: Configure memory
+ lineinfile:
+ dest: /opt/benchmark/bin/benchmark-worker
+ regexp: '^JVM_MEM='
+ line: 'JVM_MEM="-Xms12G -Xmx12G -XX:+UseG1GC -XX:MaxGCPauseMillis=10 -XX:+ParallelRefProcEnabled -XX:+UnlockExperimentalVMOptions -XX:+AggressiveOpts -XX:+DoEscapeAnalysis -XX:ParallelGCThreads=32 -XX:ConcGCThreads=32 -XX:G1NewSizePercent=50 -XX:+DisableExplicitGC -XX:-ResizePLAB -XX:+PerfDisableSharedMem -XX:+AlwaysPreTouch -XX:-UseBiasedLocking"'
+ - name: Configure memory
+ lineinfile:
+ dest: /opt/benchmark/bin/benchmark
+ regexp: '^JVM_MEM='
+ line: 'JVM_MEM="-Xmx1G"'
+ - template:
+ src: "../templates/workers.yaml"
+ dest: "/opt/benchmark/workers.yaml"
+ - name: Install benchmark systemd service
+ template:
+ src: "../templates/benchmark-worker.service"
+ dest: "/etc/systemd/system/benchmark-worker.service"
+ - systemd:
+ state: restarted
+ daemon_reload: yes
+ name: "benchmark-worker"
+ - name: wait worker
+ wait_for:
+ host: '*'
+ port: 8080
+ state: started
+ timeout: 300
+
+- name: Hosts addresses
+ hosts: localhost
+ become: false
+ tasks:
+ - debug:
+ msg: "Zookeeper servers {{ item }}"
+ with_items: "{{ groups['zookeeper'] }}"
+ - debug:
+ msg: "Kafka brokers {{ item }}"
+ with_items: "{{ groups['kafka'] }}"
+ - debug:
+ msg: "Benchmark clients {{ item }}"
+ with_items: "{{ groups['client'] }}"
diff --git a/driver-msk/deploy/alicloud/provision-kafka-alicloud.tf b/driver-msk/deploy/alicloud/provision-kafka-alicloud.tf
new file mode 100644
index 00000000..01ec2892
--- /dev/null
+++ b/driver-msk/deploy/alicloud/provision-kafka-alicloud.tf
@@ -0,0 +1,107 @@
+# Define variables
+
+variable "region" {}
+variable "private_key_file" {}
+variable "key_name" {}
+variable "image_id" {}
+variable "availability_zone" {}
+variable "instance_types" {
+ type = "map"
+}
+
+variable "num_instances" {
+ type = "map"
+}
+
+# Configure the Alicloud Provider
+provider "alicloud" {
+ region = "${var.region}"
+}
+
+
+# Create zookeeper
+resource "alicloud_instance" "zookeeper" {
+ provider = "alicloud"
+ image_id = "${var.image_id}"
+ availability_zone = "${var.availability_zone}"
+
+ internet_charge_type = "PayByBandwidth"
+
+ instance_type = "${var.instance_types["zookeeper"]}"
+ security_groups = ["${alicloud_security_group.default.id}"]
+ instance_name = "zookeeper"
+ internet_max_bandwidth_out = "1"
+ system_disk_category = "cloud_ssd"
+ system_disk_size = "64"
+ description = "developer_native_compare=message"
+ count = "${var.num_instances["zookeeper"]}"
+ key_name = "${alicloud_key_pair.key_pair.id}"
+ tags {
+ Name = "zk-${count.index}"
+ }
+}
+
+# Create kafka
+resource "alicloud_instance" "kafka" {
+ provider = "alicloud"
+ image_id = "${var.image_id}"
+
+ internet_charge_type = "PayByBandwidth"
+
+ instance_type = "${var.instance_types["kafka"]}"
+ security_groups = ["${alicloud_security_group.default.id}"]
+ instance_name = "kafka"
+ internet_max_bandwidth_out = "1"
+ description = "developer_native_compare=message"
+ count = "${var.num_instances["kafka"]}"
+ key_name = "${alicloud_key_pair.key_pair.id}"
+ tags {
+ Name = "kafka-${count.index}"
+ }
+}
+
+# Create client
+resource "alicloud_instance" "client" {
+ provider = "alicloud"
+ image_id = "${var.image_id}"
+
+ internet_charge_type = "PayByBandwidth"
+
+ instance_type = "${var.instance_types["client"]}"
+ security_groups = ["${alicloud_security_group.default.id}"]
+ instance_name = "client"
+ internet_max_bandwidth_out = "1"
+ description = "developer_native_compare=message"
+ count = "${var.num_instances["client"]}"
+ key_name = "${alicloud_key_pair.key_pair.id}"
+ tags {
+ Name = "kafka-client-${count.index}"
+ }
+}
+
+# Create security group
+resource "alicloud_security_group" "default" {
+ name = "default"
+ provider = "alicloud"
+ description = "default"
+}
+
+resource "alicloud_security_group_rule" "allow_all_tcp" {
+ type = "ingress"
+ ip_protocol = "tcp"
+ nic_type = "internet"
+ policy = "accept"
+ port_range = "1/65535"
+ priority = 1
+ security_group_id = "${alicloud_security_group.default.id}"
+ cidr_ip = "0.0.0.0/0"
+}
+
+resource "alicloud_key_pair" "key_pair" {
+ key_name = "${var.key_name}"
+ key_file = "${var.private_key_file}"
+}
+
+output "client_ssh_host" {
+ value = "${alicloud_instance.client.0.public_ip}"
+}
diff --git a/driver-msk/deploy/alicloud/terraform.tfvars b/driver-msk/deploy/alicloud/terraform.tfvars
new file mode 100644
index 00000000..bae1c6af
--- /dev/null
+++ b/driver-msk/deploy/alicloud/terraform.tfvars
@@ -0,0 +1,19 @@
+region = "cn-shenzhen"
+availability_zone = "cn-shenzhen-b"
+private_key_file = "benchmark_message_alicloud.pem"
+key_name = "key-pair-from-terraform-benchmark-kafka"
+image_id = "centos_7_04_64_20G_alibase_201701015.vhd"
+
+
+
+instance_types = {
+ "kafka" = "ecs.i1.xlarge"
+ "zookeeper" = "ecs.se1.xlarge"
+ "client" = "ecs.se1.xlarge"
+}
+
+num_instances = {
+ "client" = 4
+ "kafka" = 3
+ "zookeeper" = 3
+}
diff --git a/driver-msk/deploy/ansible.cfg b/driver-msk/deploy/ansible.cfg
new file mode 100644
index 00000000..d1fac48c
--- /dev/null
+++ b/driver-msk/deploy/ansible.cfg
@@ -0,0 +1,8 @@
+[defaults]
+host_key_checking=false
+private_key_file=~/.ssh/kafka_aws
+
+[privilege_escalation]
+become=true
+become_method='sudo'
+become_user='root'
\ No newline at end of file
diff --git a/driver-msk/deploy/deploy.yaml b/driver-msk/deploy/deploy.yaml
new file mode 100644
index 00000000..fd004eec
--- /dev/null
+++ b/driver-msk/deploy/deploy.yaml
@@ -0,0 +1,367 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements. See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership. The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied. See the License for the
+# specific language governing permissions and limitations
+# under the License.
+#
+
+- name: Collect profiles
+ hosts: client
+ become: true
+ tags:
+ - ["never", "profile"]
+ tasks:
+ - set_fact:
+ time: "{{ ansible_date_time.epoch }}"
+ - name: Generate Flame graph
+ tags:
+ - flame
+ shell: |
+ /tmp/profiler.sh -d 120 `pidof java` -f /tmp/flame.svg
+ - name: Fetch flame graph
+ tags:
+ - flame
+ fetch:
+ src: /tmp/flame.svg
+ dest: results/{{ group_names[0] }}/{{ time }}/{{ inventory_hostname }}/flame.svg
+ flat: yes
+
+ # - name: Dump & compress heap
+ # tags:
+ # - heap
+ # shell: |
+ # jattach `pidof java` dumpheap /tmp/heap
+ # chmod 777 /tmp/heap
+ # tar -cvzf /tmp/heap.tar /tmp/heap
+ # - name: Fetch heap dump
+ # tags:
+ # - heap
+ # fetch:
+ # src: /tmp/heap.tar
+ # dest: results/{{ group_names[0] }}/{{ time }}/{{ inventory_hostname }}/heap.tar
+ # flat: yes
+
+ - name: Heap allocation profile
+ tags:
+ - heap
+ shell: |
+ /tmp/profiler.sh -d 60 -e alloc -o svg -f /tmp/heap.svg 1 `pidof java`
+ chmod 777 /tmp/heap.svg
+ - name: Fetch heap dump
+ tags:
+ - heap
+ fetch:
+ src: /tmp/heap.svg
+ dest: results/{{ group_names[0] }}/{{ time }}/{{ inventory_hostname }}/heap.svg
+ flat: yes
+
+- name: Store server lists
+ hosts: all
+ connection: ssh
+ tasks:
+ - set_fact:
+# zookeeperServers: "{{ groups['zookeeper'] | map('extract', hostvars, ['ansible_default_ipv4', 'address']) | map('regex_replace', '^(.*)$', '\\1:2181') | join(',') }}"
+# boostrapServers: "{{ groups['kafka'] | map('extract', hostvars, ['private_ip']) | map('regex_replace', '^(.*)$', '\\1:9092') | join(',') }}"
+# zookeeperServers: "{{ map('extract', hostvars, ['zookeeperServers']) }}"
+# boostrapServers: "{{ map('extract', hostvars, ['boostrapServers']) }}"
+ #boostrapServers: "{{ groups['all'] | map('extract', hostvars, ['vars']) }}"
+ zookeeperServers: "{{ groups['all'] | map('extract', hostvars, ['zookeeperServers']) }}"
+ boostrapServers: "{{ groups['all'] | map('extract', hostvars, ['boostrapServers']) }}"
+
+- name: Setup common packages
+ hosts: all
+ connection: ssh
+ become: true
+ tasks:
+ - name: Remove sysstat
+ yum: pkg={{ item }} state=absent
+ with_items:
+ - sysstat
+ - name: Install RPM packages
+ yum: pkg={{ item }} state=latest
+ with_items:
+ - wget
+ - java-11
+ - sysstat
+ - vim
+ - chrony
+ - git
+ - gcc
+ - make
+
+- name: Chrony setup
+ hosts: client
+ connection: ssh
+ become: true
+ tasks:
+ - name: Set up chronyd
+ template:
+ src: "templates/chrony.conf"
+ dest: "/etc/chrony.conf"
+ - systemd:
+ state: restarted
+ daemon_reload: yes
+ name: "chronyd"
+
+- name: Setup Benchmark client
+ hosts: client
+ connection: ssh
+ become: true
+ tasks:
+ - file: path=/opt/benchmark state=absent
+ - name: Copy benchmark code
+ unarchive:
+ src: ../../package/target/openmessaging-benchmark-0.0.1-SNAPSHOT-bin.tar.gz
+ dest: /opt
+ - shell: mv /opt/openmessaging-benchmark-0.0.1-SNAPSHOT /opt/benchmark
+ - shell: tuned-adm profile latency-performance
+
+ - name: Get list of driver config files
+ raw: ls -1 /opt/benchmark/driver-kafka/*.yaml
+ register: drivers_list
+
+ - name: Configure URL
+ lineinfile:
+ dest: "{{ item }}"
+ regexp: "^ bootstrap.servers="
+ line: " bootstrap.servers={{ boostrapServers }}"
+ with_items: "{{ drivers_list.stdout_lines }}"
+ - name: Configure memory
+ lineinfile:
+ dest: /opt/benchmark/bin/benchmark-worker
+ regexp: "^JVM_MEM="
+ line: 'JVM_MEM="-Xms16G -Xmx16G -XX:+UseG1GC -XX:MaxGCPauseMillis=10 -XX:+ParallelRefProcEnabled -XX:+UnlockExperimentalVMOptions -XX:+AggressiveOpts -XX:+DoEscapeAnalysis -XX:ParallelGCThreads=32 -XX:ConcGCThreads=32 -XX:G1NewSizePercent=50 -XX:+DisableExplicitGC -XX:-ResizePLAB -XX:+PerfDisableSharedMem -XX:+AlwaysPreTouch -XX:-UseBiasedLocking"'
+ - name: Configure memory
+ lineinfile:
+ dest: /opt/benchmark/bin/benchmark
+ regexp: "^JVM_MEM="
+ line: 'JVM_MEM="-Xmx1G"'
+ - name: Setup JMX for Producer/Consumer clients
+ copy:
+ src: "monitoring/jmx_exporter/jmx_prometheus_javaagent-0.13.0.jar"
+ dest: "/opt/benchmark/jmx_prometheus_javaagent-0.13.0.jar"
+ - name: Configure JMX for Prometheus (Client)
+ template:
+ src: "monitoring/jmx_exporter/client.yml"
+ dest: "/opt/benchmark/metrics.yml"
+ - template:
+ src: "templates/workers.yaml"
+ dest: "/opt/benchmark/workers.yaml"
+ - name: Install benchmark systemd service
+ template:
+ src: "templates/benchmark-worker.service"
+ dest: "/etc/systemd/system/benchmark-worker.service"
+ - name: "Restart benchmark worker"
+ tags:
+ - configure
+ systemd:
+ state: restarted
+ daemon_reload: yes
+ name: "benchmark-worker"
+
+- name: Hosts addresses
+ hosts: localhost
+ become: false
+ tasks:
+# - debug:
+# msg: "Zookeeper servers {{ item }}"
+# with_items: "{{ groups['zookeeper'] }}"
+# - debug:
+# msg: "Kafka brokers {{ item }}"
+# with_items: "{{ groups['kafka'] }}"
+ - debug:
+ msg: "Benchmark clients {{ item }}"
+ with_items: "{{ groups['client'] }}"
+
+# Install the monitoring stack
+#- name: Install Node Exporter
+# hosts: client
+# roles:
+# - cloudalchemy.node-exporter
+
+#- hosts: prometheus
+# roles:
+# - cloudalchemy.prometheus
+# vars:
+# prometheus_scrape_configs:
+# - job_name: "broker"
+# static_configs:
+# - targets: "{{ groups['kafka'] | map('extract', hostvars, ['ansible_default_ipv4','address']) | map('regex_replace', '^(.*)$','\\1:8080') | list }}"
+# - job_name: "node"
+# static_configs:
+# - targets: "{{ groups['kafka'] | map('extract', hostvars, ['ansible_default_ipv4','address']) | map('regex_replace', '^(.*)$','\\1:9100') | list }}"
+# - targets: "{{ groups['client'] | map('extract', hostvars, ['ansible_default_ipv4','address']) | map('regex_replace', '^(.*)$','\\1:9100') | list }}"
+# - job_name: "client - jmx"
+# static_configs:
+# - targets: "{{ groups['client'] | map('extract', hostvars, ['ansible_default_ipv4', 'address']) | map('regex_replace', '^(.*)$', '\\1:9090') | list }}"
+# - job_name: "client"
+# static_configs:
+# - targets: "{{ groups['client'] | map('extract', hostvars, ['ansible_default_ipv4', 'address']) | map('regex_replace', '^(.*)$', '\\1:8081') | list }}"
+
+#- name: jmx export setup
+# hosts: prometheus
+# connection: ssh
+# become: true
+# tasks:
+# - template:
+# src: "templates/kafka-dashboard.service"
+# dest: "/etc/systemd/system/kafka-dashboard.service"
+# - systemd:
+# state: restarted
+# daemon_reload: yes
+# name: "kafka-dashboard"
+
+- hosts: prometheus
+ roles:
+ - cloudalchemy.grafana
+ vars:
+ grafana_version: 6.7.3
+ grafana_security:
+ admin_user: admin
+ admin_password: enter_your_secure_password
+ grafana_datasources:
+ - name: prometheus
+ type: prometheus
+ access: proxy
+ url: 'http://localhost:9090'
+ basicAuth: false
+ grafana_dashboards:
+ - dashboard_id: 1860
+ revision_id: 21
+ datasource: prometheus
+
+# Setup the Prometheus/Grafana monitoring stack
+#- name: Prometheus installation
+# hosts: prometheus
+# connection: ssh
+# become: true
+# tasks:
+# - name: Add Extras Repo
+# shell: yum-config-manager --enable rhui-REGION-rhel-server-extras
+# - name: Install RPM packages
+# yum: pkg={{ item }} state=latest
+# with_items:
+# - docker
+# - set_fact:
+# prometheusVersion: "2.19.3"
+# - file: path=/opt/prometheus state=absent
+# - file: path=/opt/prometheus state=directory
+# - name: Download Prometheus Binary Package
+# unarchive:
+# src: "https://github.com/prometheus/prometheus/releases/download/v{{ prometheusVersion }}/prometheus-{{ prometheusVersion }}.linux-amd64.tar.gz"
+# remote_src: yes
+# dest: /opt/prometheus
+# extra_opts: ["--strip-components=1"]
+# - set_fact:
+# private_ip: "{{ hostvars[inventory_hostname]['ansible_default_ipv4']['address'] }}"
+#
+#- name: Prometheus setup
+# hosts: prometheus
+# connection: ssh
+# become: true
+# tasks:
+# - file:
+# path: "/opt/prometheus/{{ item }}"
+# state: directory
+# with_items:
+# - data
+# - template:
+# src: "templates/prometheus.service"
+# dest: "/etc/systemd/system/prometheus.service"
+# - template:
+# src: "templates/kafka-dashboard.service"
+# dest: "/etc/systemd/system/kafka-dashboard.service"
+# - template:
+# src: "templates/prometheus.yml"
+# dest: "/opt/prometheus/prometheus.yml"
+# - systemd:
+# state: restarted
+# daemon_reload: yes
+# name: "kafka-dashboard"
+#
+#- name: Install Node exporter on Brokers to collect system metrics
+# hosts: kafka
+# become: true
+# tasks:
+# - name: Add user node_exporter
+# user:
+# name: node_exporter
+# shell: /bin/false
+# system: true
+# create_home: no
+#
+# - name: Download and extract
+# unarchive:
+# src: https://github.com/prometheus/node_exporter/releases/download/v1.0.1/node_exporter-1.0.1.linux-amd64.tar.gz
+# dest: /tmp
+# remote_src: yes
+#
+# - name: Copy bin node_exporter to /usr/local/bin
+# copy:
+# src: /tmp/node_exporter-1.0.1.linux-amd64/node_exporter
+# remote_src: yes
+# dest: /usr/local/bin/
+# owner: node_exporter
+# group: node_exporter
+# mode: u+x,g+x,o+x
+#
+# - name: Create service node_exporter.service
+# blockinfile:
+# path: /etc/systemd/system/node_exporter.service
+# block: |
+# [Unit]
+# Description=Prometheus Node Exporter
+# Wants=network-online.target
+# After=network-online.target
+# [Service]
+# User=node_exporter
+# Group=node_exporter
+# Type=simple
+# ExecStart=/usr/local/bin/node_exporter
+# [Install]
+# WantedBy=multi-user.target
+# create: true
+#
+# - name: systemctl daemon_reload
+# systemd:
+# daemon_reload: yes
+#
+# - name: Start and Enable node_exporter
+# service:
+# name: node_exporter
+# state: started
+# enabled: yes
+#
+# - name: Check whether port 9100 is available
+# wait_for:
+# port: 9100
+# state: started
+# timeout: 5
+
+- name: Install jattach and async-profiler
+ hosts: all
+ become: true
+ tasks:
+ - name: Download and extract async-profiler
+ unarchive:
+ src: https://github.com/jvm-profiling-tools/async-profiler/releases/download/v1.7.1/async-profiler-1.7.1-linux-x64.tar.gz
+ dest: /tmp
+ remote_src: yes
+
+ - name: Download and install jattach
+ get_url:
+ url: https://github.com/apangin/jattach/releases/download/v1.5/jattach
+ dest: /usr/bin/jattach
+ mode: "0755"
diff --git a/driver-msk/deploy/monitoring/Dockerfile b/driver-msk/deploy/monitoring/Dockerfile
new file mode 100644
index 00000000..0ae4874d
--- /dev/null
+++ b/driver-msk/deploy/monitoring/Dockerfile
@@ -0,0 +1,54 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements. See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership. The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied. See the License for the
+# specific language governing permissions and limitations
+# under the License.
+#
+
+FROM grafana/grafana:7.0.5
+
+USER root
+
+# Install the Grafana Image Rendering Plugin
+ENV GF_PATHS_PLUGINS="/var/lib/grafana-plugins"
+RUN mkdir -p "$GF_PATHS_PLUGINS" && \
+ chown -R grafana:grafana "$GF_PATHS_PLUGINS"
+RUN echo "http://dl-cdn.alpinelinux.org/alpine/edge/community" >> /etc/apk/repositories && \
+ echo "http://dl-cdn.alpinelinux.org/alpine/edge/main" >> /etc/apk/repositories && \
+ echo "http://dl-cdn.alpinelinux.org/alpine/edge/testing" >> /etc/apk/repositories && \
+ apk --no-cache upgrade && \
+ apk add --no-cache udev ttf-opensans chromium && \
+ rm -rf /tmp/* && \
+ rm -rf /usr/share/grafana/tools/phantomjs;
+USER grafana
+ENV GF_RENDERER_PLUGIN_CHROME_BIN="/usr/bin/chromium-browser"
+RUN grafana-cli \
+ --pluginsDir "$GF_PATHS_PLUGINS" \
+ --pluginUrl https://github.com/grafana/grafana-image-renderer/releases/latest/download/plugin-linux-x64-glibc-no-chromium.zip \
+ plugins install grafana-image-renderer;
+
+USER root
+COPY dashboards.yaml /etc/grafana/provisioning/dashboards/dashboards.yaml
+RUN apk add curl
+USER grafana
+COPY grafana.ini /etc/grafana/grafana.ini
+COPY dashboards/* /var/lib/grafana/dashboards/
+COPY start.sh /start.sh
+
+EXPOSE 3000
+
+ENV PROMETHEUS_URL http://prometheus:9090
+
+ENTRYPOINT ["/start.sh"]
diff --git a/driver-msk/deploy/monitoring/README.md b/driver-msk/deploy/monitoring/README.md
new file mode 100644
index 00000000..1db30933
--- /dev/null
+++ b/driver-msk/deploy/monitoring/README.md
@@ -0,0 +1,3 @@
+# Kafka dashboard for the Open Messaging Benchmark using the Prometheus/Grafana stack
+
+- Builds a docker image
diff --git a/driver-msk/deploy/monitoring/dashboards.yaml b/driver-msk/deploy/monitoring/dashboards.yaml
new file mode 100644
index 00000000..4a9c9c4a
--- /dev/null
+++ b/driver-msk/deploy/monitoring/dashboards.yaml
@@ -0,0 +1,45 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements. See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership. The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied. See the License for the
+# specific language governing permissions and limitations
+# under the License.
+#
+
+apiVersion: 1
+
+providers:
+ # an unique provider name. Required
+ - name: "Default Dashboards"
+ # Org id. Default to 1
+ orgId: 1
+ # name of the dashboard folder.
+ folder: ""
+ # folder UID. will be automatically generated if not specified
+ folderUid: ""
+ # provider type. Default to 'file'
+ type: file
+ # disable dashboard deletion
+ disableDeletion: true
+ # enable dashboard editing
+ editable: true
+ # how often Grafana will scan for changed dashboards
+ updateIntervalSeconds: 10
+ # allow updating provisioned dashboards from the UI
+ allowUiUpdates: true
+ options:
+ # path to dashboard files on disk. Required when using the 'file' type
+ path: /var/lib/grafana/dashboards
+ # use folder names from filesystem to create folders in Grafana
+ foldersFromFilesStructure: true
diff --git a/driver-msk/deploy/monitoring/dashboards/jvm.json b/driver-msk/deploy/monitoring/dashboards/jvm.json
new file mode 100644
index 00000000..8d5ee19b
--- /dev/null
+++ b/driver-msk/deploy/monitoring/dashboards/jvm.json
@@ -0,0 +1,2557 @@
+{
+ "annotations": {
+ "list": []
+ },
+ "editable": true,
+ "gnetId": null,
+ "graphTooltip": 0,
+ "hideControls": false,
+ "id": 2,
+ "links": [],
+ "refresh": "1m",
+ "rows": [
+ {
+ "collapse": false,
+ "height": 221,
+ "panels": [
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": null,
+ "decimals": 1,
+ "fill": 1,
+ "id": 4,
+ "legend": {
+ "avg": false,
+ "current": false,
+ "max": false,
+ "min": false,
+ "show": false,
+ "total": false,
+ "values": false
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "nullPointMode": "null",
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "scopedVars": {
+ "instance": {
+ "selected": false,
+ "text": "bookie-232tg",
+ "value": "bookie-232tg"
+ }
+ },
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "span": 3,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "irate(process_cpu_seconds_total{cluster=\"$cluster\", kubernetes_pod_name=~\"$instance\"}[30s]) * 100\n\n",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "{{kubernetes_pod_name}}",
+ "metric": "process_cpu_seconds_total",
+ "refId": "A",
+ "step": 20
+ }
+ ],
+ "thresholds": [
+ {
+ "colorMode": "ok",
+ "fill": false,
+ "line": true,
+ "op": "gt",
+ "value": 100
+ }
+ ],
+ "timeFrom": null,
+ "timeShift": null,
+ "title": "CPU",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "percent",
+ "label": "",
+ "logBase": 1,
+ "max": null,
+ "min": "0",
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ }
+ ]
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "Prometheus",
+ "fill": 1,
+ "id": 1,
+ "legend": {
+ "alignAsTable": false,
+ "avg": false,
+ "current": false,
+ "hideEmpty": false,
+ "max": false,
+ "min": false,
+ "rightSide": false,
+ "show": false,
+ "total": false,
+ "values": false
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "nullPointMode": "null",
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "scopedVars": {
+ "instance": {
+ "selected": false,
+ "text": "bookie-232tg",
+ "value": "bookie-232tg"
+ }
+ },
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "span": 3,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "sum(jvm_memory_bytes_committed{cluster=\"$cluster\", kubernetes_pod_name=~\"$instance\"})",
+ "format": "time_series",
+ "hide": false,
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "Heap size",
+ "metric": "jvm_memory_bytes_committed",
+ "refId": "A",
+ "step": 20
+ },
+ {
+ "expr": "sum(jvm_memory_bytes_used{cluster=\"$cluster\", kubernetes_pod_name=~\"$instance\"})",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "Used",
+ "metric": "jvm_memory_bytes_used",
+ "refId": "B",
+ "step": 20
+ },
+ {
+ "expr": "sum(jvm_memory_bytes_max{cluster=\"$cluster\", kubernetes_pod_name=~\"$instance\"})",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "Max",
+ "metric": "jvm_memory_bytes_max",
+ "refId": "C",
+ "step": 20
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeShift": null,
+ "title": "Heap Memory",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "decbytes",
+ "label": "",
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ]
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": null,
+ "fill": 1,
+ "id": 2,
+ "legend": {
+ "avg": false,
+ "current": false,
+ "max": false,
+ "min": false,
+ "show": false,
+ "total": false,
+ "values": false
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "nullPointMode": "null",
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "scopedVars": {
+ "instance": {
+ "selected": false,
+ "text": "bookie-232tg",
+ "value": "bookie-232tg"
+ }
+ },
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "span": 3,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "jvm_memory_direct_bytes_used{cluster=\"$cluster\", kubernetes_pod_name=~\"$instance\"}",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "Used",
+ "metric": "jvm_memory_direct_bytes_used",
+ "refId": "A",
+ "step": 20
+ },
+ {
+ "expr": "jvm_memory_direct_bytes_max{cluster=\"$cluster\", kubernetes_pod_name=~\"$instance\"}",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "Max",
+ "metric": "jvm_memory_direct_bytes_max",
+ "refId": "B",
+ "step": 20
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeShift": null,
+ "title": "Direct Memory",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "decbytes",
+ "label": "",
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": "",
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ }
+ ]
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": null,
+ "fill": 1,
+ "id": 3,
+ "legend": {
+ "avg": false,
+ "current": false,
+ "max": false,
+ "min": false,
+ "show": false,
+ "total": false,
+ "values": false
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "nullPointMode": "null",
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "scopedVars": {
+ "instance": {
+ "selected": false,
+ "text": "bookie-232tg",
+ "value": "bookie-232tg"
+ }
+ },
+ "seriesOverrides": [
+ {
+ "alias": "old count",
+ "yaxis": 2
+ },
+ {
+ "alias": "young count",
+ "yaxis": 2
+ }
+ ],
+ "spaceLength": 10,
+ "span": 3,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "sum(increase(jvm_gc_collection_seconds_sum{cluster=\"$cluster\", kubernetes_pod_name=~\"$instance\"}[30s]))",
+ "format": "time_series",
+ "hide": false,
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "Pause",
+ "metric": "jvm_gc_collection_seconds_count",
+ "refId": "B",
+ "step": 20
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeShift": null,
+ "title": "GC Pauses",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "s",
+ "label": "",
+ "logBase": 1,
+ "max": null,
+ "min": "0",
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": "",
+ "logBase": 1,
+ "max": null,
+ "min": "0",
+ "show": false
+ }
+ ]
+ }
+ ],
+ "repeat": "instance",
+ "repeatIteration": null,
+ "repeatRowId": null,
+ "showTitle": true,
+ "title": "$component $instance",
+ "titleSize": "h4"
+ },
+ {
+ "collapse": false,
+ "height": 221,
+ "panels": [
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": null,
+ "decimals": 1,
+ "fill": 1,
+ "id": 5,
+ "legend": {
+ "avg": false,
+ "current": false,
+ "max": false,
+ "min": false,
+ "show": false,
+ "total": false,
+ "values": false
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "nullPointMode": "null",
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "scopedVars": {
+ "instance": {
+ "selected": false,
+ "text": "bookie-44jrt",
+ "value": "bookie-44jrt"
+ }
+ },
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "span": 3,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "irate(process_cpu_seconds_total{cluster=\"$cluster\", kubernetes_pod_name=~\"$instance\"}[30s]) * 100\n\n",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "{{kubernetes_pod_name}}",
+ "metric": "process_cpu_seconds_total",
+ "refId": "A",
+ "step": 20
+ }
+ ],
+ "thresholds": [
+ {
+ "colorMode": "ok",
+ "fill": false,
+ "line": true,
+ "op": "gt",
+ "value": 100
+ }
+ ],
+ "timeFrom": null,
+ "timeShift": null,
+ "title": "CPU",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "percent",
+ "label": "",
+ "logBase": 1,
+ "max": null,
+ "min": "0",
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ }
+ ]
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "Prometheus",
+ "fill": 1,
+ "id": 6,
+ "legend": {
+ "alignAsTable": false,
+ "avg": false,
+ "current": false,
+ "hideEmpty": false,
+ "max": false,
+ "min": false,
+ "rightSide": false,
+ "show": false,
+ "total": false,
+ "values": false
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "nullPointMode": "null",
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "scopedVars": {
+ "instance": {
+ "selected": false,
+ "text": "bookie-44jrt",
+ "value": "bookie-44jrt"
+ }
+ },
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "span": 3,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "sum(jvm_memory_bytes_committed{cluster=\"$cluster\", kubernetes_pod_name=~\"$instance\"})",
+ "format": "time_series",
+ "hide": false,
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "Heap size",
+ "metric": "jvm_memory_bytes_committed",
+ "refId": "A",
+ "step": 20
+ },
+ {
+ "expr": "sum(jvm_memory_bytes_used{cluster=\"$cluster\", kubernetes_pod_name=~\"$instance\"})",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "Used",
+ "metric": "jvm_memory_bytes_used",
+ "refId": "B",
+ "step": 20
+ },
+ {
+ "expr": "sum(jvm_memory_bytes_max{cluster=\"$cluster\", kubernetes_pod_name=~\"$instance\"})",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "Max",
+ "metric": "jvm_memory_bytes_max",
+ "refId": "C",
+ "step": 20
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeShift": null,
+ "title": "Heap Memory",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "decbytes",
+ "label": "",
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ]
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": null,
+ "fill": 1,
+ "id": 7,
+ "legend": {
+ "avg": false,
+ "current": false,
+ "max": false,
+ "min": false,
+ "show": false,
+ "total": false,
+ "values": false
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "nullPointMode": "null",
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "scopedVars": {
+ "instance": {
+ "selected": false,
+ "text": "bookie-44jrt",
+ "value": "bookie-44jrt"
+ }
+ },
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "span": 3,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "jvm_memory_direct_bytes_used{cluster=\"$cluster\", kubernetes_pod_name=~\"$instance\"}",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "Used",
+ "metric": "jvm_memory_direct_bytes_used",
+ "refId": "A",
+ "step": 20
+ },
+ {
+ "expr": "jvm_memory_direct_bytes_max{cluster=\"$cluster\", kubernetes_pod_name=~\"$instance\"}",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "Max",
+ "metric": "jvm_memory_direct_bytes_max",
+ "refId": "B",
+ "step": 20
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeShift": null,
+ "title": "Direct Memory",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "decbytes",
+ "label": "",
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": "",
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ }
+ ]
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": null,
+ "fill": 1,
+ "id": 8,
+ "legend": {
+ "avg": false,
+ "current": false,
+ "max": false,
+ "min": false,
+ "show": false,
+ "total": false,
+ "values": false
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "nullPointMode": "null",
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "scopedVars": {
+ "instance": {
+ "selected": false,
+ "text": "bookie-44jrt",
+ "value": "bookie-44jrt"
+ }
+ },
+ "seriesOverrides": [
+ {
+ "alias": "old count",
+ "yaxis": 2
+ },
+ {
+ "alias": "young count",
+ "yaxis": 2
+ }
+ ],
+ "spaceLength": 10,
+ "span": 3,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "sum(increase(jvm_gc_collection_seconds_sum{cluster=\"$cluster\", kubernetes_pod_name=~\"$instance\"}[30s]))",
+ "format": "time_series",
+ "hide": false,
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "Pause",
+ "metric": "jvm_gc_collection_seconds_count",
+ "refId": "B",
+ "step": 20
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeShift": null,
+ "title": "GC Pauses",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "s",
+ "label": "",
+ "logBase": 1,
+ "max": null,
+ "min": "0",
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": "",
+ "logBase": 1,
+ "max": null,
+ "min": "0",
+ "show": false
+ }
+ ]
+ }
+ ],
+ "repeat": null,
+ "repeatIteration": 1496162786419,
+ "repeatRowId": 1,
+ "showTitle": true,
+ "title": "$component $instance",
+ "titleSize": "h4"
+ },
+ {
+ "collapse": false,
+ "height": 221,
+ "panels": [
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": null,
+ "decimals": 1,
+ "fill": 1,
+ "id": 9,
+ "legend": {
+ "avg": false,
+ "current": false,
+ "max": false,
+ "min": false,
+ "show": false,
+ "total": false,
+ "values": false
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "nullPointMode": "null",
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "scopedVars": {
+ "instance": {
+ "selected": false,
+ "text": "bookie-73k0d",
+ "value": "bookie-73k0d"
+ }
+ },
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "span": 3,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "irate(process_cpu_seconds_total{cluster=\"$cluster\", kubernetes_pod_name=~\"$instance\"}[30s]) * 100\n\n",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "{{kubernetes_pod_name}}",
+ "metric": "process_cpu_seconds_total",
+ "refId": "A",
+ "step": 20
+ }
+ ],
+ "thresholds": [
+ {
+ "colorMode": "ok",
+ "fill": false,
+ "line": true,
+ "op": "gt",
+ "value": 100
+ }
+ ],
+ "timeFrom": null,
+ "timeShift": null,
+ "title": "CPU",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "percent",
+ "label": "",
+ "logBase": 1,
+ "max": null,
+ "min": "0",
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ }
+ ]
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "Prometheus",
+ "fill": 1,
+ "id": 10,
+ "legend": {
+ "alignAsTable": false,
+ "avg": false,
+ "current": false,
+ "hideEmpty": false,
+ "max": false,
+ "min": false,
+ "rightSide": false,
+ "show": false,
+ "total": false,
+ "values": false
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "nullPointMode": "null",
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "scopedVars": {
+ "instance": {
+ "selected": false,
+ "text": "bookie-73k0d",
+ "value": "bookie-73k0d"
+ }
+ },
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "span": 3,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "sum(jvm_memory_bytes_committed{cluster=\"$cluster\", kubernetes_pod_name=~\"$instance\"})",
+ "format": "time_series",
+ "hide": false,
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "Heap size",
+ "metric": "jvm_memory_bytes_committed",
+ "refId": "A",
+ "step": 20
+ },
+ {
+ "expr": "sum(jvm_memory_bytes_used{cluster=\"$cluster\", kubernetes_pod_name=~\"$instance\"})",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "Used",
+ "metric": "jvm_memory_bytes_used",
+ "refId": "B",
+ "step": 20
+ },
+ {
+ "expr": "sum(jvm_memory_bytes_max{cluster=\"$cluster\", kubernetes_pod_name=~\"$instance\"})",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "Max",
+ "metric": "jvm_memory_bytes_max",
+ "refId": "C",
+ "step": 20
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeShift": null,
+ "title": "Heap Memory",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "decbytes",
+ "label": "",
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ]
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": null,
+ "fill": 1,
+ "id": 11,
+ "legend": {
+ "avg": false,
+ "current": false,
+ "max": false,
+ "min": false,
+ "show": false,
+ "total": false,
+ "values": false
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "nullPointMode": "null",
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "scopedVars": {
+ "instance": {
+ "selected": false,
+ "text": "bookie-73k0d",
+ "value": "bookie-73k0d"
+ }
+ },
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "span": 3,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "jvm_memory_direct_bytes_used{cluster=\"$cluster\", kubernetes_pod_name=~\"$instance\"}",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "Used",
+ "metric": "jvm_memory_direct_bytes_used",
+ "refId": "A",
+ "step": 20
+ },
+ {
+ "expr": "jvm_memory_direct_bytes_max{cluster=\"$cluster\", kubernetes_pod_name=~\"$instance\"}",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "Max",
+ "metric": "jvm_memory_direct_bytes_max",
+ "refId": "B",
+ "step": 20
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeShift": null,
+ "title": "Direct Memory",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "decbytes",
+ "label": "",
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": "",
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ }
+ ]
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": null,
+ "fill": 1,
+ "id": 12,
+ "legend": {
+ "avg": false,
+ "current": false,
+ "max": false,
+ "min": false,
+ "show": false,
+ "total": false,
+ "values": false
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "nullPointMode": "null",
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "scopedVars": {
+ "instance": {
+ "selected": false,
+ "text": "bookie-73k0d",
+ "value": "bookie-73k0d"
+ }
+ },
+ "seriesOverrides": [
+ {
+ "alias": "old count",
+ "yaxis": 2
+ },
+ {
+ "alias": "young count",
+ "yaxis": 2
+ }
+ ],
+ "spaceLength": 10,
+ "span": 3,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "sum(increase(jvm_gc_collection_seconds_sum{cluster=\"$cluster\", kubernetes_pod_name=~\"$instance\"}[30s]))",
+ "format": "time_series",
+ "hide": false,
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "Pause",
+ "metric": "jvm_gc_collection_seconds_count",
+ "refId": "B",
+ "step": 20
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeShift": null,
+ "title": "GC Pauses",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "s",
+ "label": "",
+ "logBase": 1,
+ "max": null,
+ "min": "0",
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": "",
+ "logBase": 1,
+ "max": null,
+ "min": "0",
+ "show": false
+ }
+ ]
+ }
+ ],
+ "repeat": null,
+ "repeatIteration": 1496162786419,
+ "repeatRowId": 1,
+ "showTitle": true,
+ "title": "$component $instance",
+ "titleSize": "h4"
+ },
+ {
+ "collapse": false,
+ "height": 221,
+ "panels": [
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": null,
+ "decimals": 1,
+ "fill": 1,
+ "id": 13,
+ "legend": {
+ "avg": false,
+ "current": false,
+ "max": false,
+ "min": false,
+ "show": false,
+ "total": false,
+ "values": false
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "nullPointMode": "null",
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "scopedVars": {
+ "instance": {
+ "selected": false,
+ "text": "bookie-h2q4k",
+ "value": "bookie-h2q4k"
+ }
+ },
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "span": 3,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "irate(process_cpu_seconds_total{cluster=\"$cluster\", kubernetes_pod_name=~\"$instance\"}[30s]) * 100\n\n",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "{{kubernetes_pod_name}}",
+ "metric": "process_cpu_seconds_total",
+ "refId": "A",
+ "step": 20
+ }
+ ],
+ "thresholds": [
+ {
+ "colorMode": "ok",
+ "fill": false,
+ "line": true,
+ "op": "gt",
+ "value": 100
+ }
+ ],
+ "timeFrom": null,
+ "timeShift": null,
+ "title": "CPU",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "percent",
+ "label": "",
+ "logBase": 1,
+ "max": null,
+ "min": "0",
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ }
+ ]
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "Prometheus",
+ "fill": 1,
+ "id": 14,
+ "legend": {
+ "alignAsTable": false,
+ "avg": false,
+ "current": false,
+ "hideEmpty": false,
+ "max": false,
+ "min": false,
+ "rightSide": false,
+ "show": false,
+ "total": false,
+ "values": false
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "nullPointMode": "null",
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "scopedVars": {
+ "instance": {
+ "selected": false,
+ "text": "bookie-h2q4k",
+ "value": "bookie-h2q4k"
+ }
+ },
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "span": 3,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "sum(jvm_memory_bytes_committed{cluster=\"$cluster\", kubernetes_pod_name=~\"$instance\"})",
+ "format": "time_series",
+ "hide": false,
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "Heap size",
+ "metric": "jvm_memory_bytes_committed",
+ "refId": "A",
+ "step": 20
+ },
+ {
+ "expr": "sum(jvm_memory_bytes_used{cluster=\"$cluster\", kubernetes_pod_name=~\"$instance\"})",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "Used",
+ "metric": "jvm_memory_bytes_used",
+ "refId": "B",
+ "step": 20
+ },
+ {
+ "expr": "sum(jvm_memory_bytes_max{cluster=\"$cluster\", kubernetes_pod_name=~\"$instance\"})",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "Max",
+ "metric": "jvm_memory_bytes_max",
+ "refId": "C",
+ "step": 20
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeShift": null,
+ "title": "Heap Memory",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "decbytes",
+ "label": "",
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ]
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": null,
+ "fill": 1,
+ "id": 15,
+ "legend": {
+ "avg": false,
+ "current": false,
+ "max": false,
+ "min": false,
+ "show": false,
+ "total": false,
+ "values": false
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "nullPointMode": "null",
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "scopedVars": {
+ "instance": {
+ "selected": false,
+ "text": "bookie-h2q4k",
+ "value": "bookie-h2q4k"
+ }
+ },
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "span": 3,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "jvm_memory_direct_bytes_used{cluster=\"$cluster\", kubernetes_pod_name=~\"$instance\"}",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "Used",
+ "metric": "jvm_memory_direct_bytes_used",
+ "refId": "A",
+ "step": 20
+ },
+ {
+ "expr": "jvm_memory_direct_bytes_max{cluster=\"$cluster\", kubernetes_pod_name=~\"$instance\"}",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "Max",
+ "metric": "jvm_memory_direct_bytes_max",
+ "refId": "B",
+ "step": 20
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeShift": null,
+ "title": "Direct Memory",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "decbytes",
+ "label": "",
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": "",
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ }
+ ]
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": null,
+ "fill": 1,
+ "id": 16,
+ "legend": {
+ "avg": false,
+ "current": false,
+ "max": false,
+ "min": false,
+ "show": false,
+ "total": false,
+ "values": false
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "nullPointMode": "null",
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "scopedVars": {
+ "instance": {
+ "selected": false,
+ "text": "bookie-h2q4k",
+ "value": "bookie-h2q4k"
+ }
+ },
+ "seriesOverrides": [
+ {
+ "alias": "old count",
+ "yaxis": 2
+ },
+ {
+ "alias": "young count",
+ "yaxis": 2
+ }
+ ],
+ "spaceLength": 10,
+ "span": 3,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "sum(increase(jvm_gc_collection_seconds_sum{cluster=\"$cluster\", kubernetes_pod_name=~\"$instance\"}[30s]))",
+ "format": "time_series",
+ "hide": false,
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "Pause",
+ "metric": "jvm_gc_collection_seconds_count",
+ "refId": "B",
+ "step": 20
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeShift": null,
+ "title": "GC Pauses",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "s",
+ "label": "",
+ "logBase": 1,
+ "max": null,
+ "min": "0",
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": "",
+ "logBase": 1,
+ "max": null,
+ "min": "0",
+ "show": false
+ }
+ ]
+ }
+ ],
+ "repeat": null,
+ "repeatIteration": 1496162786419,
+ "repeatRowId": 1,
+ "showTitle": true,
+ "title": "$component $instance",
+ "titleSize": "h4"
+ },
+ {
+ "collapse": false,
+ "height": 221,
+ "panels": [
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": null,
+ "decimals": 1,
+ "fill": 1,
+ "id": 17,
+ "legend": {
+ "avg": false,
+ "current": false,
+ "max": false,
+ "min": false,
+ "show": false,
+ "total": false,
+ "values": false
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "nullPointMode": "null",
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "scopedVars": {
+ "instance": {
+ "selected": false,
+ "text": "bookie-m2bp9",
+ "value": "bookie-m2bp9"
+ }
+ },
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "span": 3,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "irate(process_cpu_seconds_total{cluster=\"$cluster\", kubernetes_pod_name=~\"$instance\"}[30s]) * 100\n\n",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "{{kubernetes_pod_name}}",
+ "metric": "process_cpu_seconds_total",
+ "refId": "A",
+ "step": 20
+ }
+ ],
+ "thresholds": [
+ {
+ "colorMode": "ok",
+ "fill": false,
+ "line": true,
+ "op": "gt",
+ "value": 100
+ }
+ ],
+ "timeFrom": null,
+ "timeShift": null,
+ "title": "CPU",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "percent",
+ "label": "",
+ "logBase": 1,
+ "max": null,
+ "min": "0",
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ }
+ ]
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "Prometheus",
+ "fill": 1,
+ "id": 18,
+ "legend": {
+ "alignAsTable": false,
+ "avg": false,
+ "current": false,
+ "hideEmpty": false,
+ "max": false,
+ "min": false,
+ "rightSide": false,
+ "show": false,
+ "total": false,
+ "values": false
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "nullPointMode": "null",
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "scopedVars": {
+ "instance": {
+ "selected": false,
+ "text": "bookie-m2bp9",
+ "value": "bookie-m2bp9"
+ }
+ },
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "span": 3,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "sum(jvm_memory_bytes_committed{cluster=\"$cluster\", kubernetes_pod_name=~\"$instance\"})",
+ "format": "time_series",
+ "hide": false,
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "Heap size",
+ "metric": "jvm_memory_bytes_committed",
+ "refId": "A",
+ "step": 20
+ },
+ {
+ "expr": "sum(jvm_memory_bytes_used{cluster=\"$cluster\", kubernetes_pod_name=~\"$instance\"})",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "Used",
+ "metric": "jvm_memory_bytes_used",
+ "refId": "B",
+ "step": 20
+ },
+ {
+ "expr": "sum(jvm_memory_bytes_max{cluster=\"$cluster\", kubernetes_pod_name=~\"$instance\"})",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "Max",
+ "metric": "jvm_memory_bytes_max",
+ "refId": "C",
+ "step": 20
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeShift": null,
+ "title": "Heap Memory",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "decbytes",
+ "label": "",
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ]
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": null,
+ "fill": 1,
+ "id": 19,
+ "legend": {
+ "avg": false,
+ "current": false,
+ "max": false,
+ "min": false,
+ "show": false,
+ "total": false,
+ "values": false
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "nullPointMode": "null",
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "scopedVars": {
+ "instance": {
+ "selected": false,
+ "text": "bookie-m2bp9",
+ "value": "bookie-m2bp9"
+ }
+ },
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "span": 3,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "jvm_memory_direct_bytes_used{cluster=\"$cluster\", kubernetes_pod_name=~\"$instance\"}",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "Used",
+ "metric": "jvm_memory_direct_bytes_used",
+ "refId": "A",
+ "step": 20
+ },
+ {
+ "expr": "jvm_memory_direct_bytes_max{cluster=\"$cluster\", kubernetes_pod_name=~\"$instance\"}",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "Max",
+ "metric": "jvm_memory_direct_bytes_max",
+ "refId": "B",
+ "step": 20
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeShift": null,
+ "title": "Direct Memory",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "decbytes",
+ "label": "",
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": "",
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ }
+ ]
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": null,
+ "fill": 1,
+ "id": 20,
+ "legend": {
+ "avg": false,
+ "current": false,
+ "max": false,
+ "min": false,
+ "show": false,
+ "total": false,
+ "values": false
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "nullPointMode": "null",
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "scopedVars": {
+ "instance": {
+ "selected": false,
+ "text": "bookie-m2bp9",
+ "value": "bookie-m2bp9"
+ }
+ },
+ "seriesOverrides": [
+ {
+ "alias": "old count",
+ "yaxis": 2
+ },
+ {
+ "alias": "young count",
+ "yaxis": 2
+ }
+ ],
+ "spaceLength": 10,
+ "span": 3,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "sum(increase(jvm_gc_collection_seconds_sum{cluster=\"$cluster\", kubernetes_pod_name=~\"$instance\"}[30s]))",
+ "format": "time_series",
+ "hide": false,
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "Pause",
+ "metric": "jvm_gc_collection_seconds_count",
+ "refId": "B",
+ "step": 20
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeShift": null,
+ "title": "GC Pauses",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "s",
+ "label": "",
+ "logBase": 1,
+ "max": null,
+ "min": "0",
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": "",
+ "logBase": 1,
+ "max": null,
+ "min": "0",
+ "show": false
+ }
+ ]
+ }
+ ],
+ "repeat": null,
+ "repeatIteration": 1496162786419,
+ "repeatRowId": 1,
+ "showTitle": true,
+ "title": "$component $instance",
+ "titleSize": "h4"
+ },
+ {
+ "collapse": false,
+ "height": 221,
+ "panels": [
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": null,
+ "decimals": 1,
+ "fill": 1,
+ "id": 21,
+ "legend": {
+ "avg": false,
+ "current": false,
+ "max": false,
+ "min": false,
+ "show": false,
+ "total": false,
+ "values": false
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "nullPointMode": "null",
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "scopedVars": {
+ "instance": {
+ "selected": false,
+ "text": "bookie-z6djd",
+ "value": "bookie-z6djd"
+ }
+ },
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "span": 3,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "irate(process_cpu_seconds_total{cluster=\"$cluster\", kubernetes_pod_name=~\"$instance\"}[30s]) * 100\n\n",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "{{kubernetes_pod_name}}",
+ "metric": "process_cpu_seconds_total",
+ "refId": "A",
+ "step": 20
+ }
+ ],
+ "thresholds": [
+ {
+ "colorMode": "ok",
+ "fill": false,
+ "line": true,
+ "op": "gt",
+ "value": 100
+ }
+ ],
+ "timeFrom": null,
+ "timeShift": null,
+ "title": "CPU",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "percent",
+ "label": "",
+ "logBase": 1,
+ "max": null,
+ "min": "0",
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ }
+ ]
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "Prometheus",
+ "fill": 1,
+ "id": 22,
+ "legend": {
+ "alignAsTable": false,
+ "avg": false,
+ "current": false,
+ "hideEmpty": false,
+ "max": false,
+ "min": false,
+ "rightSide": false,
+ "show": false,
+ "total": false,
+ "values": false
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "nullPointMode": "null",
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "scopedVars": {
+ "instance": {
+ "selected": false,
+ "text": "bookie-z6djd",
+ "value": "bookie-z6djd"
+ }
+ },
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "span": 3,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "sum(jvm_memory_bytes_committed{cluster=\"$cluster\", kubernetes_pod_name=~\"$instance\"})",
+ "format": "time_series",
+ "hide": false,
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "Heap size",
+ "metric": "jvm_memory_bytes_committed",
+ "refId": "A",
+ "step": 20
+ },
+ {
+ "expr": "sum(jvm_memory_bytes_used{cluster=\"$cluster\", kubernetes_pod_name=~\"$instance\"})",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "Used",
+ "metric": "jvm_memory_bytes_used",
+ "refId": "B",
+ "step": 20
+ },
+ {
+ "expr": "sum(jvm_memory_bytes_max{cluster=\"$cluster\", kubernetes_pod_name=~\"$instance\"})",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "Max",
+ "metric": "jvm_memory_bytes_max",
+ "refId": "C",
+ "step": 20
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeShift": null,
+ "title": "Heap Memory",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "decbytes",
+ "label": "",
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ]
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": null,
+ "fill": 1,
+ "id": 23,
+ "legend": {
+ "avg": false,
+ "current": false,
+ "max": false,
+ "min": false,
+ "show": false,
+ "total": false,
+ "values": false
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "nullPointMode": "null",
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "scopedVars": {
+ "instance": {
+ "selected": false,
+ "text": "bookie-z6djd",
+ "value": "bookie-z6djd"
+ }
+ },
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "span": 3,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "jvm_memory_direct_bytes_used{cluster=\"$cluster\", kubernetes_pod_name=~\"$instance\"}",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "Used",
+ "metric": "jvm_memory_direct_bytes_used",
+ "refId": "A",
+ "step": 20
+ },
+ {
+ "expr": "jvm_memory_direct_bytes_max{cluster=\"$cluster\", kubernetes_pod_name=~\"$instance\"}",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "Max",
+ "metric": "jvm_memory_direct_bytes_max",
+ "refId": "B",
+ "step": 20
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeShift": null,
+ "title": "Direct Memory",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "decbytes",
+ "label": "",
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": "",
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ }
+ ]
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": null,
+ "fill": 1,
+ "id": 24,
+ "legend": {
+ "avg": false,
+ "current": false,
+ "max": false,
+ "min": false,
+ "show": false,
+ "total": false,
+ "values": false
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "nullPointMode": "null",
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "scopedVars": {
+ "instance": {
+ "selected": false,
+ "text": "bookie-z6djd",
+ "value": "bookie-z6djd"
+ }
+ },
+ "seriesOverrides": [
+ {
+ "alias": "old count",
+ "yaxis": 2
+ },
+ {
+ "alias": "young count",
+ "yaxis": 2
+ }
+ ],
+ "spaceLength": 10,
+ "span": 3,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "sum(increase(jvm_gc_collection_seconds_sum{cluster=\"$cluster\", kubernetes_pod_name=~\"$instance\"}[30s]))",
+ "format": "time_series",
+ "hide": false,
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "Pause",
+ "metric": "jvm_gc_collection_seconds_count",
+ "refId": "B",
+ "step": 20
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeShift": null,
+ "title": "GC Pauses",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "s",
+ "label": "",
+ "logBase": 1,
+ "max": null,
+ "min": "0",
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": "",
+ "logBase": 1,
+ "max": null,
+ "min": "0",
+ "show": false
+ }
+ ]
+ }
+ ],
+ "repeat": null,
+ "repeatIteration": 1496162786419,
+ "repeatRowId": 1,
+ "showTitle": true,
+ "title": "$component $instance",
+ "titleSize": "h4"
+ }
+ ],
+ "schemaVersion": 14,
+ "style": "dark",
+ "tags": [
+ "kafka"
+ ],
+ "templating": {
+ "list": [
+ {
+ "allValue": null,
+ "current": {
+ "text": "us-central",
+ "value": "us-central"
+ },
+ "datasource": "Prometheus",
+ "hide": 0,
+ "includeAll": false,
+ "label": "Cluster",
+ "multi": false,
+ "name": "cluster",
+ "options": [],
+ "query": "{cluster=~\".+\"}",
+ "refresh": 1,
+ "regex": "/.*[^_]cluster=\\\"([^\\\"]+)\\\".*/",
+ "sort": 0,
+ "tagValuesQuery": "",
+ "tags": [],
+ "tagsQuery": "",
+ "type": "query",
+ "useTags": false
+ },
+ {
+ "allValue": null,
+ "current": {
+ "tags": [],
+ "text": "bookkeeper",
+ "value": "bookkeeper"
+ },
+ "datasource": "Prometheus",
+ "hide": 0,
+ "includeAll": false,
+ "label": "Component",
+ "multi": false,
+ "name": "component",
+ "options": [],
+ "query": "{job=~\".+\"}",
+ "refresh": 1,
+ "regex": "/.*[^_]job=\\\"([^\\\"]+)\\\".*/",
+ "sort": 1,
+ "tagValuesQuery": "",
+ "tags": [],
+ "tagsQuery": "",
+ "type": "query",
+ "useTags": false
+ },
+ {
+ "allValue": null,
+ "current": {
+ "text": "All",
+ "value": [
+ "$__all"
+ ]
+ },
+ "datasource": "Prometheus",
+ "hide": 0,
+ "includeAll": true,
+ "label": "Pod",
+ "multi": true,
+ "name": "instance",
+ "options": [],
+ "query": "jvm_memory_bytes_used{cluster=~\"$cluster\", job=~\"$component\", kubernetes_pod_name=~\".+\"}",
+ "refresh": 2,
+ "regex": "/.*[^_]kubernetes_pod_name=\\\"([^\\\"]+)\\\".*/",
+ "sort": 1,
+ "tagValuesQuery": "",
+ "tags": [],
+ "tagsQuery": "",
+ "type": "query",
+ "useTags": false
+ }
+ ]
+ },
+ "time": {
+ "from": "now-1h",
+ "to": "now"
+ },
+ "timepicker": {
+ "refresh_intervals": [
+ "5s",
+ "10s",
+ "30s",
+ "1m",
+ "5m",
+ "15m",
+ "30m",
+ "1h",
+ "2h",
+ "1d"
+ ],
+ "time_options": [
+ "5m",
+ "15m",
+ "1h",
+ "6h",
+ "12h",
+ "24h",
+ "2d",
+ "7d",
+ "30d"
+ ]
+ },
+ "timezone": "browser",
+ "title": "Kafka - JVM",
+ "version": 27
+}
\ No newline at end of file
diff --git a/driver-msk/deploy/monitoring/dashboards/perf.json b/driver-msk/deploy/monitoring/dashboards/perf.json
new file mode 100644
index 00000000..04e503da
--- /dev/null
+++ b/driver-msk/deploy/monitoring/dashboards/perf.json
@@ -0,0 +1,878 @@
+{
+ "annotations": {
+ "list": [
+ {
+ "builtIn": 1,
+ "datasource": "-- Grafana --",
+ "enable": true,
+ "hide": true,
+ "iconColor": "rgba(0, 211, 255, 1)",
+ "name": "Annotations & Alerts",
+ "type": "dashboard"
+ }
+ ]
+ },
+ "editable": true,
+ "gnetId": null,
+ "graphTooltip": 0,
+ "id": 6,
+ "links": [],
+ "panels": [
+ {
+ "aliasColors": {
+ "Broker - Read": "#66cc69",
+ "Broker - Write": "#00afba",
+ "Consume rate": "blue",
+ "DD - Max throughput": "#d8365d",
+ "Produce rate": "#d8365d",
+ "{device=\"nvme1n1\", group=\"broker-os\", instance=\"10.0.0.207:9100\", job=\"kafka\"}": "green",
+ "{device=\"nvme1n1\", group=\"broker-os\", instance=\"10.0.0.212:9100\", job=\"kafka\"}": "yellow",
+ "{device=\"nvme1n1\", group=\"broker-os\", instance=\"10.0.0.5:9100\", job=\"kafka\"}": "red",
+ "{device=\"nvme2n1\", group=\"broker-os\", instance=\"10.0.0.207:9100\", job=\"kafka\"}": "blue",
+ "{device=\"nvme2n1\", group=\"broker-os\", instance=\"10.0.0.212:9100\", job=\"kafka\"}": "orange",
+ "{device=\"nvme2n1\", group=\"broker-os\", instance=\"10.0.0.5:9100\", job=\"kafka\"}": "purple"
+ },
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "Prometheus",
+ "fieldConfig": {
+ "defaults": {
+ "custom": {
+ "align": null
+ },
+ "mappings": [],
+ "thresholds": {
+ "mode": "absolute",
+ "steps": [
+ {
+ "color": "green",
+ "value": null
+ },
+ {
+ "color": "red",
+ "value": 80
+ }
+ ]
+ }
+ },
+ "overrides": []
+ },
+ "fill": 5,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 11,
+ "w": 12,
+ "x": 0,
+ "y": 0
+ },
+ "hiddenSeries": false,
+ "id": 12,
+ "legend": {
+ "alignAsTable": false,
+ "avg": false,
+ "current": false,
+ "max": false,
+ "min": false,
+ "show": true,
+ "total": false,
+ "values": false
+ },
+ "lines": false,
+ "linewidth": 1,
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pluginVersion": "7.0.5",
+ "pointradius": 2,
+ "points": true,
+ "renderer": "flot",
+ "seriesOverrides": [
+ {
+ "alias": "Consume rate",
+ "dashes": true,
+ "fill": 0,
+ "linewidth": 2,
+ "yaxis": 1
+ },
+ {
+ "alias": "Produce rate",
+ "dashes": true,
+ "fill": 0,
+ "linewidth": 2,
+ "yaxis": 1
+ },
+ {
+ "alias": "DD - Max throughput",
+ "fill": 0,
+ "lines": true,
+ "linewidth": 3,
+ "points": false
+ }
+ ],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "sum by (instance) (rate(node_disk_written_bytes_total{device=~\"nvme1n1|nvme2n1\"}[1m]))",
+ "hide": false,
+ "interval": "",
+ "intervalFactor": 1,
+ "legendFormat": "Broker - {{instance}} - Write",
+ "refId": "A"
+ },
+ {
+ "expr": "sum(rate(node_disk_read_bytes_total{device=~\"nvme1n1|nvme2n1\"}[1m]))",
+ "hide": true,
+ "interval": "",
+ "legendFormat": "Broker - Read",
+ "refId": "B"
+ },
+ {
+ "expr": "sum(rate(benchmark_producer_bytes_sent[1m]))",
+ "hide": false,
+ "interval": "",
+ "legendFormat": "Produce rate",
+ "refId": "C"
+ },
+ {
+ "expr": "sum(rate(benchmark_consumer_bytes_recv[1m]))",
+ "hide": false,
+ "interval": "",
+ "legendFormat": "Consume rate",
+ "refId": "D"
+ },
+ {
+ "expr": "655000000",
+ "interval": "",
+ "legendFormat": "DD - Max throughput",
+ "refId": "E"
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "I/O throughput + Producer/Consume Throughput",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "Bps",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ },
+ {
+ "format": "Bps",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {
+ "{device=\"nvme1n1\", group=\"broker-os\", instance=\"10.0.0.207:9100\", job=\"kafka\"}": "green",
+ "{device=\"nvme1n1\", group=\"broker-os\", instance=\"10.0.0.212:9100\", job=\"kafka\"}": "yellow",
+ "{device=\"nvme1n1\", group=\"broker-os\", instance=\"10.0.0.5:9100\", job=\"kafka\"}": "red",
+ "{device=\"nvme2n1\", group=\"broker-os\", instance=\"10.0.0.207:9100\", job=\"kafka\"}": "blue",
+ "{device=\"nvme2n1\", group=\"broker-os\", instance=\"10.0.0.212:9100\", job=\"kafka\"}": "orange",
+ "{device=\"nvme2n1\", group=\"broker-os\", instance=\"10.0.0.5:9100\", job=\"kafka\"}": "purple"
+ },
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "Prometheus",
+ "fieldConfig": {
+ "defaults": {
+ "custom": {}
+ },
+ "overrides": []
+ },
+ "fill": 1,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 9,
+ "w": 12,
+ "x": 12,
+ "y": 0
+ },
+ "hiddenSeries": false,
+ "id": 6,
+ "legend": {
+ "avg": false,
+ "current": false,
+ "max": false,
+ "min": false,
+ "show": true,
+ "total": false,
+ "values": false
+ },
+ "lines": true,
+ "linewidth": 1,
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 2,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "sum by (instance) (delta(node_disk_written_bytes_total{device=~\"nvme1n1|nvme2n1\"}[1m])/60)",
+ "interval": "",
+ "intervalFactor": 1,
+ "legendFormat": "Broker - {{instance}}",
+ "refId": "A"
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "I/O throughput",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "Bps",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": null,
+ "fieldConfig": {
+ "defaults": {
+ "custom": {}
+ },
+ "overrides": []
+ },
+ "fill": 1,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 8,
+ "w": 12,
+ "x": 12,
+ "y": 9
+ },
+ "hiddenSeries": false,
+ "id": 5,
+ "legend": {
+ "avg": false,
+ "current": false,
+ "hideEmpty": true,
+ "hideZero": true,
+ "max": false,
+ "min": false,
+ "show": false,
+ "total": false,
+ "values": false
+ },
+ "lines": true,
+ "linewidth": 1,
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 2,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "sum(delta(benchmark_producer_bytes_sent[1m])/60)",
+ "interval": "",
+ "legendFormat": "",
+ "refId": "A"
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "Publish throughput (MB/sec)",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "Bps",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {
+ "{device=\"nvme1n1\", group=\"broker-os\", instance=\"10.0.0.207:9100\", job=\"kafka\"}": "green",
+ "{device=\"nvme1n1\", group=\"broker-os\", instance=\"10.0.0.212:9100\", job=\"kafka\"}": "yellow",
+ "{device=\"nvme1n1\", group=\"broker-os\", instance=\"10.0.0.5:9100\", job=\"kafka\"}": "red",
+ "{device=\"nvme2n1\", group=\"broker-os\", instance=\"10.0.0.207:9100\", job=\"kafka\"}": "blue",
+ "{device=\"nvme2n1\", group=\"broker-os\", instance=\"10.0.0.212:9100\", job=\"kafka\"}": "orange",
+ "{device=\"nvme2n1\", group=\"broker-os\", instance=\"10.0.0.5:9100\", job=\"kafka\"}": "purple"
+ },
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "Prometheus",
+ "fieldConfig": {
+ "defaults": {
+ "custom": {}
+ },
+ "overrides": []
+ },
+ "fill": 1,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 9,
+ "w": 12,
+ "x": 0,
+ "y": 11
+ },
+ "hiddenSeries": false,
+ "id": 2,
+ "legend": {
+ "avg": false,
+ "current": false,
+ "hideEmpty": true,
+ "hideZero": true,
+ "max": false,
+ "min": false,
+ "show": true,
+ "total": false,
+ "values": false
+ },
+ "lines": true,
+ "linewidth": 1,
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 2,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "delta(node_disk_io_time_seconds_total{device=~\"nvme1n1|nvme2n1\"}[1m])/60",
+ "interval": "",
+ "intervalFactor": 1,
+ "legendFormat": "Broker - {{instance}} ➝ Disk - {{device}}",
+ "refId": "A"
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "I/O Utilization",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "percentunit",
+ "label": null,
+ "logBase": 1,
+ "max": "1",
+ "min": "0",
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": null,
+ "fieldConfig": {
+ "defaults": {
+ "custom": {}
+ },
+ "overrides": []
+ },
+ "fill": 1,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 8,
+ "w": 12,
+ "x": 0,
+ "y": 20
+ },
+ "hiddenSeries": false,
+ "id": 4,
+ "legend": {
+ "avg": false,
+ "current": false,
+ "hideEmpty": true,
+ "hideZero": true,
+ "max": false,
+ "min": false,
+ "show": false,
+ "total": false,
+ "values": false
+ },
+ "lines": true,
+ "linewidth": 1,
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 2,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "sum(delta(benchmark_producer_messages_sent[1m])/60)",
+ "interval": "",
+ "legendFormat": "",
+ "refId": "A"
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "Publish rate (messages/sec)",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "Prometheus",
+ "fieldConfig": {
+ "defaults": {
+ "custom": {}
+ },
+ "overrides": []
+ },
+ "fill": 1,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 8,
+ "w": 12,
+ "x": 0,
+ "y": 28
+ },
+ "hiddenSeries": false,
+ "id": 10,
+ "legend": {
+ "avg": false,
+ "current": false,
+ "hideEmpty": true,
+ "hideZero": true,
+ "max": false,
+ "min": false,
+ "show": true,
+ "total": false,
+ "values": false
+ },
+ "lines": true,
+ "linewidth": 1,
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 2,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "rate(kafka_log_logflushstats_logflushrateandtimems{quantile=\"0.99\"}[1m])",
+ "interval": "",
+ "legendFormat": "Broker - {{instance}}",
+ "refId": "A"
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "Log Flush Time (p99)",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "ms",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {
+ "Idle - Waiting for something to happen": "#052B51",
+ "Iowait - Waiting for I/O to complete": "blue",
+ "Nice - Niced processes executing in user mode": "yellow",
+ "Softirq - Servicing softirqs": "yellow",
+ "System - Processes executing in kernel mode": "dark-purple",
+ "User - Normal processes executing in user mode": "red",
+ "guest": "#9AC48A",
+ "idle": "#052B51",
+ "iowait": "#EAB839",
+ "irq": "#BF1B00",
+ "nice": "#C15C17",
+ "softirq": "#E24D42",
+ "steal": "#FCE2DE",
+ "system": "#508642",
+ "user": "#5195CE"
+ },
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": null,
+ "decimals": 2,
+ "description": "",
+ "fieldConfig": {
+ "defaults": {
+ "custom": {}
+ },
+ "overrides": []
+ },
+ "fill": 4,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 13,
+ "w": 24,
+ "x": 0,
+ "y": 36
+ },
+ "hiddenSeries": false,
+ "id": 8,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "max": true,
+ "min": true,
+ "rightSide": false,
+ "show": true,
+ "sideWidth": 250,
+ "sort": null,
+ "sortDesc": null,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "maxPerRow": 6,
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": true,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "repeat": null,
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "stack": true,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "sum by (mode)(irate(node_cpu_seconds_total{mode=\"user\"}[5m])) * 100",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "User - Normal processes executing in user mode",
+ "refId": "B",
+ "step": 240
+ },
+ {
+ "expr": "sum by (mode)(irate(node_cpu_seconds_total{mode=\"system\"}[5m])) * 100",
+ "format": "time_series",
+ "interval": "10s",
+ "intervalFactor": 2,
+ "legendFormat": "System - Processes executing in kernel mode",
+ "refId": "A",
+ "step": 20
+ },
+ {
+ "expr": "sum by (mode)(irate(node_cpu_seconds_total{mode=\"iowait\"}[5m])) * 100",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "Iowait - Waiting for I/O to complete",
+ "refId": "E",
+ "step": 240
+ },
+ {
+ "expr": "sum by (mode)(irate(node_cpu_seconds_total{mode=\"softirq\"}[5m])) * 100",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "Softirq - Servicing softirqs",
+ "refId": "G",
+ "step": 240
+ },
+ {
+ "expr": "sum by (mode)(irate(node_cpu_seconds_total{mode=\"idle\"}[5m])) * 100",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "Idle - Waiting for something to happen",
+ "refId": "D",
+ "step": 240
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "CPU Usage across all Brokers",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "short",
+ "label": "percentage",
+ "logBase": 1,
+ "max": "100",
+ "min": "0",
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ }
+ ],
+ "refresh": false,
+ "schemaVersion": 25,
+ "style": "dark",
+ "tags": [],
+ "templating": {
+ "list": []
+ },
+ "time": {
+ "from": "now-6h",
+ "to": "now"
+ },
+ "timepicker": {
+ "refresh_intervals": [
+ "10s",
+ "30s",
+ "1m",
+ "5m",
+ "15m",
+ "30m",
+ "1h",
+ "2h",
+ "1d"
+ ]
+ },
+ "timezone": "utc",
+ "title": "Performance Metrics - Kafka",
+ "uid": "vYWrwI7Mkafka",
+ "version": 5
+}
\ No newline at end of file
diff --git a/driver-msk/deploy/monitoring/dashboards/prometheus.json b/driver-msk/deploy/monitoring/dashboards/prometheus.json
new file mode 100644
index 00000000..40ebcf83
--- /dev/null
+++ b/driver-msk/deploy/monitoring/dashboards/prometheus.json
@@ -0,0 +1,719 @@
+{
+ "annotations": {
+ "list": []
+ },
+ "editable": true,
+ "gnetId": null,
+ "graphTooltip": 0,
+ "hideControls": true,
+ "id": 1,
+ "links": [
+ {
+ "icon": "info",
+ "tags": [],
+ "targetBlank": true,
+ "title": "Grafana Docs",
+ "tooltip": "",
+ "type": "link",
+ "url": "http://docs.grafana.org/"
+ },
+ {
+ "icon": "info",
+ "tags": [],
+ "targetBlank": true,
+ "title": "Prometheus Docs",
+ "type": "link",
+ "url": "http://prometheus.io/docs/introduction/overview/"
+ }
+ ],
+ "refresh": false,
+ "revision": "1.0",
+ "rows": [
+ {
+ "collapse": false,
+ "height": 178,
+ "panels": [
+ {
+ "cacheTimeout": null,
+ "colorBackground": false,
+ "colorValue": false,
+ "colors": [
+ "rgba(245, 54, 54, 0.9)",
+ "rgba(237, 129, 40, 0.89)",
+ "rgba(50, 172, 45, 0.97)"
+ ],
+ "datasource": "Prometheus",
+ "decimals": 1,
+ "editable": true,
+ "error": false,
+ "format": "s",
+ "gauge": {
+ "maxValue": 100,
+ "minValue": 0,
+ "show": false,
+ "thresholdLabels": false,
+ "thresholdMarkers": true
+ },
+ "id": 5,
+ "interval": null,
+ "links": [],
+ "mappingType": 1,
+ "mappingTypes": [
+ {
+ "name": "value to text",
+ "value": 1
+ },
+ {
+ "name": "range to text",
+ "value": 2
+ }
+ ],
+ "maxDataPoints": 100,
+ "nullPointMode": "connected",
+ "nullText": null,
+ "postfix": "",
+ "postfixFontSize": "50%",
+ "prefix": "",
+ "prefixFontSize": "50%",
+ "rangeMaps": [
+ {
+ "from": "null",
+ "text": "N/A",
+ "to": "null"
+ }
+ ],
+ "span": 3,
+ "sparkline": {
+ "fillColor": "rgba(31, 118, 189, 0.18)",
+ "full": false,
+ "lineColor": "rgb(31, 120, 193)",
+ "show": false
+ },
+ "tableColumn": "",
+ "targets": [
+ {
+ "expr": "(time() - process_start_time_seconds{job=\"prometheus\"})",
+ "intervalFactor": 2,
+ "refId": "A",
+ "step": 4
+ }
+ ],
+ "thresholds": "",
+ "title": "Uptime",
+ "type": "singlestat",
+ "valueFontSize": "80%",
+ "valueMaps": [
+ {
+ "op": "=",
+ "text": "N/A",
+ "value": "null"
+ }
+ ],
+ "valueName": "current"
+ },
+ {
+ "cacheTimeout": null,
+ "colorBackground": false,
+ "colorValue": false,
+ "colors": [
+ "rgba(50, 172, 45, 0.97)",
+ "rgba(237, 129, 40, 0.89)",
+ "rgba(245, 54, 54, 0.9)"
+ ],
+ "datasource": "Prometheus",
+ "editable": true,
+ "error": false,
+ "format": "none",
+ "gauge": {
+ "maxValue": 100,
+ "minValue": 0,
+ "show": false,
+ "thresholdLabels": false,
+ "thresholdMarkers": true
+ },
+ "id": 6,
+ "interval": null,
+ "links": [],
+ "mappingType": 1,
+ "mappingTypes": [
+ {
+ "name": "value to text",
+ "value": 1
+ },
+ {
+ "name": "range to text",
+ "value": 2
+ }
+ ],
+ "maxDataPoints": 100,
+ "nullPointMode": "connected",
+ "nullText": null,
+ "postfix": "",
+ "postfixFontSize": "50%",
+ "prefix": "",
+ "prefixFontSize": "50%",
+ "rangeMaps": [
+ {
+ "from": "null",
+ "text": "N/A",
+ "to": "null"
+ }
+ ],
+ "span": 3,
+ "sparkline": {
+ "fillColor": "rgba(31, 118, 189, 0.18)",
+ "full": false,
+ "lineColor": "rgb(31, 120, 193)",
+ "show": true
+ },
+ "tableColumn": "",
+ "targets": [
+ {
+ "expr": "prometheus_local_storage_memory_series",
+ "intervalFactor": 2,
+ "refId": "A",
+ "step": 4
+ }
+ ],
+ "thresholds": "1,5",
+ "title": "Local Storage Memory Series",
+ "type": "singlestat",
+ "valueFontSize": "70%",
+ "valueMaps": [],
+ "valueName": "current"
+ },
+ {
+ "cacheTimeout": null,
+ "colorBackground": false,
+ "colorValue": true,
+ "colors": [
+ "rgba(50, 172, 45, 0.97)",
+ "rgba(237, 129, 40, 0.89)",
+ "rgba(245, 54, 54, 0.9)"
+ ],
+ "datasource": "Prometheus",
+ "editable": true,
+ "error": false,
+ "format": "none",
+ "gauge": {
+ "maxValue": 100,
+ "minValue": 0,
+ "show": false,
+ "thresholdLabels": false,
+ "thresholdMarkers": true
+ },
+ "id": 7,
+ "interval": null,
+ "links": [],
+ "mappingType": 1,
+ "mappingTypes": [
+ {
+ "name": "value to text",
+ "value": 1
+ },
+ {
+ "name": "range to text",
+ "value": 2
+ }
+ ],
+ "maxDataPoints": 100,
+ "nullPointMode": "connected",
+ "nullText": null,
+ "postfix": "",
+ "postfixFontSize": "50%",
+ "prefix": "",
+ "prefixFontSize": "50%",
+ "rangeMaps": [
+ {
+ "from": "null",
+ "text": "N/A",
+ "to": "null"
+ }
+ ],
+ "span": 3,
+ "sparkline": {
+ "fillColor": "rgba(31, 118, 189, 0.18)",
+ "full": false,
+ "lineColor": "rgb(31, 120, 193)",
+ "show": true
+ },
+ "tableColumn": "",
+ "targets": [
+ {
+ "expr": "prometheus_local_storage_indexing_queue_length",
+ "intervalFactor": 2,
+ "refId": "A",
+ "step": 4
+ }
+ ],
+ "thresholds": "500,4000",
+ "title": "Internal Storage Queue Length",
+ "type": "singlestat",
+ "valueFontSize": "70%",
+ "valueMaps": [
+ {
+ "op": "=",
+ "text": "Empty",
+ "value": "0"
+ }
+ ],
+ "valueName": "current"
+ },
+ {
+ "content": " \nPrometheus \n\nYou're using Prometheus, an open-source systems monitoring and alerting toolkit originally built at SoundCloud. For more information, check out the Grafana and Prometheus projects.
",
+ "editable": true,
+ "error": false,
+ "id": 9,
+ "links": [],
+ "mode": "html",
+ "span": 3,
+ "style": {},
+ "title": "",
+ "transparent": true,
+ "type": "text"
+ }
+ ],
+ "repeat": null,
+ "repeatIteration": null,
+ "repeatRowId": null,
+ "showTitle": false,
+ "title": "New row",
+ "titleSize": "h6"
+ },
+ {
+ "collapse": false,
+ "height": 227,
+ "panels": [
+ {
+ "aliasColors": {
+ "prometheus": "#C15C17",
+ "{instance=\"localhost:9090\",job=\"prometheus\"}": "#C15C17"
+ },
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "Prometheus",
+ "editable": true,
+ "error": false,
+ "fill": 1,
+ "grid": {},
+ "id": 3,
+ "legend": {
+ "avg": false,
+ "current": false,
+ "max": false,
+ "min": false,
+ "show": true,
+ "total": false,
+ "values": false
+ },
+ "lines": true,
+ "linewidth": 2,
+ "links": [],
+ "nullPointMode": "connected",
+ "percentage": false,
+ "pointradius": 2,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "span": 9,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "rate(prometheus_local_storage_ingested_samples_total[5m])",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "{{job}}",
+ "metric": "",
+ "refId": "A",
+ "step": 2
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeShift": null,
+ "title": "Samples ingested (rate-5m)",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "cumulative"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "short",
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ },
+ {
+ "format": "short",
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ }
+ ]
+ },
+ {
+ "content": "#### Samples Ingested\nThis graph displays the count of samples ingested by the Prometheus server, as measured over the last 5 minutes, per time series in the range vector. When troubleshooting an issue on IRC or Github, this is often the first stat requested by the Prometheus team. ",
+ "editable": true,
+ "error": false,
+ "id": 8,
+ "links": [],
+ "mode": "markdown",
+ "span": 2.995914043583536,
+ "style": {},
+ "title": "",
+ "transparent": true,
+ "type": "text"
+ }
+ ],
+ "repeat": null,
+ "repeatIteration": null,
+ "repeatRowId": null,
+ "showTitle": false,
+ "title": "New row",
+ "titleSize": "h6"
+ },
+ {
+ "collapse": false,
+ "height": "250px",
+ "panels": [
+ {
+ "aliasColors": {
+ "prometheus": "#F9BA8F",
+ "{instance=\"localhost:9090\",interval=\"5s\",job=\"prometheus\"}": "#F9BA8F"
+ },
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "Prometheus",
+ "editable": true,
+ "error": false,
+ "fill": 1,
+ "grid": {},
+ "id": 2,
+ "legend": {
+ "avg": false,
+ "current": false,
+ "max": false,
+ "min": false,
+ "show": true,
+ "total": false,
+ "values": false
+ },
+ "lines": true,
+ "linewidth": 2,
+ "links": [],
+ "nullPointMode": "connected",
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "span": 5,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "rate(prometheus_target_interval_length_seconds_count[5m])",
+ "intervalFactor": 2,
+ "legendFormat": "{{job}}",
+ "refId": "A",
+ "step": 2
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeShift": null,
+ "title": "Target Scrapes (last 5m)",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "cumulative"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "short",
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ },
+ {
+ "format": "short",
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ }
+ ]
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "Prometheus",
+ "editable": true,
+ "error": false,
+ "fill": 1,
+ "grid": {},
+ "id": 14,
+ "legend": {
+ "avg": false,
+ "current": false,
+ "max": false,
+ "min": false,
+ "show": true,
+ "total": false,
+ "values": false
+ },
+ "lines": true,
+ "linewidth": 2,
+ "links": [],
+ "nullPointMode": "connected",
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "span": 4,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "prometheus_target_interval_length_seconds{quantile!=\"0.01\", quantile!=\"0.05\"}",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "{{quantile}} ({{interval}})",
+ "metric": "",
+ "refId": "A",
+ "step": 2
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeShift": null,
+ "title": "Scrape Duration",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "cumulative"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "short",
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ },
+ {
+ "format": "short",
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ }
+ ]
+ },
+ {
+ "content": "#### Scrapes\nPrometheus scrapes metrics from instrumented jobs, either directly or via an intermediary push gateway for short-lived jobs. Target scrapes will show how frequently targets are scraped, as measured over the last 5 minutes, per time series in the range vector. Scrape Duration will show how long the scrapes are taking, with percentiles available as series. ",
+ "editable": true,
+ "error": false,
+ "id": 11,
+ "links": [],
+ "mode": "markdown",
+ "span": 3,
+ "style": {},
+ "title": "",
+ "transparent": true,
+ "type": "text"
+ }
+ ],
+ "repeat": null,
+ "repeatIteration": null,
+ "repeatRowId": null,
+ "showTitle": false,
+ "title": "New row",
+ "titleSize": "h6"
+ },
+ {
+ "collapse": false,
+ "height": "250px",
+ "panels": [
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "Prometheus",
+ "decimals": null,
+ "editable": true,
+ "error": false,
+ "fill": 1,
+ "grid": {},
+ "id": 12,
+ "legend": {
+ "alignAsTable": false,
+ "avg": false,
+ "current": false,
+ "hideEmpty": true,
+ "max": false,
+ "min": false,
+ "show": true,
+ "total": false,
+ "values": false
+ },
+ "lines": true,
+ "linewidth": 2,
+ "links": [],
+ "nullPointMode": "connected",
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "span": 9,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "prometheus_evaluator_duration_seconds{quantile!=\"0.01\", quantile!=\"0.05\"}",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "{{quantile}}",
+ "refId": "A",
+ "step": 2
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeShift": null,
+ "title": "Rule Eval Duration",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "cumulative"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "percentunit",
+ "label": "",
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ },
+ {
+ "format": "short",
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ }
+ ]
+ },
+ {
+ "content": "#### Rule Evaluation Duration\nThis graph panel plots the duration for all evaluations to execute. The 50th percentile, 90th percentile and 99th percentile are shown as three separate series to help identify outliers that may be skewing the data.",
+ "editable": true,
+ "error": false,
+ "id": 15,
+ "links": [],
+ "mode": "markdown",
+ "span": 3,
+ "style": {},
+ "title": "",
+ "transparent": true,
+ "type": "text"
+ }
+ ],
+ "repeat": null,
+ "repeatIteration": null,
+ "repeatRowId": null,
+ "showTitle": false,
+ "title": "New row",
+ "titleSize": "h6"
+ }
+ ],
+ "schemaVersion": 14,
+ "style": "dark",
+ "tags": [
+ "prometheus"
+ ],
+ "templating": {
+ "list": []
+ },
+ "time": {
+ "from": "now-5m",
+ "to": "now"
+ },
+ "timepicker": {
+ "now": true,
+ "refresh_intervals": [
+ "5s",
+ "10s",
+ "30s",
+ "1m",
+ "5m",
+ "15m",
+ "30m",
+ "1h",
+ "2h",
+ "1d"
+ ],
+ "time_options": [
+ "5m",
+ "15m",
+ "1h",
+ "6h",
+ "12h",
+ "24h",
+ "2d",
+ "7d",
+ "30d"
+ ]
+ },
+ "timezone": "browser",
+ "title": "Prometheus Stats",
+ "version": 0
+}
\ No newline at end of file
diff --git a/driver-msk/deploy/monitoring/dashboards/pulsar_perf.json b/driver-msk/deploy/monitoring/dashboards/pulsar_perf.json
new file mode 100644
index 00000000..069dcf21
--- /dev/null
+++ b/driver-msk/deploy/monitoring/dashboards/pulsar_perf.json
@@ -0,0 +1,1036 @@
+{
+ "annotations": {
+ "list": [
+ {
+ "builtIn": 1,
+ "datasource": "-- Grafana --",
+ "enable": true,
+ "hide": true,
+ "iconColor": "rgba(0, 211, 255, 1)",
+ "name": "Annotations & Alerts",
+ "type": "dashboard"
+ }
+ ]
+ },
+ "editable": true,
+ "gnetId": null,
+ "graphTooltip": 0,
+ "id": 2,
+ "links": [],
+ "panels": [
+ {
+ "aliasColors": {
+ "Bookie Add Entry Latency (p99)": "#d8365d",
+ "Broker - Read": "#66cc69",
+ "Broker - Write": "#00afba",
+ "Consume rate": "#173361",
+ "DD - Max throughput": "#d8365d",
+ "Produce rate": "#d8365d",
+ "{device=\"nvme1n1\", group=\"broker-os\", instance=\"10.0.0.207:9100\", job=\"kafka\"}": "green",
+ "{device=\"nvme1n1\", group=\"broker-os\", instance=\"10.0.0.212:9100\", job=\"kafka\"}": "yellow",
+ "{device=\"nvme1n1\", group=\"broker-os\", instance=\"10.0.0.5:9100\", job=\"kafka\"}": "red",
+ "{device=\"nvme2n1\", group=\"broker-os\", instance=\"10.0.0.207:9100\", job=\"kafka\"}": "blue",
+ "{device=\"nvme2n1\", group=\"broker-os\", instance=\"10.0.0.212:9100\", job=\"kafka\"}": "orange",
+ "{device=\"nvme2n1\", group=\"broker-os\", instance=\"10.0.0.5:9100\", job=\"kafka\"}": "purple"
+ },
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "Prometheus",
+ "description": "",
+ "fieldConfig": {
+ "defaults": {
+ "custom": {
+ "align": null
+ },
+ "mappings": [],
+ "thresholds": {
+ "mode": "absolute",
+ "steps": [
+ {
+ "color": "green",
+ "value": null
+ },
+ {
+ "color": "red",
+ "value": 80
+ }
+ ]
+ }
+ },
+ "overrides": []
+ },
+ "fill": 5,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 11,
+ "w": 12,
+ "x": 0,
+ "y": 0
+ },
+ "hiddenSeries": false,
+ "id": 12,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": false,
+ "hideEmpty": true,
+ "hideZero": true,
+ "max": true,
+ "min": true,
+ "show": true,
+ "total": false,
+ "values": true
+ },
+ "lines": false,
+ "linewidth": 1,
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pluginVersion": "7.0.5",
+ "pointradius": 2,
+ "points": true,
+ "renderer": "flot",
+ "seriesOverrides": [
+ {
+ "alias": "Consume rate",
+ "dashes": true,
+ "fill": 0,
+ "linewidth": 2,
+ "yaxis": 1
+ },
+ {
+ "alias": "Produce rate",
+ "dashes": true,
+ "fill": 0,
+ "linewidth": 2,
+ "yaxis": 1
+ },
+ {
+ "alias": "DD - Max throughput",
+ "fill": 0,
+ "lines": true,
+ "linewidth": 3,
+ "points": false
+ },
+ {
+ "alias": "Bookie Add Entry Rate",
+ "lines": true
+ },
+ {
+ "alias": "Bookie Add Entry Latency (p99)",
+ "dashes": true,
+ "linewidth": 2,
+ "points": false,
+ "yaxis": 2
+ }
+ ],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "sum by (instance) (rate(node_disk_written_bytes_total{device=~\"nvme1n1|nvme2n1\"}[1m]))",
+ "hide": true,
+ "interval": "",
+ "intervalFactor": 1,
+ "legendFormat": "Broker - {{instance}} - Write",
+ "refId": "A"
+ },
+ {
+ "expr": "sum(rate(node_disk_read_bytes_total{device=~\"nvme1n1|nvme2n1\"}[1m]))",
+ "hide": true,
+ "interval": "",
+ "legendFormat": "Broker - Read",
+ "refId": "B"
+ },
+ {
+ "expr": "sum(pulsar_throughput_in)",
+ "hide": false,
+ "interval": "",
+ "legendFormat": "Produce rate",
+ "refId": "C"
+ },
+ {
+ "expr": "sum(pulsar_throughput_out)",
+ "hide": false,
+ "interval": "",
+ "legendFormat": "Consume rate",
+ "refId": "D"
+ },
+ {
+ "expr": "655000000",
+ "hide": true,
+ "interval": "",
+ "legendFormat": "DD - Max throughput",
+ "refId": "E"
+ },
+ {
+ "expr": "sum(rate(bookie_WRITE_BYTES[30s]))",
+ "hide": false,
+ "interval": "",
+ "legendFormat": "Bookie Add Entry Rate",
+ "refId": "F"
+ },
+ {
+ "expr": "avg(bookkeeper_server_ADD_ENTRY_REQUEST{success=\"true\", quantile=\"0.99\"})",
+ "hide": false,
+ "interval": "",
+ "legendFormat": "Bookie Add Entry Latency (p99)",
+ "refId": "G"
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "I/O throughput + Producer/Consume Throughput",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "Bps",
+ "label": "Throughput",
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ },
+ {
+ "format": "ms",
+ "label": "Latency",
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {
+ "Bookie Add Entry Latency (p99)": "#d8365d",
+ "Broker - Read": "#66cc69",
+ "Broker - Write": "#00afba",
+ "Consume rate": "#173361",
+ "DD - Max throughput": "#d8365d",
+ "Produce rate": "#d8365d",
+ "Value": "#d8365d",
+ "{device=\"nvme1n1\", group=\"broker-os\", instance=\"10.0.0.207:9100\", job=\"kafka\"}": "green",
+ "{device=\"nvme1n1\", group=\"broker-os\", instance=\"10.0.0.212:9100\", job=\"kafka\"}": "yellow",
+ "{device=\"nvme1n1\", group=\"broker-os\", instance=\"10.0.0.5:9100\", job=\"kafka\"}": "red",
+ "{device=\"nvme2n1\", group=\"broker-os\", instance=\"10.0.0.207:9100\", job=\"kafka\"}": "blue",
+ "{device=\"nvme2n1\", group=\"broker-os\", instance=\"10.0.0.212:9100\", job=\"kafka\"}": "orange",
+ "{device=\"nvme2n1\", group=\"broker-os\", instance=\"10.0.0.5:9100\", job=\"kafka\"}": "purple"
+ },
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "Prometheus",
+ "fieldConfig": {
+ "defaults": {
+ "custom": {
+ "align": null
+ },
+ "mappings": [],
+ "thresholds": {
+ "mode": "absolute",
+ "steps": [
+ {
+ "color": "green",
+ "value": null
+ },
+ {
+ "color": "red",
+ "value": 80
+ }
+ ]
+ }
+ },
+ "overrides": []
+ },
+ "fill": 5,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 11,
+ "w": 12,
+ "x": 12,
+ "y": 0
+ },
+ "hiddenSeries": false,
+ "id": 13,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": false,
+ "hideEmpty": true,
+ "hideZero": true,
+ "max": true,
+ "min": true,
+ "show": true,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pluginVersion": "7.0.5",
+ "pointradius": 2,
+ "points": true,
+ "renderer": "flot",
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "bookie_journal_JOURNAL_CB_QUEUE_SIZE",
+ "interval": "",
+ "legendFormat": "Number of entries - {{instance}}",
+ "refId": "A"
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "Journal Callback Queue Length",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "short",
+ "label": "Entries",
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ },
+ {
+ "format": "ms",
+ "label": "Latency",
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {
+ "{device=\"nvme1n1\", group=\"broker-os\", instance=\"10.0.0.207:9100\", job=\"kafka\"}": "green",
+ "{device=\"nvme1n1\", group=\"broker-os\", instance=\"10.0.0.212:9100\", job=\"kafka\"}": "yellow",
+ "{device=\"nvme1n1\", group=\"broker-os\", instance=\"10.0.0.5:9100\", job=\"kafka\"}": "red",
+ "{device=\"nvme2n1\", group=\"broker-os\", instance=\"10.0.0.207:9100\", job=\"kafka\"}": "blue",
+ "{device=\"nvme2n1\", group=\"broker-os\", instance=\"10.0.0.212:9100\", job=\"kafka\"}": "orange",
+ "{device=\"nvme2n1\", group=\"broker-os\", instance=\"10.0.0.5:9100\", job=\"kafka\"}": "purple"
+ },
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "Prometheus",
+ "fieldConfig": {
+ "defaults": {
+ "custom": {}
+ },
+ "overrides": []
+ },
+ "fill": 1,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 9,
+ "w": 12,
+ "x": 0,
+ "y": 11
+ },
+ "hiddenSeries": false,
+ "id": 2,
+ "legend": {
+ "avg": false,
+ "current": false,
+ "hideEmpty": true,
+ "hideZero": true,
+ "max": false,
+ "min": false,
+ "show": true,
+ "total": false,
+ "values": false
+ },
+ "lines": true,
+ "linewidth": 1,
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 2,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "delta(node_disk_io_time_seconds_total{device=~\"nvme1n1|nvme2n1\"}[1m])/60",
+ "interval": "",
+ "intervalFactor": 1,
+ "legendFormat": "Broker - {{instance}} ➝ Disk - {{device}}",
+ "refId": "A"
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "I/O Utilization",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "percentunit",
+ "label": null,
+ "logBase": 1,
+ "max": "1",
+ "min": "0",
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {
+ "{device=\"nvme1n1\", group=\"broker-os\", instance=\"10.0.0.207:9100\", job=\"kafka\"}": "green",
+ "{device=\"nvme1n1\", group=\"broker-os\", instance=\"10.0.0.212:9100\", job=\"kafka\"}": "yellow",
+ "{device=\"nvme1n1\", group=\"broker-os\", instance=\"10.0.0.5:9100\", job=\"kafka\"}": "red",
+ "{device=\"nvme2n1\", group=\"broker-os\", instance=\"10.0.0.207:9100\", job=\"kafka\"}": "blue",
+ "{device=\"nvme2n1\", group=\"broker-os\", instance=\"10.0.0.212:9100\", job=\"kafka\"}": "orange",
+ "{device=\"nvme2n1\", group=\"broker-os\", instance=\"10.0.0.5:9100\", job=\"kafka\"}": "purple"
+ },
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "Prometheus",
+ "fieldConfig": {
+ "defaults": {
+ "custom": {}
+ },
+ "overrides": []
+ },
+ "fill": 1,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 9,
+ "w": 12,
+ "x": 12,
+ "y": 11
+ },
+ "hiddenSeries": false,
+ "id": 6,
+ "legend": {
+ "avg": false,
+ "current": false,
+ "max": false,
+ "min": false,
+ "show": true,
+ "total": false,
+ "values": false
+ },
+ "lines": true,
+ "linewidth": 1,
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 2,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "sum by (instance) (delta(node_disk_written_bytes_total{device=~\"nvme1n1|nvme2n1\"}[1m])/60)",
+ "interval": "",
+ "intervalFactor": 1,
+ "legendFormat": "Broker - {{instance}}",
+ "refId": "A"
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "I/O throughput",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "Bps",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": null,
+ "fieldConfig": {
+ "defaults": {
+ "custom": {}
+ },
+ "overrides": []
+ },
+ "fill": 1,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 8,
+ "w": 12,
+ "x": 0,
+ "y": 20
+ },
+ "hiddenSeries": false,
+ "id": 4,
+ "legend": {
+ "avg": false,
+ "current": false,
+ "hideEmpty": true,
+ "hideZero": true,
+ "max": false,
+ "min": false,
+ "show": false,
+ "total": false,
+ "values": false
+ },
+ "lines": true,
+ "linewidth": 1,
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 2,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "sum(delta(benchmark_producer_messages_sent[1m])/60)",
+ "interval": "",
+ "legendFormat": "",
+ "refId": "A"
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "Publish rate (messages/sec)",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": null,
+ "fieldConfig": {
+ "defaults": {
+ "custom": {}
+ },
+ "overrides": []
+ },
+ "fill": 1,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 8,
+ "w": 12,
+ "x": 12,
+ "y": 20
+ },
+ "hiddenSeries": false,
+ "id": 5,
+ "legend": {
+ "avg": false,
+ "current": false,
+ "hideEmpty": true,
+ "hideZero": true,
+ "max": false,
+ "min": false,
+ "show": false,
+ "total": false,
+ "values": false
+ },
+ "lines": true,
+ "linewidth": 1,
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 2,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "sum(delta(benchmark_producer_bytes_sent[1m])/60)",
+ "interval": "",
+ "legendFormat": "",
+ "refId": "A"
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "Publish throughput (MB/sec)",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "Bps",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "Prometheus",
+ "fieldConfig": {
+ "defaults": {
+ "custom": {}
+ },
+ "overrides": []
+ },
+ "fill": 1,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 8,
+ "w": 12,
+ "x": 0,
+ "y": 28
+ },
+ "hiddenSeries": false,
+ "id": 10,
+ "legend": {
+ "avg": false,
+ "current": false,
+ "hideEmpty": true,
+ "hideZero": true,
+ "max": false,
+ "min": false,
+ "show": true,
+ "total": false,
+ "values": false
+ },
+ "lines": true,
+ "linewidth": 1,
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 2,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "rate(kafka_log_logflushstats_logflushrateandtimems{quantile=\"0.99\"}[1m])",
+ "interval": "",
+ "legendFormat": "Broker - {{instance}}",
+ "refId": "A"
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "Log Flush Time (p99)",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "ms",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {
+ "Idle - Waiting for something to happen": "#052B51",
+ "Iowait - Waiting for I/O to complete": "blue",
+ "Nice - Niced processes executing in user mode": "yellow",
+ "Softirq - Servicing softirqs": "yellow",
+ "System - Processes executing in kernel mode": "dark-purple",
+ "User - Normal processes executing in user mode": "red",
+ "guest": "#9AC48A",
+ "idle": "#052B51",
+ "iowait": "#EAB839",
+ "irq": "#BF1B00",
+ "nice": "#C15C17",
+ "softirq": "#E24D42",
+ "steal": "#FCE2DE",
+ "system": "#508642",
+ "user": "#5195CE"
+ },
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": null,
+ "decimals": 2,
+ "description": "",
+ "fieldConfig": {
+ "defaults": {
+ "custom": {}
+ },
+ "overrides": []
+ },
+ "fill": 4,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 13,
+ "w": 24,
+ "x": 0,
+ "y": 36
+ },
+ "hiddenSeries": false,
+ "id": 8,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "max": true,
+ "min": true,
+ "rightSide": false,
+ "show": true,
+ "sideWidth": 250,
+ "sort": null,
+ "sortDesc": null,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "maxPerRow": 6,
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": true,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "repeat": null,
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "stack": true,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "sum by (mode)(irate(node_cpu_seconds_total{mode=\"user\"}[5m])) * 100",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "User - Normal processes executing in user mode",
+ "refId": "B",
+ "step": 240
+ },
+ {
+ "expr": "sum by (mode)(irate(node_cpu_seconds_total{mode=\"system\"}[5m])) * 100",
+ "format": "time_series",
+ "interval": "10s",
+ "intervalFactor": 2,
+ "legendFormat": "System - Processes executing in kernel mode",
+ "refId": "A",
+ "step": 20
+ },
+ {
+ "expr": "sum by (mode)(irate(node_cpu_seconds_total{mode=\"iowait\"}[5m])) * 100",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "Iowait - Waiting for I/O to complete",
+ "refId": "E",
+ "step": 240
+ },
+ {
+ "expr": "sum by (mode)(irate(node_cpu_seconds_total{mode=\"softirq\"}[5m])) * 100",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "Softirq - Servicing softirqs",
+ "refId": "G",
+ "step": 240
+ },
+ {
+ "expr": "sum by (mode)(irate(node_cpu_seconds_total{mode=\"idle\"}[5m])) * 100",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "Idle - Waiting for something to happen",
+ "refId": "D",
+ "step": 240
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "CPU Usage across all Brokers",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "short",
+ "label": "percentage",
+ "logBase": 1,
+ "max": "100",
+ "min": "0",
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ }
+ ],
+ "refresh": false,
+ "schemaVersion": 25,
+ "style": "dark",
+ "tags": [],
+ "templating": {
+ "list": []
+ },
+ "time": {
+ "from": "now-6h",
+ "to": "now"
+ },
+ "timepicker": {
+ "refresh_intervals": [
+ "10s",
+ "30s",
+ "1m",
+ "5m",
+ "15m",
+ "30m",
+ "1h",
+ "2h",
+ "1d"
+ ]
+ },
+ "timezone": "utc",
+ "title": "Performance Metrics - Pulsar",
+ "uid": "vYWrwI7Mk",
+ "version": 8
+}
\ No newline at end of file
diff --git a/driver-msk/deploy/monitoring/dashboards/rabbit_perf.json b/driver-msk/deploy/monitoring/dashboards/rabbit_perf.json
new file mode 100644
index 00000000..c2e5a017
--- /dev/null
+++ b/driver-msk/deploy/monitoring/dashboards/rabbit_perf.json
@@ -0,0 +1,892 @@
+{
+ "annotations": {
+ "list": [
+ {
+ "builtIn": 1,
+ "datasource": "-- Grafana --",
+ "enable": true,
+ "hide": true,
+ "iconColor": "rgba(0, 211, 255, 1)",
+ "name": "Annotations & Alerts",
+ "type": "dashboard"
+ }
+ ]
+ },
+ "editable": true,
+ "gnetId": null,
+ "graphTooltip": 0,
+ "id": 7,
+ "links": [],
+ "panels": [
+ {
+ "aliasColors": {
+ "Broker - Read": "#66cc69",
+ "Broker - Write": "#00afba",
+ "CPU Usage": "#66cc69",
+ "Consume rate": "#ffc40c",
+ "DD - Max throughput": "#d8365d",
+ "Produce rate": "#173361",
+ "{device=\"nvme1n1\", group=\"broker-os\", instance=\"10.0.0.207:9100\", job=\"kafka\"}": "green",
+ "{device=\"nvme1n1\", group=\"broker-os\", instance=\"10.0.0.212:9100\", job=\"kafka\"}": "yellow",
+ "{device=\"nvme1n1\", group=\"broker-os\", instance=\"10.0.0.5:9100\", job=\"kafka\"}": "red",
+ "{device=\"nvme2n1\", group=\"broker-os\", instance=\"10.0.0.207:9100\", job=\"kafka\"}": "blue",
+ "{device=\"nvme2n1\", group=\"broker-os\", instance=\"10.0.0.212:9100\", job=\"kafka\"}": "orange",
+ "{device=\"nvme2n1\", group=\"broker-os\", instance=\"10.0.0.5:9100\", job=\"kafka\"}": "purple"
+ },
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "Prometheus",
+ "fieldConfig": {
+ "defaults": {
+ "custom": {
+ "align": null
+ },
+ "mappings": [],
+ "thresholds": {
+ "mode": "absolute",
+ "steps": [
+ {
+ "color": "green",
+ "value": null
+ },
+ {
+ "color": "red",
+ "value": 80
+ }
+ ]
+ }
+ },
+ "overrides": []
+ },
+ "fill": 5,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 11,
+ "w": 12,
+ "x": 0,
+ "y": 0
+ },
+ "hiddenSeries": false,
+ "id": 12,
+ "legend": {
+ "alignAsTable": false,
+ "avg": false,
+ "current": false,
+ "max": false,
+ "min": false,
+ "show": true,
+ "total": false,
+ "values": false
+ },
+ "lines": false,
+ "linewidth": 1,
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pluginVersion": "7.0.5",
+ "pointradius": 2,
+ "points": true,
+ "renderer": "flot",
+ "seriesOverrides": [
+ {
+ "alias": "Consume rate",
+ "dashes": true,
+ "fill": 0,
+ "linewidth": 2,
+ "yaxis": 1
+ },
+ {
+ "alias": "Produce rate",
+ "dashes": true,
+ "fill": 0,
+ "linewidth": 2,
+ "yaxis": 1
+ },
+ {
+ "alias": "DD - Max throughput",
+ "fill": 0,
+ "lines": true,
+ "linewidth": 3,
+ "points": false
+ },
+ {
+ "alias": "CPU Usage",
+ "lines": true,
+ "points": false,
+ "yaxis": 2
+ }
+ ],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "sum by (instance) (rate(node_disk_written_bytes_total{device=~\"nvme1n1|nvme2n1\"}[1m]))",
+ "hide": true,
+ "interval": "",
+ "intervalFactor": 1,
+ "legendFormat": "Broker - {{instance}} - Write",
+ "refId": "A"
+ },
+ {
+ "expr": "sum(rate(node_disk_read_bytes_total{device=~\"nvme1n1|nvme2n1\"}[1m]))",
+ "hide": true,
+ "interval": "",
+ "legendFormat": "Broker - Read",
+ "refId": "B"
+ },
+ {
+ "expr": "sum(rate(benchmark_producer_bytes_sent[1m]))",
+ "hide": false,
+ "interval": "",
+ "legendFormat": "Produce rate",
+ "refId": "C"
+ },
+ {
+ "expr": "sum(rate(benchmark_consumer_bytes_recv[1m]))",
+ "hide": false,
+ "interval": "",
+ "legendFormat": "Consume rate",
+ "refId": "D"
+ },
+ {
+ "expr": "655000000",
+ "hide": true,
+ "interval": "",
+ "legendFormat": "DD - Max throughput",
+ "refId": "E"
+ },
+ {
+ "expr": "(((count(count(avg(node_cpu_seconds_total)) by (cpu))) - avg(sum by (mode)(avg(irate(node_cpu_seconds_total{mode='idle'}[5m]))))) * 100) / count(count(avg(node_cpu_seconds_total)) by (cpu))",
+ "interval": "",
+ "legendFormat": "CPU Usage",
+ "refId": "F"
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "Producer/Consume Throughput + CPU Usage",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "Bps",
+ "label": "I/O throughput",
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ },
+ {
+ "format": "percent",
+ "label": "CPU Usage",
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {
+ "{device=\"nvme1n1\", group=\"broker-os\", instance=\"10.0.0.207:9100\", job=\"kafka\"}": "green",
+ "{device=\"nvme1n1\", group=\"broker-os\", instance=\"10.0.0.212:9100\", job=\"kafka\"}": "yellow",
+ "{device=\"nvme1n1\", group=\"broker-os\", instance=\"10.0.0.5:9100\", job=\"kafka\"}": "red",
+ "{device=\"nvme2n1\", group=\"broker-os\", instance=\"10.0.0.207:9100\", job=\"kafka\"}": "blue",
+ "{device=\"nvme2n1\", group=\"broker-os\", instance=\"10.0.0.212:9100\", job=\"kafka\"}": "orange",
+ "{device=\"nvme2n1\", group=\"broker-os\", instance=\"10.0.0.5:9100\", job=\"kafka\"}": "purple"
+ },
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "Prometheus",
+ "fieldConfig": {
+ "defaults": {
+ "custom": {}
+ },
+ "overrides": []
+ },
+ "fill": 1,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 9,
+ "w": 12,
+ "x": 12,
+ "y": 0
+ },
+ "hiddenSeries": false,
+ "id": 6,
+ "legend": {
+ "avg": false,
+ "current": false,
+ "max": false,
+ "min": false,
+ "show": true,
+ "total": false,
+ "values": false
+ },
+ "lines": true,
+ "linewidth": 1,
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 2,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "sum by (instance) (delta(node_disk_written_bytes_total{device=~\"nvme1n1|nvme2n1\"}[1m])/60)",
+ "interval": "",
+ "intervalFactor": 1,
+ "legendFormat": "Broker - {{instance}}",
+ "refId": "A"
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "I/O throughput",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "Bps",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": null,
+ "fieldConfig": {
+ "defaults": {
+ "custom": {}
+ },
+ "overrides": []
+ },
+ "fill": 1,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 8,
+ "w": 12,
+ "x": 12,
+ "y": 9
+ },
+ "hiddenSeries": false,
+ "id": 5,
+ "legend": {
+ "avg": false,
+ "current": false,
+ "hideEmpty": true,
+ "hideZero": true,
+ "max": false,
+ "min": false,
+ "show": false,
+ "total": false,
+ "values": false
+ },
+ "lines": true,
+ "linewidth": 1,
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 2,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "sum(delta(benchmark_producer_bytes_sent[1m])/60)",
+ "interval": "",
+ "legendFormat": "",
+ "refId": "A"
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "Publish throughput (MB/sec)",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "Bps",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {
+ "{device=\"nvme1n1\", group=\"broker-os\", instance=\"10.0.0.207:9100\", job=\"kafka\"}": "green",
+ "{device=\"nvme1n1\", group=\"broker-os\", instance=\"10.0.0.212:9100\", job=\"kafka\"}": "yellow",
+ "{device=\"nvme1n1\", group=\"broker-os\", instance=\"10.0.0.5:9100\", job=\"kafka\"}": "red",
+ "{device=\"nvme2n1\", group=\"broker-os\", instance=\"10.0.0.207:9100\", job=\"kafka\"}": "blue",
+ "{device=\"nvme2n1\", group=\"broker-os\", instance=\"10.0.0.212:9100\", job=\"kafka\"}": "orange",
+ "{device=\"nvme2n1\", group=\"broker-os\", instance=\"10.0.0.5:9100\", job=\"kafka\"}": "purple"
+ },
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "Prometheus",
+ "fieldConfig": {
+ "defaults": {
+ "custom": {}
+ },
+ "overrides": []
+ },
+ "fill": 1,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 9,
+ "w": 12,
+ "x": 0,
+ "y": 11
+ },
+ "hiddenSeries": false,
+ "id": 2,
+ "legend": {
+ "avg": false,
+ "current": false,
+ "hideEmpty": true,
+ "hideZero": true,
+ "max": false,
+ "min": false,
+ "show": true,
+ "total": false,
+ "values": false
+ },
+ "lines": true,
+ "linewidth": 1,
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 2,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "delta(node_disk_io_time_seconds_total{device=~\"nvme1n1|nvme2n1\"}[1m])/60",
+ "interval": "",
+ "intervalFactor": 1,
+ "legendFormat": "Broker - {{instance}} ➝ Disk - {{device}}",
+ "refId": "A"
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "I/O Utilization",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "percentunit",
+ "label": null,
+ "logBase": 1,
+ "max": "1",
+ "min": "0",
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": null,
+ "fieldConfig": {
+ "defaults": {
+ "custom": {}
+ },
+ "overrides": []
+ },
+ "fill": 1,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 8,
+ "w": 12,
+ "x": 0,
+ "y": 20
+ },
+ "hiddenSeries": false,
+ "id": 4,
+ "legend": {
+ "avg": false,
+ "current": false,
+ "hideEmpty": true,
+ "hideZero": true,
+ "max": false,
+ "min": false,
+ "show": false,
+ "total": false,
+ "values": false
+ },
+ "lines": true,
+ "linewidth": 1,
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 2,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "sum(delta(benchmark_producer_messages_sent[1m])/60)",
+ "interval": "",
+ "legendFormat": "",
+ "refId": "A"
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "Publish rate (messages/sec)",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "Prometheus",
+ "fieldConfig": {
+ "defaults": {
+ "custom": {}
+ },
+ "overrides": []
+ },
+ "fill": 1,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 8,
+ "w": 12,
+ "x": 0,
+ "y": 28
+ },
+ "hiddenSeries": false,
+ "id": 10,
+ "legend": {
+ "avg": false,
+ "current": false,
+ "hideEmpty": true,
+ "hideZero": true,
+ "max": false,
+ "min": false,
+ "show": true,
+ "total": false,
+ "values": false
+ },
+ "lines": true,
+ "linewidth": 1,
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 2,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "rate(kafka_log_logflushstats_logflushrateandtimems{quantile=\"0.99\"}[1m])",
+ "interval": "",
+ "legendFormat": "Broker - {{instance}}",
+ "refId": "A"
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "Log Flush Time (p99)",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "ms",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {
+ "Idle - Waiting for something to happen": "#052B51",
+ "Iowait - Waiting for I/O to complete": "blue",
+ "Nice - Niced processes executing in user mode": "yellow",
+ "Softirq - Servicing softirqs": "yellow",
+ "System - Processes executing in kernel mode": "dark-purple",
+ "User - Normal processes executing in user mode": "red",
+ "guest": "#9AC48A",
+ "idle": "#052B51",
+ "iowait": "#EAB839",
+ "irq": "#BF1B00",
+ "nice": "#C15C17",
+ "softirq": "#E24D42",
+ "steal": "#FCE2DE",
+ "system": "#508642",
+ "user": "#5195CE"
+ },
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": null,
+ "decimals": 2,
+ "description": "",
+ "fieldConfig": {
+ "defaults": {
+ "custom": {}
+ },
+ "overrides": []
+ },
+ "fill": 4,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 13,
+ "w": 24,
+ "x": 0,
+ "y": 36
+ },
+ "hiddenSeries": false,
+ "id": 8,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "max": true,
+ "min": true,
+ "rightSide": false,
+ "show": true,
+ "sideWidth": 250,
+ "sort": null,
+ "sortDesc": null,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "maxPerRow": 6,
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": true,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "repeat": null,
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "stack": true,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "sum by (mode)(irate(node_cpu_seconds_total{mode=\"user\"}[5m])) * 100",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "User - Normal processes executing in user mode",
+ "refId": "B",
+ "step": 240
+ },
+ {
+ "expr": "sum by (mode)(irate(node_cpu_seconds_total{mode=\"system\"}[5m])) * 100",
+ "format": "time_series",
+ "interval": "10s",
+ "intervalFactor": 2,
+ "legendFormat": "System - Processes executing in kernel mode",
+ "refId": "A",
+ "step": 20
+ },
+ {
+ "expr": "sum by (mode)(irate(node_cpu_seconds_total{mode=\"iowait\"}[5m])) * 100",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "Iowait - Waiting for I/O to complete",
+ "refId": "E",
+ "step": 240
+ },
+ {
+ "expr": "sum by (mode)(irate(node_cpu_seconds_total{mode=\"softirq\"}[5m])) * 100",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "Softirq - Servicing softirqs",
+ "refId": "G",
+ "step": 240
+ },
+ {
+ "expr": "sum by (mode)(irate(node_cpu_seconds_total{mode=\"idle\"}[5m])) * 100",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "Idle - Waiting for something to happen",
+ "refId": "D",
+ "step": 240
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "CPU Usage across all Brokers",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "short",
+ "label": "percentage",
+ "logBase": 1,
+ "max": "100",
+ "min": "0",
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ }
+ ],
+ "refresh": false,
+ "schemaVersion": 25,
+ "style": "dark",
+ "tags": [],
+ "templating": {
+ "list": []
+ },
+ "time": {
+ "from": "now-6h",
+ "to": "now"
+ },
+ "timepicker": {
+ "refresh_intervals": [
+ "10s",
+ "30s",
+ "1m",
+ "5m",
+ "15m",
+ "30m",
+ "1h",
+ "2h",
+ "1d"
+ ]
+ },
+ "timezone": "utc",
+ "title": "Performance Metrics - RabbitMQ",
+ "uid": "vYWrwI7Mkrabbit",
+ "version": 3
+}
\ No newline at end of file
diff --git a/driver-msk/deploy/monitoring/dashboards/system.json b/driver-msk/deploy/monitoring/dashboards/system.json
new file mode 100644
index 00000000..aa51c95f
--- /dev/null
+++ b/driver-msk/deploy/monitoring/dashboards/system.json
@@ -0,0 +1,14051 @@
+{
+ "annotations": {
+ "list": [
+ {
+ "builtIn": 1,
+ "datasource": "-- Grafana --",
+ "enable": true,
+ "hide": true,
+ "iconColor": "rgba(0, 211, 255, 1)",
+ "name": "Annotations & Alerts",
+ "type": "dashboard"
+ }
+ ]
+ },
+ "editable": true,
+ "gnetId": 1860,
+ "graphTooltip": 0,
+ "id": 2,
+ "iteration": 1597269377076,
+ "links": [],
+ "panels": [
+ {
+ "aliasColors": {
+ "benchmark_producer_produce_latency{group=\"client\", instance=\"10.0.0.23:8081\", job=\"kafka\", quantile=\"0.99\", success=\"true\"}": "blue",
+ "benchmark_producer_produce_latency{group=\"client\", instance=\"10.0.0.61:8081\", job=\"kafka\", quantile=\"0.99\", success=\"true\"}": "yellow"
+ },
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "Prometheus",
+ "fieldConfig": {
+ "defaults": {
+ "custom": {}
+ },
+ "overrides": []
+ },
+ "fill": 1,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 8,
+ "w": 24,
+ "x": 0,
+ "y": 0
+ },
+ "hiddenSeries": false,
+ "id": 317,
+ "legend": {
+ "avg": false,
+ "current": false,
+ "hideEmpty": true,
+ "max": false,
+ "min": false,
+ "show": true,
+ "total": false,
+ "values": false
+ },
+ "lines": false,
+ "linewidth": 1,
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 0.5,
+ "points": true,
+ "renderer": "flot",
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "benchmark_producer_produce_latency{quantile=\"0.99\"}",
+ "interval": "",
+ "legendFormat": "",
+ "refId": "A"
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "Publish latency - p99",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "ms",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "collapsed": true,
+ "datasource": "${DS_PROMETHEUS}",
+ "gridPos": {
+ "h": 1,
+ "w": 24,
+ "x": 0,
+ "y": 8
+ },
+ "id": 261,
+ "panels": [
+ {
+ "cacheTimeout": null,
+ "datasource": "${DS_PROMETHEUS}",
+ "description": "Busy state of all CPU cores together",
+ "fieldConfig": {
+ "defaults": {
+ "color": {
+ "mode": "thresholds"
+ },
+ "custom": {},
+ "mappings": [
+ {
+ "id": 0,
+ "op": "=",
+ "text": "N/A",
+ "type": 1,
+ "value": "null"
+ }
+ ],
+ "max": 100,
+ "min": 0,
+ "nullValueMode": "null",
+ "thresholds": {
+ "mode": "absolute",
+ "steps": [
+ {
+ "color": "rgba(50, 172, 45, 0.97)",
+ "value": null
+ },
+ {
+ "color": "rgba(237, 129, 40, 0.89)",
+ "value": 85
+ },
+ {
+ "color": "rgba(245, 54, 54, 0.9)",
+ "value": 95
+ }
+ ]
+ },
+ "unit": "percent"
+ },
+ "overrides": []
+ },
+ "gridPos": {
+ "h": 4,
+ "w": 3,
+ "x": 0,
+ "y": 9
+ },
+ "id": 20,
+ "links": [],
+ "options": {
+ "orientation": "horizontal",
+ "reduceOptions": {
+ "calcs": [
+ "lastNotNull"
+ ],
+ "fields": "",
+ "values": false
+ },
+ "showThresholdLabels": false,
+ "showThresholdMarkers": true
+ },
+ "pluginVersion": "7.0.5",
+ "targets": [
+ {
+ "expr": "(((count(count(node_cpu_seconds_total{instance=~\"$node:$port\",job=~\"$job\"}) by (cpu))) - avg(sum by (mode)(irate(node_cpu_seconds_total{mode='idle',instance=~\"$node:$port\",job=~\"$job\"}[5m])))) * 100) / count(count(node_cpu_seconds_total{instance=~\"$node:$port\",job=~\"$job\"}) by (cpu))",
+ "hide": false,
+ "intervalFactor": 1,
+ "legendFormat": "",
+ "refId": "A",
+ "step": 900
+ }
+ ],
+ "title": "CPU Busy",
+ "type": "gauge"
+ },
+ {
+ "cacheTimeout": null,
+ "datasource": "${DS_PROMETHEUS}",
+ "description": "Busy state of all CPU cores together (5 min average)",
+ "fieldConfig": {
+ "defaults": {
+ "color": {
+ "mode": "thresholds"
+ },
+ "custom": {},
+ "mappings": [
+ {
+ "id": 0,
+ "op": "=",
+ "text": "N/A",
+ "type": 1,
+ "value": "null"
+ }
+ ],
+ "max": 100,
+ "min": 0,
+ "nullValueMode": "null",
+ "thresholds": {
+ "mode": "absolute",
+ "steps": [
+ {
+ "color": "rgba(50, 172, 45, 0.97)",
+ "value": null
+ },
+ {
+ "color": "rgba(237, 129, 40, 0.89)",
+ "value": 85
+ },
+ {
+ "color": "rgba(245, 54, 54, 0.9)",
+ "value": 95
+ }
+ ]
+ },
+ "unit": "percent"
+ },
+ "overrides": []
+ },
+ "gridPos": {
+ "h": 4,
+ "w": 3,
+ "x": 3,
+ "y": 9
+ },
+ "id": 155,
+ "links": [],
+ "options": {
+ "orientation": "horizontal",
+ "reduceOptions": {
+ "calcs": [
+ "lastNotNull"
+ ],
+ "fields": "",
+ "values": false
+ },
+ "showThresholdLabels": false,
+ "showThresholdMarkers": true
+ },
+ "pluginVersion": "7.0.5",
+ "targets": [
+ {
+ "expr": "avg(node_load5{instance=~\"$node:$port\",job=~\"$job\"}) / count(count(node_cpu_seconds_total{instance=~\"$node:$port\",job=~\"$job\"}) by (cpu)) * 100",
+ "format": "time_series",
+ "hide": false,
+ "intervalFactor": 1,
+ "refId": "A",
+ "step": 900
+ }
+ ],
+ "title": "Sys Load (5m avg)",
+ "type": "gauge"
+ },
+ {
+ "cacheTimeout": null,
+ "datasource": "${DS_PROMETHEUS}",
+ "description": "Busy state of all CPU cores together (15 min average)",
+ "fieldConfig": {
+ "defaults": {
+ "color": {
+ "mode": "thresholds"
+ },
+ "custom": {},
+ "mappings": [
+ {
+ "id": 0,
+ "op": "=",
+ "text": "N/A",
+ "type": 1,
+ "value": "null"
+ }
+ ],
+ "max": 100,
+ "min": 0,
+ "nullValueMode": "null",
+ "thresholds": {
+ "mode": "absolute",
+ "steps": [
+ {
+ "color": "rgba(50, 172, 45, 0.97)",
+ "value": null
+ },
+ {
+ "color": "rgba(237, 129, 40, 0.89)",
+ "value": 85
+ },
+ {
+ "color": "rgba(245, 54, 54, 0.9)",
+ "value": 95
+ }
+ ]
+ },
+ "unit": "percent"
+ },
+ "overrides": []
+ },
+ "gridPos": {
+ "h": 4,
+ "w": 3,
+ "x": 6,
+ "y": 9
+ },
+ "id": 19,
+ "links": [],
+ "options": {
+ "orientation": "horizontal",
+ "reduceOptions": {
+ "calcs": [
+ "lastNotNull"
+ ],
+ "fields": "",
+ "values": false
+ },
+ "showThresholdLabels": false,
+ "showThresholdMarkers": true
+ },
+ "pluginVersion": "7.0.5",
+ "targets": [
+ {
+ "expr": "avg(node_load15{instance=~\"$node:$port\",job=~\"$job\"}) / count(count(node_cpu_seconds_total{instance=~\"$node:$port\",job=~\"$job\"}) by (cpu)) * 100",
+ "hide": false,
+ "intervalFactor": 1,
+ "refId": "A",
+ "step": 900
+ }
+ ],
+ "title": "Sys Load (15m avg)",
+ "type": "gauge"
+ },
+ {
+ "cacheTimeout": null,
+ "datasource": "${DS_PROMETHEUS}",
+ "description": "Non available RAM memory",
+ "fieldConfig": {
+ "defaults": {
+ "color": {
+ "mode": "thresholds"
+ },
+ "custom": {},
+ "decimals": 0,
+ "mappings": [],
+ "max": 100,
+ "min": 0,
+ "nullValueMode": "null",
+ "thresholds": {
+ "mode": "absolute",
+ "steps": [
+ {
+ "color": "rgba(50, 172, 45, 0.97)",
+ "value": null
+ },
+ {
+ "color": "rgba(237, 129, 40, 0.89)",
+ "value": 80
+ },
+ {
+ "color": "rgba(245, 54, 54, 0.9)",
+ "value": 90
+ }
+ ]
+ },
+ "unit": "percent"
+ },
+ "overrides": []
+ },
+ "gridPos": {
+ "h": 4,
+ "w": 3,
+ "x": 9,
+ "y": 9
+ },
+ "hideTimeOverride": false,
+ "id": 16,
+ "links": [],
+ "options": {
+ "orientation": "horizontal",
+ "reduceOptions": {
+ "calcs": [
+ "lastNotNull"
+ ],
+ "fields": "",
+ "values": false
+ },
+ "showThresholdLabels": false,
+ "showThresholdMarkers": true
+ },
+ "pluginVersion": "7.0.5",
+ "targets": [
+ {
+ "expr": "((node_memory_MemTotal_bytes{instance=~\"$node:$port\",job=~\"$job\"} - node_memory_MemFree_bytes{instance=~\"$node:$port\",job=~\"$job\"}) / (node_memory_MemTotal_bytes{instance=~\"$node:$port\",job=~\"$job\"} )) * 100",
+ "format": "time_series",
+ "hide": true,
+ "intervalFactor": 1,
+ "refId": "A",
+ "step": 900
+ },
+ {
+ "expr": "100 - ((node_memory_MemAvailable_bytes{instance=~\"$node:$port\",job=~\"$job\"} * 100) / node_memory_MemTotal_bytes{instance=~\"$node:$port\",job=~\"$job\"})",
+ "format": "time_series",
+ "hide": false,
+ "intervalFactor": 1,
+ "refId": "B",
+ "step": 900
+ }
+ ],
+ "title": "RAM Used",
+ "type": "gauge"
+ },
+ {
+ "cacheTimeout": null,
+ "datasource": "${DS_PROMETHEUS}",
+ "description": "Used Swap",
+ "fieldConfig": {
+ "defaults": {
+ "color": {
+ "mode": "thresholds"
+ },
+ "custom": {},
+ "mappings": [
+ {
+ "id": 0,
+ "op": "=",
+ "text": "N/A",
+ "type": 1,
+ "value": "null"
+ }
+ ],
+ "max": 100,
+ "min": 0,
+ "nullValueMode": "null",
+ "thresholds": {
+ "mode": "absolute",
+ "steps": [
+ {
+ "color": "rgba(50, 172, 45, 0.97)",
+ "value": null
+ },
+ {
+ "color": "rgba(237, 129, 40, 0.89)",
+ "value": 10
+ },
+ {
+ "color": "rgba(245, 54, 54, 0.9)",
+ "value": 25
+ }
+ ]
+ },
+ "unit": "percent"
+ },
+ "overrides": []
+ },
+ "gridPos": {
+ "h": 4,
+ "w": 3,
+ "x": 12,
+ "y": 9
+ },
+ "id": 21,
+ "links": [],
+ "options": {
+ "orientation": "horizontal",
+ "reduceOptions": {
+ "calcs": [
+ "lastNotNull"
+ ],
+ "fields": "",
+ "values": false
+ },
+ "showThresholdLabels": false,
+ "showThresholdMarkers": true
+ },
+ "pluginVersion": "7.0.5",
+ "targets": [
+ {
+ "expr": "((node_memory_SwapTotal_bytes{instance=~\"$node:$port\",job=~\"$job\"} - node_memory_SwapFree_bytes{instance=~\"$node:$port\",job=~\"$job\"}) / (node_memory_SwapTotal_bytes{instance=~\"$node:$port\",job=~\"$job\"} )) * 100",
+ "intervalFactor": 1,
+ "refId": "A",
+ "step": 900
+ }
+ ],
+ "title": "SWAP Used",
+ "type": "gauge"
+ },
+ {
+ "cacheTimeout": null,
+ "datasource": "${DS_PROMETHEUS}",
+ "description": "Used Root FS",
+ "fieldConfig": {
+ "defaults": {
+ "color": {
+ "mode": "thresholds"
+ },
+ "custom": {},
+ "mappings": [
+ {
+ "id": 0,
+ "op": "=",
+ "text": "N/A",
+ "type": 1,
+ "value": "null"
+ }
+ ],
+ "max": 100,
+ "min": 0,
+ "nullValueMode": "null",
+ "thresholds": {
+ "mode": "absolute",
+ "steps": [
+ {
+ "color": "rgba(50, 172, 45, 0.97)",
+ "value": null
+ },
+ {
+ "color": "rgba(237, 129, 40, 0.89)",
+ "value": 80
+ },
+ {
+ "color": "rgba(245, 54, 54, 0.9)",
+ "value": 90
+ }
+ ]
+ },
+ "unit": "percent"
+ },
+ "overrides": []
+ },
+ "gridPos": {
+ "h": 4,
+ "w": 3,
+ "x": 15,
+ "y": 9
+ },
+ "id": 154,
+ "links": [],
+ "options": {
+ "orientation": "horizontal",
+ "reduceOptions": {
+ "calcs": [
+ "lastNotNull"
+ ],
+ "fields": "",
+ "values": false
+ },
+ "showThresholdLabels": false,
+ "showThresholdMarkers": true
+ },
+ "pluginVersion": "7.0.5",
+ "targets": [
+ {
+ "expr": "100 - ((node_filesystem_avail_bytes{instance=~\"$node:$port\",job=~\"$job\",mountpoint=\"/\",fstype!=\"rootfs\"} * 100) / node_filesystem_size_bytes{instance=~\"$node:$port\",job=~\"$job\",mountpoint=\"/\",fstype!=\"rootfs\"})",
+ "format": "time_series",
+ "intervalFactor": 1,
+ "refId": "A",
+ "step": 900
+ }
+ ],
+ "title": "Root FS Used",
+ "type": "gauge"
+ },
+ {
+ "cacheTimeout": null,
+ "colorBackground": false,
+ "colorValue": false,
+ "colors": [
+ "rgba(245, 54, 54, 0.9)",
+ "rgba(237, 129, 40, 0.89)",
+ "rgba(50, 172, 45, 0.97)"
+ ],
+ "datasource": "${DS_PROMETHEUS}",
+ "description": "Total number of CPU cores",
+ "fieldConfig": {
+ "defaults": {
+ "custom": {}
+ },
+ "overrides": []
+ },
+ "format": "short",
+ "gauge": {
+ "maxValue": 100,
+ "minValue": 0,
+ "show": false,
+ "thresholdLabels": false,
+ "thresholdMarkers": true
+ },
+ "gridPos": {
+ "h": 2,
+ "w": 2,
+ "x": 18,
+ "y": 9
+ },
+ "id": 14,
+ "interval": null,
+ "links": [],
+ "mappingType": 1,
+ "mappingTypes": [
+ {
+ "name": "value to text",
+ "value": 1
+ },
+ {
+ "name": "range to text",
+ "value": 2
+ }
+ ],
+ "maxDataPoints": 100,
+ "maxPerRow": 6,
+ "nullPointMode": "null",
+ "nullText": null,
+ "postfix": "",
+ "postfixFontSize": "50%",
+ "prefix": "",
+ "prefixFontSize": "50%",
+ "rangeMaps": [
+ {
+ "from": "null",
+ "text": "N/A",
+ "to": "null"
+ }
+ ],
+ "sparkline": {
+ "fillColor": "rgba(31, 118, 189, 0.18)",
+ "full": false,
+ "lineColor": "rgb(31, 120, 193)",
+ "show": false
+ },
+ "tableColumn": "",
+ "targets": [
+ {
+ "expr": "count(count(node_cpu_seconds_total{instance=~\"$node:$port\",job=~\"$job\"}) by (cpu))",
+ "interval": "",
+ "intervalFactor": 1,
+ "legendFormat": "",
+ "refId": "A",
+ "step": 900
+ }
+ ],
+ "thresholds": "",
+ "title": "CPU Cores",
+ "type": "singlestat",
+ "valueFontSize": "50%",
+ "valueMaps": [
+ {
+ "op": "=",
+ "text": "N/A",
+ "value": "null"
+ }
+ ],
+ "valueName": "current"
+ },
+ {
+ "cacheTimeout": null,
+ "colorBackground": false,
+ "colorValue": false,
+ "colors": [
+ "rgba(245, 54, 54, 0.9)",
+ "rgba(237, 129, 40, 0.89)",
+ "rgba(50, 172, 45, 0.97)"
+ ],
+ "datasource": "${DS_PROMETHEUS}",
+ "decimals": 1,
+ "description": "System uptime",
+ "fieldConfig": {
+ "defaults": {
+ "custom": {}
+ },
+ "overrides": []
+ },
+ "format": "s",
+ "gauge": {
+ "maxValue": 100,
+ "minValue": 0,
+ "show": false,
+ "thresholdLabels": false,
+ "thresholdMarkers": true
+ },
+ "gridPos": {
+ "h": 2,
+ "w": 4,
+ "x": 20,
+ "y": 9
+ },
+ "hideTimeOverride": true,
+ "id": 15,
+ "interval": null,
+ "links": [],
+ "mappingType": 1,
+ "mappingTypes": [
+ {
+ "name": "value to text",
+ "value": 1
+ },
+ {
+ "name": "range to text",
+ "value": 2
+ }
+ ],
+ "maxDataPoints": 100,
+ "nullPointMode": "null",
+ "nullText": null,
+ "postfix": "s",
+ "postfixFontSize": "50%",
+ "prefix": "",
+ "prefixFontSize": "50%",
+ "rangeMaps": [
+ {
+ "from": "null",
+ "text": "N/A",
+ "to": "null"
+ }
+ ],
+ "sparkline": {
+ "fillColor": "rgba(31, 118, 189, 0.18)",
+ "full": false,
+ "lineColor": "rgb(31, 120, 193)",
+ "show": false
+ },
+ "tableColumn": "{group=\"broker-os\", instance=\"10.0.0.15:9100\", job=\"kafka\"}",
+ "targets": [
+ {
+ "expr": "node_time_seconds{instance=~\"$node:$port\",job=~\"$job\"} - node_boot_time_seconds{instance=~\"$node:$port\",job=~\"$job\"}",
+ "intervalFactor": 2,
+ "refId": "A",
+ "step": 1800
+ }
+ ],
+ "thresholds": "",
+ "title": "Uptime",
+ "type": "singlestat",
+ "valueFontSize": "50%",
+ "valueMaps": [
+ {
+ "op": "=",
+ "text": "N/A",
+ "value": "null"
+ }
+ ],
+ "valueName": "current"
+ },
+ {
+ "cacheTimeout": null,
+ "colorBackground": false,
+ "colorValue": false,
+ "colors": [
+ "rgba(50, 172, 45, 0.97)",
+ "rgba(237, 129, 40, 0.89)",
+ "rgba(245, 54, 54, 0.9)"
+ ],
+ "datasource": "${DS_PROMETHEUS}",
+ "decimals": 0,
+ "description": "Total RootFS",
+ "fieldConfig": {
+ "defaults": {
+ "custom": {}
+ },
+ "overrides": []
+ },
+ "format": "bytes",
+ "gauge": {
+ "maxValue": 100,
+ "minValue": 0,
+ "show": false,
+ "thresholdLabels": false,
+ "thresholdMarkers": true
+ },
+ "gridPos": {
+ "h": 2,
+ "w": 2,
+ "x": 18,
+ "y": 11
+ },
+ "id": 23,
+ "interval": null,
+ "links": [],
+ "mappingType": 1,
+ "mappingTypes": [
+ {
+ "name": "value to text",
+ "value": 1
+ },
+ {
+ "name": "range to text",
+ "value": 2
+ }
+ ],
+ "maxDataPoints": 100,
+ "maxPerRow": 6,
+ "nullPointMode": "null",
+ "nullText": null,
+ "postfix": "",
+ "postfixFontSize": "50%",
+ "prefix": "",
+ "prefixFontSize": "50%",
+ "rangeMaps": [
+ {
+ "from": "null",
+ "text": "N/A",
+ "to": "null"
+ }
+ ],
+ "sparkline": {
+ "fillColor": "rgba(31, 118, 189, 0.18)",
+ "full": false,
+ "lineColor": "rgb(31, 120, 193)",
+ "show": false
+ },
+ "tableColumn": "node_filesystem_size_bytes{device=\"/dev/nvme0n1p2\", fstype=\"xfs\", group=\"broker-os\", instance=\"10.0.0.15:9100\", job=\"kafka\", mountpoint=\"/\"}",
+ "targets": [
+ {
+ "expr": "node_filesystem_size_bytes{instance=~\"$node:$port\",job=~\"$job\",mountpoint=\"/\",fstype!=\"rootfs\"}",
+ "format": "time_series",
+ "hide": false,
+ "intervalFactor": 1,
+ "refId": "A",
+ "step": 900
+ }
+ ],
+ "thresholds": "70,90",
+ "title": "RootFS Total",
+ "type": "singlestat",
+ "valueFontSize": "50%",
+ "valueMaps": [
+ {
+ "op": "=",
+ "text": "N/A",
+ "value": "null"
+ }
+ ],
+ "valueName": "current"
+ },
+ {
+ "cacheTimeout": null,
+ "colorBackground": false,
+ "colorValue": false,
+ "colors": [
+ "rgba(245, 54, 54, 0.9)",
+ "rgba(237, 129, 40, 0.89)",
+ "rgba(50, 172, 45, 0.97)"
+ ],
+ "datasource": "${DS_PROMETHEUS}",
+ "decimals": 0,
+ "description": "Total RAM",
+ "fieldConfig": {
+ "defaults": {
+ "custom": {}
+ },
+ "overrides": []
+ },
+ "format": "bytes",
+ "gauge": {
+ "maxValue": 100,
+ "minValue": 0,
+ "show": false,
+ "thresholdLabels": false,
+ "thresholdMarkers": true
+ },
+ "gridPos": {
+ "h": 2,
+ "w": 2,
+ "x": 20,
+ "y": 11
+ },
+ "id": 75,
+ "interval": null,
+ "links": [],
+ "mappingType": 1,
+ "mappingTypes": [
+ {
+ "name": "value to text",
+ "value": 1
+ },
+ {
+ "name": "range to text",
+ "value": 2
+ }
+ ],
+ "maxDataPoints": 100,
+ "maxPerRow": 6,
+ "nullPointMode": "null",
+ "nullText": null,
+ "postfix": "",
+ "postfixFontSize": "70%",
+ "prefix": "",
+ "prefixFontSize": "50%",
+ "rangeMaps": [
+ {
+ "from": "null",
+ "text": "N/A",
+ "to": "null"
+ }
+ ],
+ "sparkline": {
+ "fillColor": "rgba(31, 118, 189, 0.18)",
+ "full": false,
+ "lineColor": "rgb(31, 120, 193)",
+ "show": false
+ },
+ "tableColumn": "node_memory_MemTotal_bytes{group=\"broker-os\", instance=\"10.0.0.15:9100\", job=\"kafka\"}",
+ "targets": [
+ {
+ "expr": "node_memory_MemTotal_bytes{instance=~\"$node:$port\",job=~\"$job\"}",
+ "intervalFactor": 1,
+ "refId": "A",
+ "step": 900
+ }
+ ],
+ "thresholds": "",
+ "title": "RAM Total",
+ "type": "singlestat",
+ "valueFontSize": "50%",
+ "valueMaps": [
+ {
+ "op": "=",
+ "text": "N/A",
+ "value": "null"
+ }
+ ],
+ "valueName": "current"
+ },
+ {
+ "cacheTimeout": null,
+ "colorBackground": false,
+ "colorValue": false,
+ "colors": [
+ "rgba(245, 54, 54, 0.9)",
+ "rgba(237, 129, 40, 0.89)",
+ "rgba(50, 172, 45, 0.97)"
+ ],
+ "datasource": "${DS_PROMETHEUS}",
+ "decimals": 0,
+ "description": "Total SWAP",
+ "fieldConfig": {
+ "defaults": {
+ "custom": {}
+ },
+ "overrides": []
+ },
+ "format": "bytes",
+ "gauge": {
+ "maxValue": 100,
+ "minValue": 0,
+ "show": false,
+ "thresholdLabels": false,
+ "thresholdMarkers": true
+ },
+ "gridPos": {
+ "h": 2,
+ "w": 2,
+ "x": 22,
+ "y": 11
+ },
+ "id": 18,
+ "interval": null,
+ "links": [],
+ "mappingType": 1,
+ "mappingTypes": [
+ {
+ "name": "value to text",
+ "value": 1
+ },
+ {
+ "name": "range to text",
+ "value": 2
+ }
+ ],
+ "maxDataPoints": 100,
+ "maxPerRow": 6,
+ "nullPointMode": "null",
+ "nullText": null,
+ "postfix": "",
+ "postfixFontSize": "70%",
+ "prefix": "",
+ "prefixFontSize": "50%",
+ "rangeMaps": [
+ {
+ "from": "null",
+ "text": "N/A",
+ "to": "null"
+ }
+ ],
+ "sparkline": {
+ "fillColor": "rgba(31, 118, 189, 0.18)",
+ "full": false,
+ "lineColor": "rgb(31, 120, 193)",
+ "show": false
+ },
+ "tableColumn": "node_memory_SwapTotal_bytes{group=\"broker-os\", instance=\"10.0.0.15:9100\", job=\"kafka\"}",
+ "targets": [
+ {
+ "expr": "node_memory_SwapTotal_bytes{instance=~\"$node:$port\",job=~\"$job\"}",
+ "intervalFactor": 1,
+ "refId": "A",
+ "step": 900
+ }
+ ],
+ "thresholds": "",
+ "title": "SWAP Total",
+ "type": "singlestat",
+ "valueFontSize": "50%",
+ "valueMaps": [
+ {
+ "op": "=",
+ "text": "N/A",
+ "value": "null"
+ }
+ ],
+ "valueName": "current"
+ }
+ ],
+ "repeat": null,
+ "title": "Quick CPU / Mem / Disk",
+ "type": "row"
+ },
+ {
+ "collapsed": true,
+ "datasource": "${DS_PROMETHEUS}",
+ "gridPos": {
+ "h": 1,
+ "w": 24,
+ "x": 0,
+ "y": 9
+ },
+ "id": 263,
+ "panels": [
+ {
+ "aliasColors": {
+ "Busy": "#EAB839",
+ "Busy Iowait": "#890F02",
+ "Busy other": "#1F78C1",
+ "Idle": "#052B51",
+ "Idle - Waiting for something to happen": "#052B51",
+ "guest": "#9AC48A",
+ "idle": "#052B51",
+ "iowait": "#EAB839",
+ "irq": "#BF1B00",
+ "nice": "#C15C17",
+ "softirq": "#E24D42",
+ "steal": "#FCE2DE",
+ "system": "#508642",
+ "user": "#5195CE"
+ },
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "decimals": 2,
+ "description": "Basic CPU info",
+ "fieldConfig": {
+ "defaults": {
+ "custom": {}
+ },
+ "overrides": []
+ },
+ "fill": 4,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 7,
+ "w": 12,
+ "x": 0,
+ "y": 10
+ },
+ "hiddenSeries": false,
+ "id": 77,
+ "legend": {
+ "alignAsTable": false,
+ "avg": false,
+ "current": false,
+ "max": false,
+ "min": false,
+ "rightSide": false,
+ "show": true,
+ "sideWidth": 250,
+ "sort": null,
+ "sortDesc": null,
+ "total": false,
+ "values": false
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "maxPerRow": 6,
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": true,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [
+ {
+ "alias": "Busy Iowait",
+ "color": "#890F02"
+ },
+ {
+ "alias": "Idle",
+ "color": "#7EB26D"
+ },
+ {
+ "alias": "Busy System",
+ "color": "#EAB839"
+ },
+ {
+ "alias": "Busy User",
+ "color": "#0A437C"
+ },
+ {
+ "alias": "Busy Other",
+ "color": "#6D1F62"
+ }
+ ],
+ "spaceLength": 10,
+ "stack": true,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "sum by (instance)(irate(node_cpu_seconds_total{mode=\"system\",instance=~\"$node:$port\",job=~\"$job\"}[5m])) * 100",
+ "format": "time_series",
+ "hide": false,
+ "intervalFactor": 2,
+ "legendFormat": "Busy System",
+ "refId": "A",
+ "step": 240
+ },
+ {
+ "expr": "sum by (instance)(irate(node_cpu_seconds_total{mode='user',instance=~\"$node:$port\",job=~\"$job\"}[5m])) * 100",
+ "format": "time_series",
+ "hide": false,
+ "intervalFactor": 2,
+ "legendFormat": "Busy User",
+ "refId": "B",
+ "step": 240
+ },
+ {
+ "expr": "sum by (instance)(irate(node_cpu_seconds_total{mode='iowait',instance=~\"$node:$port\",job=~\"$job\"}[5m])) * 100",
+ "format": "time_series",
+ "intervalFactor": 2,
+ "legendFormat": "Busy Iowait",
+ "refId": "C",
+ "step": 240
+ },
+ {
+ "expr": "sum by (instance)(irate(node_cpu_seconds_total{mode=~\".*irq\",instance=~\"$node:$port\",job=~\"$job\"}[5m])) * 100",
+ "format": "time_series",
+ "intervalFactor": 2,
+ "legendFormat": "Busy IRQs",
+ "refId": "D",
+ "step": 240
+ },
+ {
+ "expr": "sum (irate(node_cpu_seconds_total{mode!='idle',mode!='user',mode!='system',mode!='iowait',mode!='irq',mode!='softirq',instance=~\"$node:$port\",job=~\"$job\"}[5m])) * 100",
+ "format": "time_series",
+ "intervalFactor": 2,
+ "legendFormat": "Busy Other",
+ "refId": "E",
+ "step": 240
+ },
+ {
+ "expr": "sum by (mode)(irate(node_cpu_seconds_total{mode='idle',instance=~\"$node:$port\",job=~\"$job\"}[5m])) * 100",
+ "format": "time_series",
+ "intervalFactor": 2,
+ "legendFormat": "Idle",
+ "refId": "F",
+ "step": 240
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "CPU Basic",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "short",
+ "label": "",
+ "logBase": 1,
+ "max": "100",
+ "min": "0",
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {
+ "Apps": "#629E51",
+ "Buffers": "#614D93",
+ "Cache": "#6D1F62",
+ "Cached": "#511749",
+ "Committed": "#508642",
+ "Free": "#0A437C",
+ "Hardware Corrupted - Amount of RAM that the kernel identified as corrupted / not working": "#CFFAFF",
+ "Inactive": "#584477",
+ "PageTables": "#0A50A1",
+ "Page_Tables": "#0A50A1",
+ "RAM_Free": "#E0F9D7",
+ "SWAP Used": "#BF1B00",
+ "Slab": "#806EB7",
+ "Slab_Cache": "#E0752D",
+ "Swap": "#BF1B00",
+ "Swap Used": "#BF1B00",
+ "Swap_Cache": "#C15C17",
+ "Swap_Free": "#2F575E",
+ "Unused": "#EAB839"
+ },
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "decimals": 2,
+ "description": "Basic memory usage",
+ "fieldConfig": {
+ "defaults": {
+ "custom": {}
+ },
+ "overrides": []
+ },
+ "fill": 4,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 7,
+ "w": 12,
+ "x": 12,
+ "y": 10
+ },
+ "hiddenSeries": false,
+ "id": 78,
+ "legend": {
+ "alignAsTable": false,
+ "avg": false,
+ "current": false,
+ "max": false,
+ "min": false,
+ "rightSide": false,
+ "show": true,
+ "sideWidth": 350,
+ "total": false,
+ "values": false
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "maxPerRow": 6,
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [
+ {
+ "alias": "RAM Total",
+ "color": "#E0F9D7",
+ "fill": 0,
+ "stack": false
+ },
+ {
+ "alias": "RAM Cache + Buffer",
+ "color": "#052B51"
+ },
+ {
+ "alias": "RAM Free",
+ "color": "#7EB26D"
+ },
+ {
+ "alias": "Avaliable",
+ "color": "#DEDAF7",
+ "fill": 0,
+ "stack": false
+ }
+ ],
+ "spaceLength": 10,
+ "stack": true,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "node_memory_MemTotal_bytes{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "hide": false,
+ "intervalFactor": 2,
+ "legendFormat": "RAM Total",
+ "refId": "A",
+ "step": 240
+ },
+ {
+ "expr": "node_memory_MemTotal_bytes{instance=~\"$node:$port\",job=~\"$job\"} - node_memory_MemFree_bytes{instance=~\"$node:$port\",job=~\"$job\"} - (node_memory_Cached_bytes{instance=~\"$node:$port\",job=~\"$job\"} + node_memory_Buffers_bytes{instance=~\"$node:$port\",job=~\"$job\"})",
+ "format": "time_series",
+ "hide": false,
+ "intervalFactor": 2,
+ "legendFormat": "RAM Used",
+ "refId": "B",
+ "step": 240
+ },
+ {
+ "expr": "node_memory_Cached_bytes{instance=~\"$node:$port\",job=~\"$job\"} + node_memory_Buffers_bytes{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "intervalFactor": 2,
+ "legendFormat": "RAM Cache + Buffer",
+ "refId": "C",
+ "step": 240
+ },
+ {
+ "expr": "node_memory_MemFree_bytes{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "intervalFactor": 2,
+ "legendFormat": "RAM Free",
+ "refId": "D",
+ "step": 240
+ },
+ {
+ "expr": "(node_memory_SwapTotal_bytes{instance=~\"$node:$port\",job=~\"$job\"} - node_memory_SwapFree_bytes{instance=~\"$node:$port\",job=~\"$job\"})",
+ "format": "time_series",
+ "intervalFactor": 2,
+ "legendFormat": "SWAP Used",
+ "refId": "E",
+ "step": 240
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "Memory Basic",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "bytes",
+ "label": "",
+ "logBase": 1,
+ "max": null,
+ "min": "0",
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {
+ "Recv_bytes_eth2": "#7EB26D",
+ "Recv_bytes_lo": "#0A50A1",
+ "Recv_drop_eth2": "#6ED0E0",
+ "Recv_drop_lo": "#E0F9D7",
+ "Recv_errs_eth2": "#BF1B00",
+ "Recv_errs_lo": "#CCA300",
+ "Trans_bytes_eth2": "#7EB26D",
+ "Trans_bytes_lo": "#0A50A1",
+ "Trans_drop_eth2": "#6ED0E0",
+ "Trans_drop_lo": "#E0F9D7",
+ "Trans_errs_eth2": "#BF1B00",
+ "Trans_errs_lo": "#CCA300",
+ "recv_bytes_lo": "#0A50A1",
+ "recv_drop_eth0": "#99440A",
+ "recv_drop_lo": "#967302",
+ "recv_errs_eth0": "#BF1B00",
+ "recv_errs_lo": "#890F02",
+ "trans_bytes_eth0": "#7EB26D",
+ "trans_bytes_lo": "#0A50A1",
+ "trans_drop_eth0": "#99440A",
+ "trans_drop_lo": "#967302",
+ "trans_errs_eth0": "#BF1B00",
+ "trans_errs_lo": "#890F02"
+ },
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "description": "Basic network info per interface",
+ "fieldConfig": {
+ "defaults": {
+ "custom": {}
+ },
+ "overrides": []
+ },
+ "fill": 4,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 7,
+ "w": 12,
+ "x": 0,
+ "y": 17
+ },
+ "hiddenSeries": false,
+ "id": 74,
+ "legend": {
+ "alignAsTable": false,
+ "avg": false,
+ "current": false,
+ "hideEmpty": false,
+ "hideZero": false,
+ "max": false,
+ "min": false,
+ "rightSide": false,
+ "show": true,
+ "sort": "current",
+ "sortDesc": true,
+ "total": false,
+ "values": false
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [
+ {
+ "alias": "/.*trans.*/",
+ "transform": "negative-Y"
+ }
+ ],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "irate(node_network_receive_bytes_total{instance=~\"$node:$port\",job=~\"$job\"}[5m])*8",
+ "format": "time_series",
+ "intervalFactor": 2,
+ "legendFormat": "recv {{device}}",
+ "refId": "A",
+ "step": 240
+ },
+ {
+ "expr": "irate(node_network_transmit_bytes_total{instance=~\"$node:$port\",job=~\"$job\"}[5m])*8",
+ "format": "time_series",
+ "intervalFactor": 2,
+ "legendFormat": "trans {{device}} ",
+ "refId": "B",
+ "step": 240
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "Network Traffic Basic",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "bps",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ },
+ {
+ "format": "pps",
+ "label": "",
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "decimals": 3,
+ "description": "Disk space used of all filesystems mounted",
+ "fieldConfig": {
+ "defaults": {
+ "custom": {}
+ },
+ "overrides": []
+ },
+ "fill": 4,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 7,
+ "w": 12,
+ "x": 12,
+ "y": 17
+ },
+ "height": "",
+ "hiddenSeries": false,
+ "id": 152,
+ "legend": {
+ "alignAsTable": false,
+ "avg": false,
+ "current": false,
+ "max": false,
+ "min": false,
+ "rightSide": false,
+ "show": true,
+ "sort": "current",
+ "sortDesc": false,
+ "total": false,
+ "values": false
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "maxPerRow": 6,
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "100 - ((node_filesystem_avail_bytes{instance=~\"$node:$port\",job=~\"$job\",device!~'rootfs'} * 100) / node_filesystem_size_bytes{instance=~\"$node:$port\",job=~\"$job\",device!~'rootfs'})",
+ "format": "time_series",
+ "intervalFactor": 2,
+ "legendFormat": "{{mountpoint}}",
+ "refId": "A",
+ "step": 240
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "Disk Space Used Basic",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "percent",
+ "label": null,
+ "logBase": 1,
+ "max": "100",
+ "min": "0",
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ }
+ ],
+ "repeat": null,
+ "title": "Basic CPU / Mem / Net / Disk",
+ "type": "row"
+ },
+ {
+ "collapsed": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "gridPos": {
+ "h": 1,
+ "w": 24,
+ "x": 0,
+ "y": 10
+ },
+ "id": 265,
+ "panels": [],
+ "repeat": null,
+ "title": "CPU / Memory / Net / Disk",
+ "type": "row"
+ },
+ {
+ "aliasColors": {
+ "Idle - Waiting for something to happen": "#052B51",
+ "guest": "#9AC48A",
+ "idle": "#052B51",
+ "iowait": "#EAB839",
+ "irq": "#BF1B00",
+ "nice": "#C15C17",
+ "softirq": "#E24D42",
+ "steal": "#FCE2DE",
+ "system": "#508642",
+ "user": "#5195CE"
+ },
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "decimals": 2,
+ "description": "",
+ "fieldConfig": {
+ "defaults": {
+ "custom": {}
+ },
+ "overrides": []
+ },
+ "fill": 4,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 12,
+ "w": 12,
+ "x": 0,
+ "y": 11
+ },
+ "hiddenSeries": false,
+ "id": 3,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "max": true,
+ "min": true,
+ "rightSide": false,
+ "show": true,
+ "sideWidth": 250,
+ "sort": null,
+ "sortDesc": null,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "maxPerRow": 6,
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": true,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "repeat": null,
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "stack": true,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "sum by (mode)(irate(node_cpu_seconds_total{mode=\"system\",instance=~\"$node:$port\",job=~\"$job\"}[5m])) * 100",
+ "format": "time_series",
+ "interval": "10s",
+ "intervalFactor": 2,
+ "legendFormat": "System - Processes executing in kernel mode",
+ "refId": "A",
+ "step": 20
+ },
+ {
+ "expr": "sum by (mode)(irate(node_cpu_seconds_total{mode='user',instance=~\"$node:$port\",job=~\"$job\"}[5m])) * 100",
+ "format": "time_series",
+ "intervalFactor": 2,
+ "legendFormat": "User - Normal processes executing in user mode",
+ "refId": "B",
+ "step": 240
+ },
+ {
+ "expr": "sum by (mode)(irate(node_cpu_seconds_total{mode='nice',instance=~\"$node:$port\",job=~\"$job\"}[5m])) * 100",
+ "format": "time_series",
+ "intervalFactor": 2,
+ "legendFormat": "Nice - Niced processes executing in user mode",
+ "refId": "C",
+ "step": 240
+ },
+ {
+ "expr": "sum by (mode)(irate(node_cpu_seconds_total{mode='idle',instance=~\"$node:$port\",job=~\"$job\"}[5m])) * 100",
+ "format": "time_series",
+ "intervalFactor": 2,
+ "legendFormat": "Idle - Waiting for something to happen",
+ "refId": "D",
+ "step": 240
+ },
+ {
+ "expr": "sum by (mode)(irate(node_cpu_seconds_total{mode='iowait',instance=~\"$node:$port\",job=~\"$job\"}[5m])) * 100",
+ "format": "time_series",
+ "intervalFactor": 2,
+ "legendFormat": "Iowait - Waiting for I/O to complete",
+ "refId": "E",
+ "step": 240
+ },
+ {
+ "expr": "sum by (mode)(irate(node_cpu_seconds_total{mode='irq',instance=~\"$node:$port\",job=~\"$job\"}[5m])) * 100",
+ "format": "time_series",
+ "intervalFactor": 2,
+ "legendFormat": "Irq - Servicing interrupts",
+ "refId": "F",
+ "step": 240
+ },
+ {
+ "expr": "sum by (mode)(irate(node_cpu_seconds_total{mode='softirq',instance=~\"$node:$port\",job=~\"$job\"}[5m])) * 100",
+ "format": "time_series",
+ "intervalFactor": 2,
+ "legendFormat": "Softirq - Servicing softirqs",
+ "refId": "G",
+ "step": 240
+ },
+ {
+ "expr": "sum by (mode)(irate(node_cpu_seconds_total{mode='steal',instance=~\"$node:$port\",job=~\"$job\"}[5m])) * 100",
+ "format": "time_series",
+ "intervalFactor": 2,
+ "legendFormat": "Steal - Time spent in other operating systems when running in a virtualized environment",
+ "refId": "H",
+ "step": 240
+ },
+ {
+ "expr": "sum by (mode)(irate(node_cpu_seconds_total{mode='guest',instance=~\"$node:$port\",job=~\"$job\"}[5m])) * 100",
+ "format": "time_series",
+ "intervalFactor": 2,
+ "legendFormat": "Guest - Time spent running a virtual CPU for a guest operating system",
+ "refId": "I",
+ "step": 240
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "CPU",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "short",
+ "label": "percentage",
+ "logBase": 1,
+ "max": "100",
+ "min": "0",
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {
+ "Apps": "#629E51",
+ "Buffers": "#614D93",
+ "Cache": "#6D1F62",
+ "Cached": "#511749",
+ "Committed": "#508642",
+ "Free": "#0A437C",
+ "Hardware Corrupted - Amount of RAM that the kernel identified as corrupted / not working": "#CFFAFF",
+ "Inactive": "#584477",
+ "PageTables": "#0A50A1",
+ "Page_Tables": "#0A50A1",
+ "RAM_Free": "#E0F9D7",
+ "Slab": "#806EB7",
+ "Slab_Cache": "#E0752D",
+ "Swap": "#BF1B00",
+ "Swap - Swap memory usage": "#BF1B00",
+ "Swap_Cache": "#C15C17",
+ "Swap_Free": "#2F575E",
+ "Unused": "#EAB839",
+ "Unused - Free memory unassigned": "#052B51"
+ },
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "decimals": 2,
+ "description": "",
+ "fieldConfig": {
+ "defaults": {
+ "custom": {}
+ },
+ "overrides": []
+ },
+ "fill": 4,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 12,
+ "w": 12,
+ "x": 12,
+ "y": 11
+ },
+ "hiddenSeries": false,
+ "id": 24,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "max": true,
+ "min": true,
+ "rightSide": false,
+ "show": true,
+ "sideWidth": 350,
+ "sort": null,
+ "sortDesc": null,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "maxPerRow": 6,
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [
+ {
+ "alias": "/.*Hardware Corrupted - *./",
+ "stack": false
+ }
+ ],
+ "spaceLength": 10,
+ "stack": true,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "node_memory_MemTotal_bytes{instance=~\"$node:$port\",job=~\"$job\"} - node_memory_MemFree_bytes{instance=~\"$node:$port\",job=~\"$job\"} - node_memory_Buffers_bytes{instance=~\"$node:$port\",job=~\"$job\"} - node_memory_Cached_bytes{instance=~\"$node:$port\",job=~\"$job\"} - node_memory_Slab_bytes{instance=~\"$node:$port\",job=~\"$job\"} - node_memory_PageTables_bytes{instance=~\"$node:$port\",job=~\"$job\"} - node_memory_SwapCached_bytes{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "hide": false,
+ "intervalFactor": 2,
+ "legendFormat": "Apps - Memory used by user-space applications",
+ "refId": "A",
+ "step": 240
+ },
+ {
+ "expr": "node_memory_PageTables_bytes{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "hide": false,
+ "intervalFactor": 2,
+ "legendFormat": "PageTables - Memory used to map between virtual and physical memory addresses",
+ "refId": "B",
+ "step": 240
+ },
+ {
+ "expr": "node_memory_SwapCached_bytes{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "intervalFactor": 2,
+ "legendFormat": "SwapCache - Memory that keeps track of pages that have been fetched from swap but not yet been modified",
+ "refId": "C",
+ "step": 240
+ },
+ {
+ "expr": "node_memory_Slab_bytes{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "hide": false,
+ "intervalFactor": 2,
+ "legendFormat": "Slab - Memory used by the kernel to cache data structures for its own use (caches like inode, dentry, etc)",
+ "refId": "D",
+ "step": 240
+ },
+ {
+ "expr": "node_memory_Cached_bytes{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "hide": false,
+ "intervalFactor": 2,
+ "legendFormat": "Cache - Parked file data (file content) cache",
+ "refId": "E",
+ "step": 240
+ },
+ {
+ "expr": "node_memory_Buffers_bytes{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "hide": false,
+ "intervalFactor": 2,
+ "legendFormat": "Buffers - Block device (e.g. harddisk) cache",
+ "refId": "F",
+ "step": 240
+ },
+ {
+ "expr": "node_memory_MemFree_bytes{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "hide": false,
+ "intervalFactor": 2,
+ "legendFormat": "Unused - Free memory unassigned",
+ "refId": "G",
+ "step": 240
+ },
+ {
+ "expr": "(node_memory_SwapTotal_bytes{instance=~\"$node:$port\",job=~\"$job\"} - node_memory_SwapFree_bytes{instance=~\"$node:$port\",job=~\"$job\"})",
+ "format": "time_series",
+ "hide": false,
+ "intervalFactor": 2,
+ "legendFormat": "Swap - Swap space used",
+ "refId": "H",
+ "step": 240
+ },
+ {
+ "expr": "node_memory_HardwareCorrupted_bytes{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "hide": false,
+ "intervalFactor": 2,
+ "legendFormat": "Hardware Corrupted - Amount of RAM that the kernel identified as corrupted / not working",
+ "refId": "I",
+ "step": 240
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "Memory Stack",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "bytes",
+ "label": "bytes",
+ "logBase": 1,
+ "max": null,
+ "min": "0",
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {
+ "receive_packets_eth0": "#7EB26D",
+ "receive_packets_lo": "#E24D42",
+ "transmit_packets_eth0": "#7EB26D",
+ "transmit_packets_lo": "#E24D42"
+ },
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "fieldConfig": {
+ "defaults": {
+ "custom": {}
+ },
+ "overrides": []
+ },
+ "fill": 4,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 12,
+ "w": 12,
+ "x": 0,
+ "y": 23
+ },
+ "hiddenSeries": false,
+ "id": 84,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "max": true,
+ "min": true,
+ "rightSide": false,
+ "show": true,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [
+ {
+ "alias": "/.*Trans.*/",
+ "transform": "negative-Y"
+ }
+ ],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "irate(node_network_receive_bytes_total{instance=~\"$node:$port\",job=~\"$job\"}[5m])*8",
+ "format": "time_series",
+ "intervalFactor": 2,
+ "legendFormat": "{{device}} - Receive",
+ "refId": "A",
+ "step": 240
+ },
+ {
+ "expr": "irate(node_network_transmit_bytes_total{instance=~\"$node:$port\",job=~\"$job\"}[5m])*8",
+ "format": "time_series",
+ "intervalFactor": 2,
+ "legendFormat": "{{device}} - Transmit",
+ "refId": "B",
+ "step": 240
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "Network Traffic",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "bps",
+ "label": "bits out (-) / in (+)",
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "decimals": 3,
+ "description": "",
+ "fieldConfig": {
+ "defaults": {
+ "custom": {}
+ },
+ "overrides": []
+ },
+ "fill": 4,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 12,
+ "w": 12,
+ "x": 12,
+ "y": 23
+ },
+ "height": "",
+ "hiddenSeries": false,
+ "id": 156,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "max": true,
+ "min": true,
+ "rightSide": false,
+ "show": true,
+ "sort": "current",
+ "sortDesc": false,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "maxPerRow": 6,
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "node_filesystem_size_bytes{instance=~\"$node:$port\",job=~\"$job\",device!~'rootfs'} - node_filesystem_avail_bytes{instance=~\"$node:$port\",job=~\"$job\",device!~'rootfs'}",
+ "format": "time_series",
+ "intervalFactor": 2,
+ "legendFormat": "{{mountpoint}}",
+ "refId": "A",
+ "step": 240
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "Disk Space Used",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "bytes",
+ "label": "bytes",
+ "logBase": 1,
+ "max": null,
+ "min": "0",
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "Prometheus",
+ "description": "",
+ "fieldConfig": {
+ "defaults": {
+ "custom": {}
+ },
+ "overrides": []
+ },
+ "fill": 2,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 12,
+ "w": 12,
+ "x": 0,
+ "y": 35
+ },
+ "hiddenSeries": false,
+ "id": 229,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "hideZero": true,
+ "max": true,
+ "min": true,
+ "rightSide": false,
+ "show": true,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "maxPerRow": 6,
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [
+ {
+ "alias": "/.*Read.*/",
+ "transform": "negative-Y"
+ },
+ {
+ "alias": "/.*sda_.*/",
+ "color": "#7EB26D"
+ },
+ {
+ "alias": "/.*sdb_.*/",
+ "color": "#EAB839"
+ },
+ {
+ "alias": "/.*sdc_.*/",
+ "color": "#6ED0E0"
+ },
+ {
+ "alias": "/.*sdd_.*/",
+ "color": "#EF843C"
+ },
+ {
+ "alias": "/.*sde_.*/",
+ "color": "#E24D42"
+ },
+ {
+ "alias": "/.*sda1.*/",
+ "color": "#584477"
+ },
+ {
+ "alias": "/.*sda2_.*/",
+ "color": "#BA43A9"
+ },
+ {
+ "alias": "/.*sda3_.*/",
+ "color": "#F4D598"
+ },
+ {
+ "alias": "/.*sdb1.*/",
+ "color": "#0A50A1"
+ },
+ {
+ "alias": "/.*sdb2.*/",
+ "color": "#BF1B00"
+ },
+ {
+ "alias": "/.*sdb2.*/",
+ "color": "#BF1B00"
+ },
+ {
+ "alias": "/.*sdb3.*/",
+ "color": "#E0752D"
+ },
+ {
+ "alias": "/.*sdc1.*/",
+ "color": "#962D82"
+ },
+ {
+ "alias": "/.*sdc2.*/",
+ "color": "#614D93"
+ },
+ {
+ "alias": "/.*sdc3.*/",
+ "color": "#9AC48A"
+ },
+ {
+ "alias": "/.*sdd1.*/",
+ "color": "#65C5DB"
+ },
+ {
+ "alias": "/.*sdd2.*/",
+ "color": "#F9934E"
+ },
+ {
+ "alias": "/.*sdd3.*/",
+ "color": "#EA6460"
+ },
+ {
+ "alias": "/.*sde1.*/",
+ "color": "#E0F9D7"
+ },
+ {
+ "alias": "/.*sdd2.*/",
+ "color": "#FCEACA"
+ },
+ {
+ "alias": "/.*sde3.*/",
+ "color": "#F9E2D2"
+ }
+ ],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "irate(node_disk_reads_completed_total{instance=~\"$node:$port\"}[5m])",
+ "interval": "",
+ "intervalFactor": 4,
+ "legendFormat": "{{device}} - Reads completed",
+ "refId": "A",
+ "step": 480
+ },
+ {
+ "expr": "irate(node_disk_writes_completed_total{instance=~\"$node:$port\"}[5m])",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "{{device}} - Writes completed",
+ "refId": "B",
+ "step": 240
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "Disk IOps",
+ "tooltip": {
+ "shared": false,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "iops",
+ "label": "IO read (-) / write (+)",
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {
+ "io time": "#890F02"
+ },
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "decimals": 3,
+ "description": "",
+ "fieldConfig": {
+ "defaults": {
+ "custom": {}
+ },
+ "overrides": []
+ },
+ "fill": 4,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 12,
+ "w": 12,
+ "x": 12,
+ "y": 35
+ },
+ "hiddenSeries": false,
+ "id": 42,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "max": true,
+ "min": true,
+ "rightSide": false,
+ "show": true,
+ "sort": null,
+ "sortDesc": null,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "maxPerRow": 6,
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [
+ {
+ "alias": "/.*read*./",
+ "transform": "negative-Y"
+ },
+ {
+ "alias": "/.*sda.*/",
+ "color": "#7EB26D"
+ },
+ {
+ "alias": "/.*sdb.*/",
+ "color": "#EAB839"
+ },
+ {
+ "alias": "/.*sdc.*/",
+ "color": "#6ED0E0"
+ },
+ {
+ "alias": "/.*sdd.*/",
+ "color": "#EF843C"
+ },
+ {
+ "alias": "/.*sde.*/",
+ "color": "#E24D42"
+ }
+ ],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "irate(node_disk_read_bytes_total{instance=~\"$node:$port\"}[5m])",
+ "format": "time_series",
+ "hide": false,
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "{{device}} - Successfully read bytes",
+ "refId": "A",
+ "step": 240
+ },
+ {
+ "expr": "irate(node_disk_written_bytes_total{instance=~\"$node:$port\"}[5m])",
+ "format": "time_series",
+ "hide": false,
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "{{device}} - Successfully written bytes",
+ "refId": "B",
+ "step": 240
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "I/O Usage Read / Write",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": false,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "bytes",
+ "label": "bytes read (-) / write (+)",
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ },
+ {
+ "format": "ms",
+ "label": "",
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {
+ "io time": "#890F02"
+ },
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "decimals": 3,
+ "description": "",
+ "fieldConfig": {
+ "defaults": {
+ "custom": {}
+ },
+ "overrides": []
+ },
+ "fill": 4,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 12,
+ "w": 12,
+ "x": 0,
+ "y": 47
+ },
+ "hiddenSeries": false,
+ "id": 127,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "max": true,
+ "min": true,
+ "rightSide": false,
+ "show": true,
+ "sort": null,
+ "sortDesc": null,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "maxPerRow": 6,
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "irate(node_disk_io_time_seconds_total{instance=~\"$node:$port\"} [5m])",
+ "format": "time_series",
+ "hide": false,
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "{{device}} - Time spent doing I/Os",
+ "refId": "A",
+ "step": 240
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "I/O Usage Times",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": false,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "s",
+ "label": "time",
+ "logBase": 1,
+ "max": null,
+ "min": "0",
+ "show": true
+ },
+ {
+ "format": "s",
+ "label": "",
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "collapsed": true,
+ "datasource": "${DS_PROMETHEUS}",
+ "gridPos": {
+ "h": 1,
+ "w": 24,
+ "x": 0,
+ "y": 59
+ },
+ "id": 266,
+ "panels": [
+ {
+ "aliasColors": {
+ "Apps": "#629E51",
+ "Buffers": "#614D93",
+ "Cache": "#6D1F62",
+ "Cached": "#511749",
+ "Committed": "#508642",
+ "Free": "#0A437C",
+ "Hardware Corrupted - Amount of RAM that the kernel identified as corrupted / not working": "#CFFAFF",
+ "Inactive": "#584477",
+ "PageTables": "#0A50A1",
+ "Page_Tables": "#0A50A1",
+ "RAM_Free": "#E0F9D7",
+ "Slab": "#806EB7",
+ "Slab_Cache": "#E0752D",
+ "Swap": "#BF1B00",
+ "Swap_Cache": "#C15C17",
+ "Swap_Free": "#2F575E",
+ "Unused": "#EAB839"
+ },
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "decimals": 2,
+ "fieldConfig": {
+ "defaults": {
+ "custom": {}
+ },
+ "overrides": []
+ },
+ "fill": 2,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 0,
+ "y": 12
+ },
+ "hiddenSeries": false,
+ "id": 136,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "max": true,
+ "min": true,
+ "rightSide": false,
+ "show": true,
+ "sideWidth": 350,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "maxPerRow": 2,
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "stack": true,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "node_memory_Inactive_bytes{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "intervalFactor": 2,
+ "legendFormat": "Inactive - Memory which has been less recently used. It is more eligible to be reclaimed for other purposes",
+ "refId": "A",
+ "step": 4
+ },
+ {
+ "expr": "node_memory_Active_bytes{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "intervalFactor": 2,
+ "legendFormat": "Active - Memory that has been used more recently and usually not reclaimed unless absolutely necessary",
+ "refId": "B",
+ "step": 4
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "Memory Active / Inactive",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "cumulative"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "bytes",
+ "label": "bytes",
+ "logBase": 1,
+ "max": null,
+ "min": "0",
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {
+ "Apps": "#629E51",
+ "Buffers": "#614D93",
+ "Cache": "#6D1F62",
+ "Cached": "#511749",
+ "Committed": "#508642",
+ "Free": "#0A437C",
+ "Hardware Corrupted - Amount of RAM that the kernel identified as corrupted / not working": "#CFFAFF",
+ "Inactive": "#584477",
+ "PageTables": "#0A50A1",
+ "Page_Tables": "#0A50A1",
+ "RAM_Free": "#E0F9D7",
+ "Slab": "#806EB7",
+ "Slab_Cache": "#E0752D",
+ "Swap": "#BF1B00",
+ "Swap_Cache": "#C15C17",
+ "Swap_Free": "#2F575E",
+ "Unused": "#EAB839"
+ },
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "decimals": 2,
+ "fieldConfig": {
+ "defaults": {
+ "custom": {}
+ },
+ "overrides": []
+ },
+ "fill": 2,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 12,
+ "y": 12
+ },
+ "hiddenSeries": false,
+ "id": 135,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "max": true,
+ "min": true,
+ "rightSide": false,
+ "show": true,
+ "sideWidth": 350,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "maxPerRow": 6,
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [
+ {
+ "alias": "/.*Committed_AS - *./"
+ },
+ {
+ "alias": "/.*CommitLimit - *./",
+ "color": "#BF1B00",
+ "fill": 0
+ }
+ ],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "node_memory_Committed_AS_bytes{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "intervalFactor": 2,
+ "legendFormat": "Committed_AS - Amount of memory presently allocated on the system",
+ "refId": "A",
+ "step": 4
+ },
+ {
+ "expr": "node_memory_CommitLimit_bytes{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "intervalFactor": 2,
+ "legendFormat": "CommitLimit - Amount of memory currently available to be allocated on the system",
+ "refId": "B",
+ "step": 4
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "Memory Commited",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "cumulative"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "bytes",
+ "label": "bytes",
+ "logBase": 1,
+ "max": null,
+ "min": "0",
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {
+ "Apps": "#629E51",
+ "Buffers": "#614D93",
+ "Cache": "#6D1F62",
+ "Cached": "#511749",
+ "Committed": "#508642",
+ "Free": "#0A437C",
+ "Hardware Corrupted - Amount of RAM that the kernel identified as corrupted / not working": "#CFFAFF",
+ "Inactive": "#584477",
+ "PageTables": "#0A50A1",
+ "Page_Tables": "#0A50A1",
+ "RAM_Free": "#E0F9D7",
+ "Slab": "#806EB7",
+ "Slab_Cache": "#E0752D",
+ "Swap": "#BF1B00",
+ "Swap_Cache": "#C15C17",
+ "Swap_Free": "#2F575E",
+ "Unused": "#EAB839"
+ },
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "decimals": 2,
+ "fieldConfig": {
+ "defaults": {
+ "custom": {}
+ },
+ "overrides": []
+ },
+ "fill": 2,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 0,
+ "y": 22
+ },
+ "hiddenSeries": false,
+ "id": 191,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "max": true,
+ "min": true,
+ "rightSide": false,
+ "show": true,
+ "sideWidth": 350,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "maxPerRow": 6,
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "stack": true,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "node_memory_Inactive_file_bytes{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "hide": false,
+ "intervalFactor": 2,
+ "legendFormat": "Inactive_file - File-backed memory on inactive LRU list",
+ "refId": "A",
+ "step": 4
+ },
+ {
+ "expr": "node_memory_Inactive_anon_bytes{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "hide": false,
+ "intervalFactor": 2,
+ "legendFormat": "Inactive_anon - Anonymous and swap cache on inactive LRU list, including tmpfs (shmem)",
+ "refId": "B",
+ "step": 4
+ },
+ {
+ "expr": "node_memory_Active_file_bytes{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "hide": false,
+ "intervalFactor": 2,
+ "legendFormat": "Active_file - File-backed memory on active LRU list",
+ "refId": "C",
+ "step": 4
+ },
+ {
+ "expr": "node_memory_Active_anon_bytes{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "hide": false,
+ "intervalFactor": 2,
+ "legendFormat": "Active_anon - Anonymous and swap cache on active least-recently-used (LRU) list, including tmpfs",
+ "refId": "D",
+ "step": 4
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "Memory Active / Inactive Detail",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "cumulative"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "bytes",
+ "label": "bytes",
+ "logBase": 1,
+ "max": null,
+ "min": "0",
+ "show": true
+ },
+ {
+ "format": "bytes",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {
+ "Active": "#99440A",
+ "Buffers": "#58140C",
+ "Cache": "#6D1F62",
+ "Cached": "#511749",
+ "Committed": "#508642",
+ "Dirty": "#6ED0E0",
+ "Free": "#B7DBAB",
+ "Inactive": "#EA6460",
+ "Mapped": "#052B51",
+ "PageTables": "#0A50A1",
+ "Page_Tables": "#0A50A1",
+ "Slab_Cache": "#EAB839",
+ "Swap": "#BF1B00",
+ "Swap_Cache": "#C15C17",
+ "Total": "#511749",
+ "Total RAM": "#052B51",
+ "Total RAM + Swap": "#052B51",
+ "Total Swap": "#614D93",
+ "VmallocUsed": "#EA6460"
+ },
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "decimals": 2,
+ "fieldConfig": {
+ "defaults": {
+ "custom": {}
+ },
+ "overrides": []
+ },
+ "fill": 2,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 12,
+ "y": 22
+ },
+ "hiddenSeries": false,
+ "id": 130,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "max": true,
+ "min": true,
+ "rightSide": false,
+ "show": true,
+ "sideWidth": null,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "maxPerRow": 2,
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "node_memory_Writeback_bytes{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "intervalFactor": 2,
+ "legendFormat": "Writeback - Memory which is actively being written back to disk",
+ "refId": "A",
+ "step": 4
+ },
+ {
+ "expr": "node_memory_WritebackTmp_bytes{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "intervalFactor": 2,
+ "legendFormat": "WritebackTmp - Memory used by FUSE for temporary writeback buffers",
+ "refId": "B",
+ "step": 4
+ },
+ {
+ "expr": "node_memory_Dirty_bytes{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "intervalFactor": 2,
+ "legendFormat": "Dirty - Memory which is waiting to get written back to the disk",
+ "refId": "C",
+ "step": 4
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "Memory Writeback and Dirty",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "bytes",
+ "label": "bytes",
+ "logBase": 1,
+ "max": null,
+ "min": "0",
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {
+ "Apps": "#629E51",
+ "Buffers": "#614D93",
+ "Cache": "#6D1F62",
+ "Cached": "#511749",
+ "Committed": "#508642",
+ "Free": "#0A437C",
+ "Hardware Corrupted - Amount of RAM that the kernel identified as corrupted / not working": "#CFFAFF",
+ "Inactive": "#584477",
+ "PageTables": "#0A50A1",
+ "Page_Tables": "#0A50A1",
+ "RAM_Free": "#E0F9D7",
+ "Slab": "#806EB7",
+ "Slab_Cache": "#E0752D",
+ "Swap": "#BF1B00",
+ "Swap_Cache": "#C15C17",
+ "Swap_Free": "#2F575E",
+ "Unused": "#EAB839"
+ },
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "decimals": 2,
+ "fieldConfig": {
+ "defaults": {
+ "custom": {}
+ },
+ "overrides": []
+ },
+ "fill": 2,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 0,
+ "y": 32
+ },
+ "hiddenSeries": false,
+ "id": 138,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "max": true,
+ "min": true,
+ "rightSide": false,
+ "show": true,
+ "sideWidth": 350,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "maxPerRow": 6,
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [
+ {
+ "alias": "ShmemHugePages - Memory used by shared memory (shmem) and tmpfs allocated with huge pages",
+ "fill": 0
+ },
+ {
+ "alias": "ShmemHugePages - Memory used by shared memory (shmem) and tmpfs allocated with huge pages",
+ "fill": 0
+ }
+ ],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "node_memory_Mapped_bytes{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "intervalFactor": 2,
+ "legendFormat": "Mapped - Used memory in mapped pages files which have been mmaped, such as libraries",
+ "refId": "A",
+ "step": 4
+ },
+ {
+ "expr": "node_memory_Shmem_bytes{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "intervalFactor": 2,
+ "legendFormat": "Shmem - Used shared memory (shared between several processes, thus including RAM disks)",
+ "refId": "B",
+ "step": 4
+ },
+ {
+ "expr": "node_memory_ShmemHugePages_bytes{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "ShmemHugePages - Memory used by shared memory (shmem) and tmpfs allocated with huge pages",
+ "refId": "C",
+ "step": 4
+ },
+ {
+ "expr": "node_memory_ShmemPmdMapped_bytes{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "ShmemPmdMapped - Ammount of shared (shmem/tmpfs) memory backed by huge pages",
+ "refId": "D",
+ "step": 4
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "Memory Shared and Mapped",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "cumulative"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "bytes",
+ "label": "bytes",
+ "logBase": 1,
+ "max": null,
+ "min": "0",
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {
+ "Active": "#99440A",
+ "Buffers": "#58140C",
+ "Cache": "#6D1F62",
+ "Cached": "#511749",
+ "Committed": "#508642",
+ "Dirty": "#6ED0E0",
+ "Free": "#B7DBAB",
+ "Inactive": "#EA6460",
+ "Mapped": "#052B51",
+ "PageTables": "#0A50A1",
+ "Page_Tables": "#0A50A1",
+ "Slab_Cache": "#EAB839",
+ "Swap": "#BF1B00",
+ "Swap_Cache": "#C15C17",
+ "Total": "#511749",
+ "Total RAM": "#052B51",
+ "Total RAM + Swap": "#052B51",
+ "Total Swap": "#614D93",
+ "VmallocUsed": "#EA6460"
+ },
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "decimals": 2,
+ "fieldConfig": {
+ "defaults": {
+ "custom": {}
+ },
+ "overrides": []
+ },
+ "fill": 2,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 12,
+ "y": 32
+ },
+ "hiddenSeries": false,
+ "id": 131,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "max": true,
+ "min": true,
+ "rightSide": false,
+ "show": true,
+ "sideWidth": null,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "maxPerRow": 2,
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "stack": true,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "node_memory_SUnreclaim_bytes{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "intervalFactor": 2,
+ "legendFormat": "SUnreclaim - Part of Slab, that cannot be reclaimed on memory pressure",
+ "refId": "A",
+ "step": 4
+ },
+ {
+ "expr": "node_memory_SReclaimable_bytes{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "intervalFactor": 2,
+ "legendFormat": "SReclaimable - Part of Slab, that might be reclaimed, such as caches",
+ "refId": "B",
+ "step": 4
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "Memory Slab",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "bytes",
+ "label": "bytes",
+ "logBase": 1,
+ "max": null,
+ "min": "0",
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {
+ "Active": "#99440A",
+ "Buffers": "#58140C",
+ "Cache": "#6D1F62",
+ "Cached": "#511749",
+ "Committed": "#508642",
+ "Dirty": "#6ED0E0",
+ "Free": "#B7DBAB",
+ "Inactive": "#EA6460",
+ "Mapped": "#052B51",
+ "PageTables": "#0A50A1",
+ "Page_Tables": "#0A50A1",
+ "Slab_Cache": "#EAB839",
+ "Swap": "#BF1B00",
+ "Swap_Cache": "#C15C17",
+ "Total": "#511749",
+ "Total RAM": "#052B51",
+ "Total RAM + Swap": "#052B51",
+ "VmallocUsed": "#EA6460"
+ },
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "decimals": 2,
+ "fieldConfig": {
+ "defaults": {
+ "custom": {}
+ },
+ "overrides": []
+ },
+ "fill": 2,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 0,
+ "y": 42
+ },
+ "hiddenSeries": false,
+ "id": 70,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "max": true,
+ "min": true,
+ "rightSide": false,
+ "show": true,
+ "sideWidth": null,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "maxPerRow": 6,
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "node_memory_VmallocChunk_bytes{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "hide": false,
+ "intervalFactor": 2,
+ "legendFormat": "VmallocChunk - Largest contigious block of vmalloc area which is free",
+ "refId": "A",
+ "step": 4
+ },
+ {
+ "expr": "node_memory_VmallocTotal_bytes{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "hide": false,
+ "intervalFactor": 2,
+ "legendFormat": "VmallocTotal - Total size of vmalloc memory area",
+ "refId": "B",
+ "step": 4
+ },
+ {
+ "expr": "node_memory_VmallocUsed_bytes{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "hide": false,
+ "intervalFactor": 2,
+ "legendFormat": "VmallocUsed - Amount of vmalloc area which is used",
+ "refId": "C",
+ "step": 4
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "Memory Vmalloc",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "bytes",
+ "label": "bytes",
+ "logBase": 1,
+ "max": null,
+ "min": "0",
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {
+ "Apps": "#629E51",
+ "Buffers": "#614D93",
+ "Cache": "#6D1F62",
+ "Cached": "#511749",
+ "Committed": "#508642",
+ "Free": "#0A437C",
+ "Hardware Corrupted - Amount of RAM that the kernel identified as corrupted / not working": "#CFFAFF",
+ "Inactive": "#584477",
+ "PageTables": "#0A50A1",
+ "Page_Tables": "#0A50A1",
+ "RAM_Free": "#E0F9D7",
+ "Slab": "#806EB7",
+ "Slab_Cache": "#E0752D",
+ "Swap": "#BF1B00",
+ "Swap_Cache": "#C15C17",
+ "Swap_Free": "#2F575E",
+ "Unused": "#EAB839"
+ },
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "decimals": 2,
+ "fieldConfig": {
+ "defaults": {
+ "custom": {}
+ },
+ "overrides": []
+ },
+ "fill": 2,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 12,
+ "y": 42
+ },
+ "hiddenSeries": false,
+ "id": 159,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "max": true,
+ "min": true,
+ "rightSide": false,
+ "show": true,
+ "sideWidth": 350,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "maxPerRow": 6,
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "node_memory_Bounce_bytes{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "intervalFactor": 2,
+ "legendFormat": "Bounce - Memory used for block device bounce buffers",
+ "refId": "A",
+ "step": 4
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "Memory Bounce",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "cumulative"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "bytes",
+ "label": "bytes",
+ "logBase": 1,
+ "max": null,
+ "min": "0",
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {
+ "Active": "#99440A",
+ "Buffers": "#58140C",
+ "Cache": "#6D1F62",
+ "Cached": "#511749",
+ "Committed": "#508642",
+ "Dirty": "#6ED0E0",
+ "Free": "#B7DBAB",
+ "Inactive": "#EA6460",
+ "Mapped": "#052B51",
+ "PageTables": "#0A50A1",
+ "Page_Tables": "#0A50A1",
+ "Slab_Cache": "#EAB839",
+ "Swap": "#BF1B00",
+ "Swap_Cache": "#C15C17",
+ "Total": "#511749",
+ "Total RAM": "#052B51",
+ "Total RAM + Swap": "#052B51",
+ "VmallocUsed": "#EA6460"
+ },
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "decimals": 2,
+ "fieldConfig": {
+ "defaults": {
+ "custom": {}
+ },
+ "overrides": []
+ },
+ "fill": 2,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 0,
+ "y": 52
+ },
+ "hiddenSeries": false,
+ "id": 129,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "max": true,
+ "min": true,
+ "rightSide": false,
+ "show": true,
+ "sideWidth": null,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "maxPerRow": 6,
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [
+ {
+ "alias": "/.*Inactive *./",
+ "transform": "negative-Y"
+ }
+ ],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "node_memory_AnonHugePages_bytes{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "intervalFactor": 2,
+ "legendFormat": "AnonHugePages - Memory in anonymous huge pages",
+ "refId": "A",
+ "step": 4
+ },
+ {
+ "expr": "node_memory_AnonPages_bytes{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "intervalFactor": 2,
+ "legendFormat": "AnonPages - Memory in user pages not backed by files",
+ "refId": "B",
+ "step": 4
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "Memory Anonymous",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "bytes",
+ "label": "bytes",
+ "logBase": 1,
+ "max": null,
+ "min": "0",
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {
+ "Apps": "#629E51",
+ "Buffers": "#614D93",
+ "Cache": "#6D1F62",
+ "Cached": "#511749",
+ "Committed": "#508642",
+ "Free": "#0A437C",
+ "Hardware Corrupted - Amount of RAM that the kernel identified as corrupted / not working": "#CFFAFF",
+ "Inactive": "#584477",
+ "PageTables": "#0A50A1",
+ "Page_Tables": "#0A50A1",
+ "RAM_Free": "#E0F9D7",
+ "Slab": "#806EB7",
+ "Slab_Cache": "#E0752D",
+ "Swap": "#BF1B00",
+ "Swap_Cache": "#C15C17",
+ "Swap_Free": "#2F575E",
+ "Unused": "#EAB839"
+ },
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "decimals": 2,
+ "fieldConfig": {
+ "defaults": {
+ "custom": {}
+ },
+ "overrides": []
+ },
+ "fill": 2,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 12,
+ "y": 52
+ },
+ "hiddenSeries": false,
+ "id": 160,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "max": true,
+ "min": true,
+ "rightSide": false,
+ "show": true,
+ "sideWidth": 350,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "maxPerRow": 2,
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "node_memory_KernelStack_bytes{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "intervalFactor": 2,
+ "legendFormat": "KernelStack - Kernel memory stack. This is not reclaimable",
+ "refId": "A",
+ "step": 4
+ },
+ {
+ "expr": "node_memory_Percpu_bytes{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "PerCPU - Per CPU memory allocated dynamically by loadable modules",
+ "refId": "B",
+ "step": 4
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "Memory Kernel / CPU",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "cumulative"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "bytes",
+ "label": "bytes",
+ "logBase": 1,
+ "max": null,
+ "min": "0",
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {
+ "Active": "#99440A",
+ "Buffers": "#58140C",
+ "Cache": "#6D1F62",
+ "Cached": "#511749",
+ "Committed": "#508642",
+ "Dirty": "#6ED0E0",
+ "Free": "#B7DBAB",
+ "Inactive": "#EA6460",
+ "Mapped": "#052B51",
+ "PageTables": "#0A50A1",
+ "Page_Tables": "#0A50A1",
+ "Slab_Cache": "#EAB839",
+ "Swap": "#BF1B00",
+ "Swap_Cache": "#C15C17",
+ "Total": "#511749",
+ "Total RAM": "#806EB7",
+ "Total RAM + Swap": "#806EB7",
+ "VmallocUsed": "#EA6460"
+ },
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "decimals": 2,
+ "fieldConfig": {
+ "defaults": {
+ "custom": {}
+ },
+ "overrides": []
+ },
+ "fill": 2,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 0,
+ "y": 62
+ },
+ "hiddenSeries": false,
+ "id": 140,
+ "legend": {
+ "alignAsTable": true,
+ "avg": false,
+ "current": true,
+ "max": true,
+ "min": true,
+ "rightSide": false,
+ "show": true,
+ "sideWidth": null,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "maxPerRow": 6,
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "node_memory_HugePages_Free{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "intervalFactor": 2,
+ "legendFormat": "HugePages_Free - Huge pages in the pool that are not yet allocated",
+ "refId": "A",
+ "step": 4
+ },
+ {
+ "expr": "node_memory_HugePages_Rsvd{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "intervalFactor": 2,
+ "legendFormat": "HugePages_Rsvd - Huge pages for which a commitment to allocate from the pool has been made, but no allocation has yet been made",
+ "refId": "B",
+ "step": 4
+ },
+ {
+ "expr": "node_memory_HugePages_Surp{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "intervalFactor": 2,
+ "legendFormat": "HugePages_Surp - Huge pages in the pool above the value in /proc/sys/vm/nr_hugepages",
+ "refId": "C",
+ "step": 4
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "Memory HugePages Counter",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "short",
+ "label": "pages",
+ "logBase": 1,
+ "max": null,
+ "min": "0",
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": "",
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {
+ "Active": "#99440A",
+ "Buffers": "#58140C",
+ "Cache": "#6D1F62",
+ "Cached": "#511749",
+ "Committed": "#508642",
+ "Dirty": "#6ED0E0",
+ "Free": "#B7DBAB",
+ "Inactive": "#EA6460",
+ "Mapped": "#052B51",
+ "PageTables": "#0A50A1",
+ "Page_Tables": "#0A50A1",
+ "Slab_Cache": "#EAB839",
+ "Swap": "#BF1B00",
+ "Swap_Cache": "#C15C17",
+ "Total": "#511749",
+ "Total RAM": "#806EB7",
+ "Total RAM + Swap": "#806EB7",
+ "VmallocUsed": "#EA6460"
+ },
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "decimals": 2,
+ "fieldConfig": {
+ "defaults": {
+ "custom": {}
+ },
+ "overrides": []
+ },
+ "fill": 2,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 12,
+ "y": 62
+ },
+ "hiddenSeries": false,
+ "id": 71,
+ "legend": {
+ "alignAsTable": true,
+ "avg": false,
+ "current": true,
+ "max": true,
+ "min": true,
+ "rightSide": false,
+ "show": true,
+ "sideWidth": null,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "maxPerRow": 2,
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "node_memory_HugePages_Total{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "intervalFactor": 2,
+ "legendFormat": "HugePages - Total size of the pool of huge pages",
+ "refId": "A",
+ "step": 4
+ },
+ {
+ "expr": "node_memory_Hugepagesize_bytes{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "intervalFactor": 2,
+ "legendFormat": "Hugepagesize - Huge Page size",
+ "refId": "B",
+ "step": 4
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "Memory HugePages Size",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "bytes",
+ "label": "bytes",
+ "logBase": 1,
+ "max": null,
+ "min": "0",
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": "",
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {
+ "Active": "#99440A",
+ "Buffers": "#58140C",
+ "Cache": "#6D1F62",
+ "Cached": "#511749",
+ "Committed": "#508642",
+ "Dirty": "#6ED0E0",
+ "Free": "#B7DBAB",
+ "Inactive": "#EA6460",
+ "Mapped": "#052B51",
+ "PageTables": "#0A50A1",
+ "Page_Tables": "#0A50A1",
+ "Slab_Cache": "#EAB839",
+ "Swap": "#BF1B00",
+ "Swap_Cache": "#C15C17",
+ "Total": "#511749",
+ "Total RAM": "#052B51",
+ "Total RAM + Swap": "#052B51",
+ "VmallocUsed": "#EA6460"
+ },
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "decimals": 2,
+ "fieldConfig": {
+ "defaults": {
+ "custom": {}
+ },
+ "overrides": []
+ },
+ "fill": 2,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 0,
+ "y": 72
+ },
+ "hiddenSeries": false,
+ "id": 128,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": false,
+ "hideEmpty": false,
+ "hideZero": false,
+ "max": true,
+ "min": true,
+ "rightSide": false,
+ "show": true,
+ "sideWidth": null,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "maxPerRow": 6,
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "node_memory_DirectMap1G_bytes{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "intervalFactor": 2,
+ "legendFormat": "DirectMap1G - Amount of pages mapped as this size",
+ "refId": "A",
+ "step": 4
+ },
+ {
+ "expr": "node_memory_DirectMap2M_bytes{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "DirectMap2M - Amount of pages mapped as this size",
+ "refId": "B",
+ "step": 4
+ },
+ {
+ "expr": "node_memory_DirectMap4k_bytes{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "DirectMap4K - Amount of pages mapped as this size",
+ "refId": "C",
+ "step": 4
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "Memory DirectMap",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "bytes",
+ "label": "bytes",
+ "logBase": 1,
+ "max": null,
+ "min": "0",
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {
+ "Apps": "#629E51",
+ "Buffers": "#614D93",
+ "Cache": "#6D1F62",
+ "Cached": "#511749",
+ "Committed": "#508642",
+ "Free": "#0A437C",
+ "Hardware Corrupted - Amount of RAM that the kernel identified as corrupted / not working": "#CFFAFF",
+ "Inactive": "#584477",
+ "PageTables": "#0A50A1",
+ "Page_Tables": "#0A50A1",
+ "RAM_Free": "#E0F9D7",
+ "Slab": "#806EB7",
+ "Slab_Cache": "#E0752D",
+ "Swap": "#BF1B00",
+ "Swap_Cache": "#C15C17",
+ "Swap_Free": "#2F575E",
+ "Unused": "#EAB839"
+ },
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "decimals": 2,
+ "fieldConfig": {
+ "defaults": {
+ "custom": {}
+ },
+ "overrides": []
+ },
+ "fill": 2,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 12,
+ "y": 72
+ },
+ "hiddenSeries": false,
+ "id": 137,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "max": true,
+ "min": true,
+ "rightSide": false,
+ "show": true,
+ "sideWidth": 350,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "maxPerRow": 6,
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "node_memory_Unevictable_bytes{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "intervalFactor": 2,
+ "legendFormat": "Unevictable - Amount of unevictable memory that can't be swapped out for a variety of reasons",
+ "refId": "A",
+ "step": 4
+ },
+ {
+ "expr": "node_memory_Mlocked_bytes{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "intervalFactor": 2,
+ "legendFormat": "MLocked - Size of pages locked to memory using the mlock() system call",
+ "refId": "B",
+ "step": 4
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "Memory Unevictable and MLocked",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "cumulative"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "bytes",
+ "label": "bytes",
+ "logBase": 1,
+ "max": null,
+ "min": "0",
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {
+ "Active": "#99440A",
+ "Buffers": "#58140C",
+ "Cache": "#6D1F62",
+ "Cached": "#511749",
+ "Committed": "#508642",
+ "Dirty": "#6ED0E0",
+ "Free": "#B7DBAB",
+ "Inactive": "#EA6460",
+ "Mapped": "#052B51",
+ "PageTables": "#0A50A1",
+ "Page_Tables": "#0A50A1",
+ "Slab_Cache": "#EAB839",
+ "Swap": "#BF1B00",
+ "Swap_Cache": "#C15C17",
+ "Total": "#511749",
+ "Total RAM": "#052B51",
+ "Total RAM + Swap": "#052B51",
+ "Total Swap": "#614D93",
+ "VmallocUsed": "#EA6460"
+ },
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "decimals": 2,
+ "fieldConfig": {
+ "defaults": {
+ "custom": {}
+ },
+ "overrides": []
+ },
+ "fill": 2,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 0,
+ "y": 82
+ },
+ "hiddenSeries": false,
+ "id": 132,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "max": true,
+ "min": true,
+ "rightSide": false,
+ "show": true,
+ "sideWidth": null,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "maxPerRow": 6,
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "node_memory_NFS_Unstable_bytes{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "intervalFactor": 2,
+ "legendFormat": "NFS Unstable - Memory in NFS pages sent to the server, but not yet commited to the storage",
+ "refId": "A",
+ "step": 4
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "Memory NFS",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "bytes",
+ "label": "bytes",
+ "logBase": 1,
+ "max": null,
+ "min": "0",
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ }
+ ],
+ "repeat": null,
+ "title": "Memory Meminfo",
+ "type": "row"
+ },
+ {
+ "collapsed": true,
+ "datasource": "${DS_PROMETHEUS}",
+ "gridPos": {
+ "h": 1,
+ "w": 24,
+ "x": 0,
+ "y": 60
+ },
+ "id": 267,
+ "panels": [
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "fill": 2,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 0,
+ "y": 23
+ },
+ "hiddenSeries": false,
+ "id": 176,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "max": true,
+ "min": true,
+ "rightSide": false,
+ "show": true,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "maxPerRow": 6,
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [
+ {
+ "alias": "/.*out/",
+ "transform": "negative-Y"
+ }
+ ],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "irate(node_vmstat_pgpgin{instance=~\"$node:$port\",job=~\"$job\"}[5m])",
+ "format": "time_series",
+ "intervalFactor": 2,
+ "legendFormat": "Pagesin - Page in operations",
+ "refId": "A",
+ "step": 4
+ },
+ {
+ "expr": "irate(node_vmstat_pgpgout{instance=~\"$node:$port\",job=~\"$job\"}[5m])",
+ "format": "time_series",
+ "intervalFactor": 2,
+ "legendFormat": "Pagesout - Page out operations",
+ "refId": "B",
+ "step": 4
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "Memory Pages In / Out",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "short",
+ "label": "pages out (-) / in (+)",
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "fill": 2,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 12,
+ "y": 23
+ },
+ "hiddenSeries": false,
+ "id": 22,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "max": true,
+ "min": true,
+ "rightSide": false,
+ "show": true,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "maxPerRow": 6,
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [
+ {
+ "alias": "/.*out/",
+ "transform": "negative-Y"
+ }
+ ],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "irate(node_vmstat_pswpin{instance=~\"$node:$port\",job=~\"$job\"}[5m])",
+ "format": "time_series",
+ "intervalFactor": 2,
+ "legendFormat": "Pswpin - Pages swapped in",
+ "refId": "A",
+ "step": 4
+ },
+ {
+ "expr": "irate(node_vmstat_pswpout{instance=~\"$node:$port\",job=~\"$job\"}[5m])",
+ "format": "time_series",
+ "intervalFactor": 2,
+ "legendFormat": "Pswpout - Pages swapped out",
+ "refId": "B",
+ "step": 4
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "Memory Pages Swap In / Out",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "short",
+ "label": "pages out (-) / in (+)",
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {
+ "Apps": "#629E51",
+ "Buffers": "#614D93",
+ "Cache": "#6D1F62",
+ "Cached": "#511749",
+ "Committed": "#508642",
+ "Free": "#0A437C",
+ "Hardware Corrupted - Amount of RAM that the kernel identified as corrupted / not working": "#CFFAFF",
+ "Inactive": "#584477",
+ "PageTables": "#0A50A1",
+ "Page_Tables": "#0A50A1",
+ "RAM_Free": "#E0F9D7",
+ "Slab": "#806EB7",
+ "Slab_Cache": "#E0752D",
+ "Swap": "#BF1B00",
+ "Swap_Cache": "#C15C17",
+ "Swap_Free": "#2F575E",
+ "Unused": "#EAB839"
+ },
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "decimals": 2,
+ "fill": 2,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 0,
+ "y": 33
+ },
+ "hiddenSeries": false,
+ "id": 175,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "max": true,
+ "min": true,
+ "rightSide": false,
+ "show": true,
+ "sideWidth": 350,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "maxPerRow": 6,
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [
+ {
+ "alias": "Pgfault - Page major and minor fault operations",
+ "fill": 0,
+ "stack": false
+ }
+ ],
+ "spaceLength": 10,
+ "stack": true,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "irate(node_vmstat_pgfault{instance=~\"$node:$port\",job=~\"$job\"}[5m])",
+ "format": "time_series",
+ "intervalFactor": 2,
+ "legendFormat": "Pgfault - Page major and minor fault operations",
+ "refId": "A",
+ "step": 4
+ },
+ {
+ "expr": "irate(node_vmstat_pgmajfault{instance=~\"$node:$port\",job=~\"$job\"}[5m])",
+ "format": "time_series",
+ "intervalFactor": 2,
+ "legendFormat": "Pgmajfault - Major page fault operations",
+ "refId": "B",
+ "step": 4
+ },
+ {
+ "expr": "irate(node_vmstat_pgfault{instance=~\"$node:$port\",job=~\"$job\"}[5m]) - irate(node_vmstat_pgmajfault{instance=~\"$node:$port\",job=~\"$job\"}[5m])",
+ "format": "time_series",
+ "intervalFactor": 2,
+ "legendFormat": "Pgminfault - Minor page fault operations",
+ "refId": "C",
+ "step": 4
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "Memory Page Faults",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "cumulative"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "short",
+ "label": "faults",
+ "logBase": 1,
+ "max": null,
+ "min": "0",
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {
+ "Active": "#99440A",
+ "Buffers": "#58140C",
+ "Cache": "#6D1F62",
+ "Cached": "#511749",
+ "Committed": "#508642",
+ "Dirty": "#6ED0E0",
+ "Free": "#B7DBAB",
+ "Inactive": "#EA6460",
+ "Mapped": "#052B51",
+ "PageTables": "#0A50A1",
+ "Page_Tables": "#0A50A1",
+ "Slab_Cache": "#EAB839",
+ "Swap": "#BF1B00",
+ "Swap_Cache": "#C15C17",
+ "Total": "#511749",
+ "Total RAM": "#052B51",
+ "Total RAM + Swap": "#052B51",
+ "Total Swap": "#614D93",
+ "VmallocUsed": "#EA6460"
+ },
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "decimals": 2,
+ "fill": 2,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 12,
+ "y": 33
+ },
+ "hiddenSeries": false,
+ "id": 307,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "max": true,
+ "min": true,
+ "rightSide": false,
+ "show": true,
+ "sideWidth": null,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "maxPerRow": 6,
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "irate(node_vmstat_oom_kill{instance=~\"$node:$port\",job=~\"$job\"}[5m])",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "oom killer invocations ",
+ "refId": "A",
+ "step": 4
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "OOM Killer",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "short",
+ "label": "counter",
+ "logBase": 1,
+ "max": null,
+ "min": "0",
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ }
+ ],
+ "repeat": null,
+ "title": "Memory Vmstat",
+ "type": "row"
+ },
+ {
+ "collapsed": true,
+ "datasource": "${DS_PROMETHEUS}",
+ "gridPos": {
+ "h": 1,
+ "w": 24,
+ "x": 0,
+ "y": 61
+ },
+ "id": 293,
+ "panels": [
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "description": "",
+ "fill": 2,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 0,
+ "y": 24
+ },
+ "hiddenSeries": false,
+ "id": 260,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "max": true,
+ "min": true,
+ "show": true,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [
+ {
+ "alias": "/.*Variation*./",
+ "color": "#890F02"
+ }
+ ],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "node_timex_estimated_error_seconds{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "hide": false,
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "Estimated error in seconds",
+ "refId": "A",
+ "step": 240
+ },
+ {
+ "expr": "node_timex_offset_seconds{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "hide": false,
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "Time offset in between local system and reference clock",
+ "refId": "B",
+ "step": 240
+ },
+ {
+ "expr": "node_timex_maxerror_seconds{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "hide": false,
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "Maximum error in seconds",
+ "refId": "C",
+ "step": 240
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "Time Syncronized Drift",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "s",
+ "label": "seconds",
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": "counter",
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "description": "",
+ "fill": 2,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 12,
+ "y": 24
+ },
+ "hiddenSeries": false,
+ "id": 291,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "max": true,
+ "min": true,
+ "show": true,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "node_timex_loop_time_constant{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "Phase-locked loop time adjust",
+ "refId": "A",
+ "step": 240
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "Time PLL Adjust",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "short",
+ "label": "counter",
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "description": "",
+ "fill": 2,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 0,
+ "y": 34
+ },
+ "hiddenSeries": false,
+ "id": 168,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "max": true,
+ "min": true,
+ "show": true,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [
+ {
+ "alias": "/.*Variation*./",
+ "color": "#890F02"
+ }
+ ],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "node_timex_sync_status{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "Is clock synchronized to a reliable server (1 = yes, 0 = no)",
+ "refId": "A",
+ "step": 240
+ },
+ {
+ "expr": "node_timex_frequency_adjustment_ratio{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "Local clock frequency adjustment",
+ "refId": "B",
+ "step": 240
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "Time Syncronized Status",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "short",
+ "label": "counter",
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "description": "",
+ "fill": 2,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 12,
+ "y": 34
+ },
+ "hiddenSeries": false,
+ "id": 294,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "max": true,
+ "min": true,
+ "show": true,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "node_timex_tick_seconds{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "Seconds between clock ticks",
+ "refId": "A",
+ "step": 240
+ },
+ {
+ "expr": "node_timex_tai_offset_seconds{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "International Atomic Time (TAI) offset",
+ "refId": "B",
+ "step": 240
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "Time Misc",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "s",
+ "label": "seconds",
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ }
+ ],
+ "title": "System Timesync",
+ "type": "row"
+ },
+ {
+ "collapsed": true,
+ "datasource": "${DS_PROMETHEUS}",
+ "gridPos": {
+ "h": 1,
+ "w": 24,
+ "x": 0,
+ "y": 62
+ },
+ "id": 312,
+ "panels": [
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "fill": 2,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 0,
+ "y": 7
+ },
+ "hiddenSeries": false,
+ "id": 62,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "max": true,
+ "min": true,
+ "show": true,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "maxPerRow": 6,
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "node_procs_blocked{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "intervalFactor": 2,
+ "legendFormat": "Processes blocked waiting for I/O to complete",
+ "refId": "A",
+ "step": 240
+ },
+ {
+ "expr": "node_procs_running{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "intervalFactor": 2,
+ "legendFormat": "Processes in runnable state",
+ "refId": "B",
+ "step": 240
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "Processes Status",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "short",
+ "label": "counter",
+ "logBase": 1,
+ "max": null,
+ "min": "0",
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "fill": 2,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 12,
+ "y": 7
+ },
+ "hiddenSeries": false,
+ "id": 315,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "max": true,
+ "min": true,
+ "show": true,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "maxPerRow": 6,
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "stack": true,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "node_processes_state{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "{{ state }}",
+ "refId": "A",
+ "step": 240
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "Processes State",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "short",
+ "label": "counter",
+ "logBase": 1,
+ "max": null,
+ "min": "0",
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "fill": 2,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 0,
+ "y": 17
+ },
+ "hiddenSeries": false,
+ "id": 148,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "max": true,
+ "min": true,
+ "show": true,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "maxPerRow": 6,
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "irate(node_forks_total{instance=~\"$node:$port\",job=~\"$job\"}[5m])",
+ "format": "time_series",
+ "hide": false,
+ "intervalFactor": 2,
+ "legendFormat": "Processes forks second",
+ "refId": "A",
+ "step": 240
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "Processes Forks",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "short",
+ "label": "forks / sec",
+ "logBase": 1,
+ "max": null,
+ "min": "0",
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "fill": 2,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 12,
+ "y": 17
+ },
+ "hiddenSeries": false,
+ "id": 149,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "max": true,
+ "min": true,
+ "show": true,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [
+ {
+ "alias": "/.*Max.*/",
+ "fill": 0
+ }
+ ],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "irate(process_virtual_memory_bytes{instance=~\"$node:$port\",job=~\"$job\"}[5m])",
+ "hide": false,
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "Processes virtual memory size in bytes",
+ "refId": "A",
+ "step": 240
+ },
+ {
+ "expr": "process_resident_memory_max_bytes{instance=~\"$node:$port\",job=~\"$job\"}",
+ "hide": false,
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "Maximum amount of virtual memory available in bytes",
+ "refId": "B",
+ "step": 240
+ },
+ {
+ "expr": "irate(process_virtual_memory_bytes{instance=~\"$node:$port\",job=~\"$job\"}[5m])",
+ "hide": false,
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "Processes virtual memory size in bytes",
+ "refId": "C",
+ "step": 240
+ },
+ {
+ "expr": "irate(process_virtual_memory_max_bytes{instance=~\"$node:$port\",job=~\"$job\"}[5m])",
+ "hide": false,
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "Maximum amount of virtual memory available in bytes",
+ "refId": "D",
+ "step": 240
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "Processes Memory",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "decbytes",
+ "label": "bytes",
+ "logBase": 1,
+ "max": null,
+ "min": "0",
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "fill": 2,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 0,
+ "y": 27
+ },
+ "hiddenSeries": false,
+ "id": 313,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "max": true,
+ "min": true,
+ "show": true,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "maxPerRow": 6,
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [
+ {
+ "alias": "PIDs limit",
+ "color": "#F2495C",
+ "fill": 0
+ }
+ ],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "node_processes_pids{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "Number of PIDs",
+ "refId": "A",
+ "step": 240
+ },
+ {
+ "expr": "node_processes_max_processes{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "PIDs limit",
+ "refId": "B",
+ "step": 240
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "PIDs Number and Limit",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "short",
+ "label": "counter",
+ "logBase": 1,
+ "max": null,
+ "min": "0",
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "fill": 2,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 12,
+ "y": 27
+ },
+ "hiddenSeries": false,
+ "id": 305,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "max": true,
+ "min": true,
+ "show": true,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "maxPerRow": 6,
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [
+ {
+ "alias": "/.*waiting.*/",
+ "transform": "negative-Y"
+ }
+ ],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "irate(node_schedstat_running_seconds_total{instance=~\"$node:$port\",job=~\"$job\"}[5m])",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "CPU {{ cpu }} - seconds spent running a process",
+ "refId": "A",
+ "step": 240
+ },
+ {
+ "expr": "irate(node_schedstat_waiting_seconds_total{instance=~\"$node:$port\",job=~\"$job\"}[5m])",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "CPU {{ cpu }} - seconds spent by processing waiting for this CPU",
+ "refId": "B",
+ "step": 240
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "Process schdeule stats Running / Waiting",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "s",
+ "label": "seconds",
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "fill": 2,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 12,
+ "y": 37
+ },
+ "hiddenSeries": false,
+ "id": 314,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "max": true,
+ "min": true,
+ "show": true,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "maxPerRow": 6,
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [
+ {
+ "alias": "Threads limit",
+ "color": "#F2495C",
+ "fill": 0
+ }
+ ],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "node_processes_threads{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "Allocated threads",
+ "refId": "A",
+ "step": 240
+ },
+ {
+ "expr": "node_processes_max_threads{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "Threads limit",
+ "refId": "B",
+ "step": 240
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "Threads Number and Limit",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "short",
+ "label": "counter",
+ "logBase": 1,
+ "max": null,
+ "min": "0",
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ }
+ ],
+ "title": "System Processes",
+ "type": "row"
+ },
+ {
+ "collapsed": true,
+ "datasource": "${DS_PROMETHEUS}",
+ "gridPos": {
+ "h": 1,
+ "w": 24,
+ "x": 0,
+ "y": 63
+ },
+ "id": 269,
+ "panels": [
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "fieldConfig": {
+ "defaults": {
+ "custom": {}
+ },
+ "overrides": []
+ },
+ "fill": 2,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 0,
+ "y": 64
+ },
+ "hiddenSeries": false,
+ "id": 8,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "max": true,
+ "min": true,
+ "show": true,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "maxPerRow": 6,
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "repeat": null,
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "irate(node_context_switches_total{instance=~\"$node:$port\",job=~\"$job\"}[5m])",
+ "format": "time_series",
+ "intervalFactor": 2,
+ "legendFormat": "Context switches",
+ "refId": "A",
+ "step": 240
+ },
+ {
+ "expr": "irate(node_intr_total{instance=~\"$node:$port\",job=~\"$job\"}[5m])",
+ "format": "time_series",
+ "hide": false,
+ "intervalFactor": 2,
+ "legendFormat": "Interrupts",
+ "refId": "B",
+ "step": 240
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "Context Switches / Interrupts",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "short",
+ "label": "counter",
+ "logBase": 1,
+ "max": null,
+ "min": "0",
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "fieldConfig": {
+ "defaults": {
+ "custom": {}
+ },
+ "overrides": []
+ },
+ "fill": 2,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 12,
+ "y": 64
+ },
+ "hiddenSeries": false,
+ "id": 7,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "max": true,
+ "min": true,
+ "show": true,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "maxPerRow": 6,
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "repeat": null,
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "node_load1{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "intervalFactor": 4,
+ "legendFormat": "Load 1m",
+ "refId": "A",
+ "step": 480
+ },
+ {
+ "expr": "node_load5{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "intervalFactor": 4,
+ "legendFormat": "Load 5m",
+ "refId": "B",
+ "step": 480
+ },
+ {
+ "expr": "node_load15{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "intervalFactor": 4,
+ "legendFormat": "Load 15m",
+ "refId": "C",
+ "step": 480
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "System Load",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "short",
+ "label": "counter",
+ "logBase": 1,
+ "max": null,
+ "min": "0",
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "fieldConfig": {
+ "defaults": {
+ "custom": {}
+ },
+ "overrides": []
+ },
+ "fill": 2,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 0,
+ "y": 74
+ },
+ "hiddenSeries": false,
+ "id": 259,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "max": true,
+ "min": true,
+ "show": true,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [
+ {
+ "alias": "/.*Critical*./",
+ "color": "#E24D42",
+ "fill": 0
+ },
+ {
+ "alias": "/.*Max*./",
+ "color": "#EF843C",
+ "fill": 0
+ }
+ ],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "irate(node_interrupts_total{instance=~\"$node:$port\",job=~\"$job\"}[5m])",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "{{ type }} - {{ info }}",
+ "refId": "A",
+ "step": 240
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "Interrupts Detail",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "short",
+ "label": "counter",
+ "logBase": 1,
+ "max": null,
+ "min": "0",
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "fieldConfig": {
+ "defaults": {
+ "custom": {}
+ },
+ "overrides": []
+ },
+ "fill": 2,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 12,
+ "y": 74
+ },
+ "hiddenSeries": false,
+ "id": 306,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "max": true,
+ "min": true,
+ "show": true,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "maxPerRow": 6,
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "irate(node_schedstat_timeslices_total{instance=~\"$node:$port\",job=~\"$job\"}[5m])",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "CPU {{ cpu }}",
+ "refId": "A",
+ "step": 240
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "Schedule timeslices executed by each cpu",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "short",
+ "label": "counter",
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "fieldConfig": {
+ "defaults": {
+ "custom": {}
+ },
+ "overrides": []
+ },
+ "fill": 2,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 0,
+ "y": 84
+ },
+ "hiddenSeries": false,
+ "id": 151,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "max": true,
+ "min": true,
+ "show": true,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "maxPerRow": 6,
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "node_entropy_available_bits{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "intervalFactor": 2,
+ "legendFormat": "Entropy available to random number generators",
+ "refId": "A",
+ "step": 240
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "Entropy",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "short",
+ "label": "counter",
+ "logBase": 1,
+ "max": null,
+ "min": "0",
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "fieldConfig": {
+ "defaults": {
+ "custom": {}
+ },
+ "overrides": []
+ },
+ "fill": 2,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 12,
+ "y": 84
+ },
+ "hiddenSeries": false,
+ "id": 308,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "max": true,
+ "min": true,
+ "show": true,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "maxPerRow": 6,
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "irate(process_cpu_seconds_total{instance=~\"$node:$port\",job=~\"$job\"}[5m])",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "Time spent",
+ "refId": "A",
+ "step": 240
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "CPU time spent in user and system contexts",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "s",
+ "label": "seconds",
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "fieldConfig": {
+ "defaults": {
+ "custom": {}
+ },
+ "overrides": []
+ },
+ "fill": 2,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 0,
+ "y": 94
+ },
+ "hiddenSeries": false,
+ "id": 64,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "max": true,
+ "min": true,
+ "show": true,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [
+ {
+ "alias": "/.*Max*./",
+ "color": "#890F02",
+ "fill": 0
+ }
+ ],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "process_max_fds{instance=~\"$node:$port\",job=~\"$job\"}",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "Maximum open file descriptors",
+ "refId": "A",
+ "step": 240
+ },
+ {
+ "expr": "process_open_fds{instance=~\"$node:$port\",job=~\"$job\"}",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "Open file descriptors",
+ "refId": "B",
+ "step": 240
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "File Descriptors",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "short",
+ "label": "counter",
+ "logBase": 1,
+ "max": null,
+ "min": "0",
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ }
+ ],
+ "repeat": null,
+ "title": "System Misc",
+ "type": "row"
+ },
+ {
+ "collapsed": true,
+ "datasource": "${DS_PROMETHEUS}",
+ "gridPos": {
+ "h": 1,
+ "w": 24,
+ "x": 0,
+ "y": 64
+ },
+ "id": 304,
+ "panels": [
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "fieldConfig": {
+ "defaults": {
+ "custom": {}
+ },
+ "overrides": []
+ },
+ "fill": 2,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 0,
+ "y": 17
+ },
+ "hiddenSeries": false,
+ "id": 158,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "max": true,
+ "min": true,
+ "show": true,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [
+ {
+ "alias": "/.*Critical*./",
+ "color": "#E24D42",
+ "fill": 0
+ },
+ {
+ "alias": "/.*Max*./",
+ "color": "#EF843C",
+ "fill": 0
+ }
+ ],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "node_hwmon_temp_celsius{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "{{ chip }} {{ sensor }} temp",
+ "refId": "A",
+ "step": 240
+ },
+ {
+ "expr": "node_hwmon_temp_crit_alarm_celsius{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "hide": true,
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "{{ chip }} {{ sensor }} Critical Alarm",
+ "refId": "B",
+ "step": 240
+ },
+ {
+ "expr": "node_hwmon_temp_crit_celsius{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "{{ chip }} {{ sensor }} Critical",
+ "refId": "C",
+ "step": 240
+ },
+ {
+ "expr": "node_hwmon_temp_crit_hyst_celsius{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "hide": true,
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "{{ chip }} {{ sensor }} Critical Historical",
+ "refId": "D",
+ "step": 240
+ },
+ {
+ "expr": "node_hwmon_temp_max_celsius{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "hide": true,
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "{{ chip }} {{ sensor }} Max",
+ "refId": "E",
+ "step": 240
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "Hardware temperature monitor",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "celsius",
+ "label": "temperature",
+ "logBase": 1,
+ "max": null,
+ "min": "0",
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "fieldConfig": {
+ "defaults": {
+ "custom": {}
+ },
+ "overrides": []
+ },
+ "fill": 2,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 12,
+ "y": 17
+ },
+ "hiddenSeries": false,
+ "id": 300,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "max": true,
+ "min": true,
+ "show": true,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [
+ {
+ "alias": "/.*Max*./",
+ "color": "#EF843C",
+ "fill": 0
+ }
+ ],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "node_cooling_device_cur_state{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "hide": false,
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "Current {{ name }} in {{ type }}",
+ "refId": "A",
+ "step": 240
+ },
+ {
+ "expr": "node_cooling_device_max_state{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "Max {{ name }} in {{ type }}",
+ "refId": "B",
+ "step": 240
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "Throttle cooling device",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "short",
+ "label": "counter",
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "fieldConfig": {
+ "defaults": {
+ "custom": {}
+ },
+ "overrides": []
+ },
+ "fill": 2,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 0,
+ "y": 27
+ },
+ "hiddenSeries": false,
+ "id": 302,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "max": true,
+ "min": true,
+ "show": true,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "node_power_supply_online{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "hide": false,
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "{{ power_supply }} online",
+ "refId": "A",
+ "step": 240
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "Power supply",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "short",
+ "label": "counter",
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ }
+ ],
+ "title": "Hardware Misc",
+ "type": "row"
+ },
+ {
+ "collapsed": true,
+ "datasource": "${DS_PROMETHEUS}",
+ "gridPos": {
+ "h": 1,
+ "w": 24,
+ "x": 0,
+ "y": 65
+ },
+ "id": 296,
+ "panels": [
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "fill": 2,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 0,
+ "y": 10
+ },
+ "hiddenSeries": false,
+ "id": 297,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "max": true,
+ "min": true,
+ "show": true,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "irate(node_systemd_socket_accepted_connections_total{instance=~\"$node:$port\",job=~\"$job\"}[5m])",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "{{ name }} Connections",
+ "refId": "A",
+ "step": 240
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "Systemd Sockets",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "short",
+ "label": "counter",
+ "logBase": 1,
+ "max": null,
+ "min": "0",
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "fill": 2,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 12,
+ "y": 10
+ },
+ "hiddenSeries": false,
+ "id": 298,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "max": true,
+ "min": true,
+ "show": true,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [
+ {
+ "alias": "Failed",
+ "color": "#F2495C"
+ },
+ {
+ "alias": "Inactive",
+ "color": "#FF9830"
+ },
+ {
+ "alias": "Active",
+ "color": "#73BF69"
+ },
+ {
+ "alias": "Deactivating",
+ "color": "#FFCB7D"
+ },
+ {
+ "alias": "Activating",
+ "color": "#C8F2C2"
+ }
+ ],
+ "spaceLength": 10,
+ "stack": true,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "node_systemd_units{instance=~\"$node:$port\",job=~\"$job\",state=\"activating\"}",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "Activating",
+ "refId": "A",
+ "step": 240
+ },
+ {
+ "expr": "node_systemd_units{instance=~\"$node:$port\",job=~\"$job\",state=\"active\"}",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "Active",
+ "refId": "B",
+ "step": 240
+ },
+ {
+ "expr": "node_systemd_units{instance=~\"$node:$port\",job=~\"$job\",state=\"deactivating\"}",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "Deactivating",
+ "refId": "C",
+ "step": 240
+ },
+ {
+ "expr": "node_systemd_units{instance=~\"$node:$port\",job=~\"$job\",state=\"failed\"}",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "Failed",
+ "refId": "D",
+ "step": 240
+ },
+ {
+ "expr": "node_systemd_units{instance=~\"$node:$port\",job=~\"$job\",state=\"inactive\"}",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "Inactive",
+ "refId": "E",
+ "step": 240
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "Systemd Units State",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "short",
+ "label": "counter",
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ }
+ ],
+ "title": "Systemd",
+ "type": "row"
+ },
+ {
+ "collapsed": true,
+ "datasource": "${DS_PROMETHEUS}",
+ "gridPos": {
+ "h": 1,
+ "w": 24,
+ "x": 0,
+ "y": 66
+ },
+ "id": 270,
+ "panels": [
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "description": "",
+ "fill": 2,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 0,
+ "y": 29
+ },
+ "hiddenSeries": false,
+ "id": 9,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "hideZero": true,
+ "max": true,
+ "min": true,
+ "rightSide": false,
+ "show": true,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "maxPerRow": 6,
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "repeat": null,
+ "seriesOverrides": [
+ {
+ "alias": "/.*Read.*/",
+ "transform": "negative-Y"
+ },
+ {
+ "alias": "/.*sda_.*/",
+ "color": "#7EB26D"
+ },
+ {
+ "alias": "/.*sdb_.*/",
+ "color": "#EAB839"
+ },
+ {
+ "alias": "/.*sdc_.*/",
+ "color": "#6ED0E0"
+ },
+ {
+ "alias": "/.*sdd_.*/",
+ "color": "#EF843C"
+ },
+ {
+ "alias": "/.*sde_.*/",
+ "color": "#E24D42"
+ },
+ {
+ "alias": "/.*sda1.*/",
+ "color": "#584477"
+ },
+ {
+ "alias": "/.*sda2_.*/",
+ "color": "#BA43A9"
+ },
+ {
+ "alias": "/.*sda3_.*/",
+ "color": "#F4D598"
+ },
+ {
+ "alias": "/.*sdb1.*/",
+ "color": "#0A50A1"
+ },
+ {
+ "alias": "/.*sdb2.*/",
+ "color": "#BF1B00"
+ },
+ {
+ "alias": "/.*sdb3.*/",
+ "color": "#E0752D"
+ },
+ {
+ "alias": "/.*sdc1.*/",
+ "color": "#962D82"
+ },
+ {
+ "alias": "/.*sdc2.*/",
+ "color": "#614D93"
+ },
+ {
+ "alias": "/.*sdc3.*/",
+ "color": "#9AC48A"
+ },
+ {
+ "alias": "/.*sdd1.*/",
+ "color": "#65C5DB"
+ },
+ {
+ "alias": "/.*sdd2.*/",
+ "color": "#F9934E"
+ },
+ {
+ "alias": "/.*sdd3.*/",
+ "color": "#EA6460"
+ },
+ {
+ "alias": "/.*sde1.*/",
+ "color": "#E0F9D7"
+ },
+ {
+ "alias": "/.*sdd2.*/",
+ "color": "#FCEACA"
+ },
+ {
+ "alias": "/.*sde3.*/",
+ "color": "#F9E2D2"
+ }
+ ],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "irate(node_disk_reads_completed_total{instance=~\"$node:$port\",job=~\"$job\"}[5m])",
+ "intervalFactor": 4,
+ "legendFormat": "{{device}} - Reads completed",
+ "refId": "A",
+ "step": 8
+ },
+ {
+ "expr": "irate(node_disk_writes_completed_total{instance=~\"$node:$port\",job=~\"$job\"}[5m])",
+ "intervalFactor": 2,
+ "legendFormat": "{{device}} - Writes completed",
+ "refId": "B",
+ "step": 4
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "Disk IOps Completed",
+ "tooltip": {
+ "shared": false,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "iops",
+ "label": "IO read (-) / write (+)",
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "description": "",
+ "fill": 2,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 12,
+ "y": 29
+ },
+ "hiddenSeries": false,
+ "id": 33,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "hideZero": true,
+ "max": true,
+ "min": true,
+ "rightSide": false,
+ "show": true,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "maxPerRow": 6,
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [
+ {
+ "alias": "/.*Read.*/",
+ "transform": "negative-Y"
+ },
+ {
+ "alias": "/.*sda_.*/",
+ "color": "#7EB26D"
+ },
+ {
+ "alias": "/.*sdb_.*/",
+ "color": "#EAB839"
+ },
+ {
+ "alias": "/.*sdc_.*/",
+ "color": "#6ED0E0"
+ },
+ {
+ "alias": "/.*sdd_.*/",
+ "color": "#EF843C"
+ },
+ {
+ "alias": "/.*sde_.*/",
+ "color": "#E24D42"
+ },
+ {
+ "alias": "/.*sda1.*/",
+ "color": "#584477"
+ },
+ {
+ "alias": "/.*sda2_.*/",
+ "color": "#BA43A9"
+ },
+ {
+ "alias": "/.*sda3_.*/",
+ "color": "#F4D598"
+ },
+ {
+ "alias": "/.*sdb1.*/",
+ "color": "#0A50A1"
+ },
+ {
+ "alias": "/.*sdb2.*/",
+ "color": "#BF1B00"
+ },
+ {
+ "alias": "/.*sdb3.*/",
+ "color": "#E0752D"
+ },
+ {
+ "alias": "/.*sdc1.*/",
+ "color": "#962D82"
+ },
+ {
+ "alias": "/.*sdc2.*/",
+ "color": "#614D93"
+ },
+ {
+ "alias": "/.*sdc3.*/",
+ "color": "#9AC48A"
+ },
+ {
+ "alias": "/.*sdd1.*/",
+ "color": "#65C5DB"
+ },
+ {
+ "alias": "/.*sdd2.*/",
+ "color": "#F9934E"
+ },
+ {
+ "alias": "/.*sdd3.*/",
+ "color": "#EA6460"
+ },
+ {
+ "alias": "/.*sde1.*/",
+ "color": "#E0F9D7"
+ },
+ {
+ "alias": "/.*sdd2.*/",
+ "color": "#FCEACA"
+ },
+ {
+ "alias": "/.*sde3.*/",
+ "color": "#F9E2D2"
+ }
+ ],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "irate(node_disk_read_bytes_total{instance=~\"$node:$port\",job=~\"$job\"}[5m])",
+ "format": "time_series",
+ "intervalFactor": 4,
+ "legendFormat": "{{device}} - Read bytes",
+ "refId": "A",
+ "step": 8
+ },
+ {
+ "expr": "irate(node_disk_written_bytes_total{instance=~\"$node:$port\",job=~\"$job\"}[5m])",
+ "format": "time_series",
+ "intervalFactor": 2,
+ "legendFormat": "{{device}} - Written bytes",
+ "refId": "B",
+ "step": 4
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "Disk R/W Data",
+ "tooltip": {
+ "shared": false,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "Bps",
+ "label": "bytes read (-) / write (+)",
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "description": "",
+ "fill": 3,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 0,
+ "y": 39
+ },
+ "hiddenSeries": false,
+ "id": 37,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "hideZero": true,
+ "max": true,
+ "min": true,
+ "rightSide": false,
+ "show": true,
+ "sort": "current",
+ "sortDesc": true,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "maxPerRow": 6,
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [
+ {
+ "alias": "/.*Read.*/",
+ "transform": "negative-Y"
+ },
+ {
+ "alias": "/.*sda_.*/",
+ "color": "#7EB26D"
+ },
+ {
+ "alias": "/.*sdb_.*/",
+ "color": "#EAB839"
+ },
+ {
+ "alias": "/.*sdc_.*/",
+ "color": "#6ED0E0"
+ },
+ {
+ "alias": "/.*sdd_.*/",
+ "color": "#EF843C"
+ },
+ {
+ "alias": "/.*sde_.*/",
+ "color": "#E24D42"
+ },
+ {
+ "alias": "/.*sda1.*/",
+ "color": "#584477"
+ },
+ {
+ "alias": "/.*sda2_.*/",
+ "color": "#BA43A9"
+ },
+ {
+ "alias": "/.*sda3_.*/",
+ "color": "#F4D598"
+ },
+ {
+ "alias": "/.*sdb1.*/",
+ "color": "#0A50A1"
+ },
+ {
+ "alias": "/.*sdb2.*/",
+ "color": "#BF1B00"
+ },
+ {
+ "alias": "/.*sdb3.*/",
+ "color": "#E0752D"
+ },
+ {
+ "alias": "/.*sdc1.*/",
+ "color": "#962D82"
+ },
+ {
+ "alias": "/.*sdc2.*/",
+ "color": "#614D93"
+ },
+ {
+ "alias": "/.*sdc3.*/",
+ "color": "#9AC48A"
+ },
+ {
+ "alias": "/.*sdd1.*/",
+ "color": "#65C5DB"
+ },
+ {
+ "alias": "/.*sdd2.*/",
+ "color": "#F9934E"
+ },
+ {
+ "alias": "/.*sdd3.*/",
+ "color": "#EA6460"
+ },
+ {
+ "alias": "/.*sde1.*/",
+ "color": "#E0F9D7"
+ },
+ {
+ "alias": "/.*sdd2.*/",
+ "color": "#FCEACA"
+ },
+ {
+ "alias": "/.*sde3.*/",
+ "color": "#F9E2D2"
+ }
+ ],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "irate(node_disk_read_time_seconds_total{instance=~\"$node:$port\",job=~\"$job\"}[5m])",
+ "hide": false,
+ "intervalFactor": 4,
+ "legendFormat": "{{device}} - Read time",
+ "refId": "A",
+ "step": 8
+ },
+ {
+ "expr": "irate(node_disk_write_time_seconds_total{instance=~\"$node:$port\",job=~\"$job\"}[5m])",
+ "hide": false,
+ "intervalFactor": 2,
+ "legendFormat": "{{device}} - Write time",
+ "refId": "B",
+ "step": 4
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "Disk R/W Time",
+ "tooltip": {
+ "shared": false,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "s",
+ "label": "time. read (-) / write (+)",
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "description": "",
+ "fill": 2,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 12,
+ "y": 39
+ },
+ "hiddenSeries": false,
+ "id": 35,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "hideZero": true,
+ "max": true,
+ "min": true,
+ "rightSide": false,
+ "show": true,
+ "sort": "current",
+ "sortDesc": true,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "maxPerRow": 6,
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [
+ {
+ "alias": "/.*sda_.*/",
+ "color": "#7EB26D"
+ },
+ {
+ "alias": "/.*sdb_.*/",
+ "color": "#EAB839"
+ },
+ {
+ "alias": "/.*sdc_.*/",
+ "color": "#6ED0E0"
+ },
+ {
+ "alias": "/.*sdd_.*/",
+ "color": "#EF843C"
+ },
+ {
+ "alias": "/.*sde_.*/",
+ "color": "#E24D42"
+ },
+ {
+ "alias": "/.*sda1.*/",
+ "color": "#584477"
+ },
+ {
+ "alias": "/.*sda2_.*/",
+ "color": "#BA43A9"
+ },
+ {
+ "alias": "/.*sda3_.*/",
+ "color": "#F4D598"
+ },
+ {
+ "alias": "/.*sdb1.*/",
+ "color": "#0A50A1"
+ },
+ {
+ "alias": "/.*sdb2.*/",
+ "color": "#BF1B00"
+ },
+ {
+ "alias": "/.*sdb3.*/",
+ "color": "#E0752D"
+ },
+ {
+ "alias": "/.*sdc1.*/",
+ "color": "#962D82"
+ },
+ {
+ "alias": "/.*sdc2.*/",
+ "color": "#614D93"
+ },
+ {
+ "alias": "/.*sdc3.*/",
+ "color": "#9AC48A"
+ },
+ {
+ "alias": "/.*sdd1.*/",
+ "color": "#65C5DB"
+ },
+ {
+ "alias": "/.*sdd2.*/",
+ "color": "#F9934E"
+ },
+ {
+ "alias": "/.*sdd3.*/",
+ "color": "#EA6460"
+ },
+ {
+ "alias": "/.*sde1.*/",
+ "color": "#E0F9D7"
+ },
+ {
+ "alias": "/.*sdd2.*/",
+ "color": "#FCEACA"
+ },
+ {
+ "alias": "/.*sde3.*/",
+ "color": "#F9E2D2"
+ }
+ ],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "irate(node_disk_io_time_weighted_seconds_total{instance=~\"$node:$port\",job=~\"$job\"}[5m])",
+ "intervalFactor": 4,
+ "legendFormat": "{{device}} - IO time weighted",
+ "refId": "A",
+ "step": 8
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "Disk IOs Weighted",
+ "tooltip": {
+ "shared": false,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "s",
+ "label": "time",
+ "logBase": 1,
+ "max": null,
+ "min": "0",
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "description": "",
+ "fill": 2,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 0,
+ "y": 49
+ },
+ "hiddenSeries": false,
+ "id": 133,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "hideZero": true,
+ "max": true,
+ "min": true,
+ "rightSide": false,
+ "show": true,
+ "sort": "current",
+ "sortDesc": true,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "maxPerRow": 6,
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [
+ {
+ "alias": "/.*Read.*/",
+ "transform": "negative-Y"
+ },
+ {
+ "alias": "/.*sda_.*/",
+ "color": "#7EB26D"
+ },
+ {
+ "alias": "/.*sdb_.*/",
+ "color": "#EAB839"
+ },
+ {
+ "alias": "/.*sdc_.*/",
+ "color": "#6ED0E0"
+ },
+ {
+ "alias": "/.*sdd_.*/",
+ "color": "#EF843C"
+ },
+ {
+ "alias": "/.*sde_.*/",
+ "color": "#E24D42"
+ },
+ {
+ "alias": "/.*sda1.*/",
+ "color": "#584477"
+ },
+ {
+ "alias": "/.*sda2_.*/",
+ "color": "#BA43A9"
+ },
+ {
+ "alias": "/.*sda3_.*/",
+ "color": "#F4D598"
+ },
+ {
+ "alias": "/.*sdb1.*/",
+ "color": "#0A50A1"
+ },
+ {
+ "alias": "/.*sdb2.*/",
+ "color": "#BF1B00"
+ },
+ {
+ "alias": "/.*sdb3.*/",
+ "color": "#E0752D"
+ },
+ {
+ "alias": "/.*sdc1.*/",
+ "color": "#962D82"
+ },
+ {
+ "alias": "/.*sdc2.*/",
+ "color": "#614D93"
+ },
+ {
+ "alias": "/.*sdc3.*/",
+ "color": "#9AC48A"
+ },
+ {
+ "alias": "/.*sdd1.*/",
+ "color": "#65C5DB"
+ },
+ {
+ "alias": "/.*sdd2.*/",
+ "color": "#F9934E"
+ },
+ {
+ "alias": "/.*sdd3.*/",
+ "color": "#EA6460"
+ },
+ {
+ "alias": "/.*sde1.*/",
+ "color": "#E0F9D7"
+ },
+ {
+ "alias": "/.*sdd2.*/",
+ "color": "#FCEACA"
+ },
+ {
+ "alias": "/.*sde3.*/",
+ "color": "#F9E2D2"
+ }
+ ],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "irate(node_disk_reads_merged_total{instance=~\"$node:$port\",job=~\"$job\"}[5m])",
+ "intervalFactor": 2,
+ "legendFormat": "{{device}} - Read merged",
+ "refId": "A",
+ "step": 4
+ },
+ {
+ "expr": "irate(node_disk_writes_merged_total{instance=~\"$node:$port\",job=~\"$job\"}[5m])",
+ "intervalFactor": 2,
+ "legendFormat": "{{device}} - Write merged",
+ "refId": "B",
+ "step": 4
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "Disk R/W Merged",
+ "tooltip": {
+ "shared": false,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "iops",
+ "label": "I/Os",
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "description": "",
+ "fill": 3,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 12,
+ "y": 49
+ },
+ "hiddenSeries": false,
+ "id": 36,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "hideZero": true,
+ "max": true,
+ "min": true,
+ "rightSide": false,
+ "show": true,
+ "sort": "current",
+ "sortDesc": true,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "maxPerRow": 6,
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [
+ {
+ "alias": "/.*sda_.*/",
+ "color": "#7EB26D"
+ },
+ {
+ "alias": "/.*sdb_.*/",
+ "color": "#EAB839"
+ },
+ {
+ "alias": "/.*sdc_.*/",
+ "color": "#6ED0E0"
+ },
+ {
+ "alias": "/.*sdd_.*/",
+ "color": "#EF843C"
+ },
+ {
+ "alias": "/.*sde_.*/",
+ "color": "#E24D42"
+ },
+ {
+ "alias": "/.*sda1.*/",
+ "color": "#584477"
+ },
+ {
+ "alias": "/.*sda2_.*/",
+ "color": "#BA43A9"
+ },
+ {
+ "alias": "/.*sda3_.*/",
+ "color": "#F4D598"
+ },
+ {
+ "alias": "/.*sdb1.*/",
+ "color": "#0A50A1"
+ },
+ {
+ "alias": "/.*sdb2.*/",
+ "color": "#BF1B00"
+ },
+ {
+ "alias": "/.*sdb3.*/",
+ "color": "#E0752D"
+ },
+ {
+ "alias": "/.*sdc1.*/",
+ "color": "#962D82"
+ },
+ {
+ "alias": "/.*sdc2.*/",
+ "color": "#614D93"
+ },
+ {
+ "alias": "/.*sdc3.*/",
+ "color": "#9AC48A"
+ },
+ {
+ "alias": "/.*sdd1.*/",
+ "color": "#65C5DB"
+ },
+ {
+ "alias": "/.*sdd2.*/",
+ "color": "#F9934E"
+ },
+ {
+ "alias": "/.*sdd3.*/",
+ "color": "#EA6460"
+ },
+ {
+ "alias": "/.*sde1.*/",
+ "color": "#E0F9D7"
+ },
+ {
+ "alias": "/.*sdd2.*/",
+ "color": "#FCEACA"
+ },
+ {
+ "alias": "/.*sde3.*/",
+ "color": "#F9E2D2"
+ }
+ ],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "irate(node_disk_io_time_seconds_total{instance=~\"$node:$port\",job=~\"$job\"}[5m])",
+ "intervalFactor": 4,
+ "legendFormat": "{{device}} - IO time",
+ "refId": "A",
+ "step": 8
+ },
+ {
+ "expr": "irate(node_disk_discard_time_seconds_total{instance=~\"$node:$port\",job=~\"$job\"}[5m])",
+ "interval": "",
+ "intervalFactor": 4,
+ "legendFormat": "{{device}} - discard time",
+ "refId": "B",
+ "step": 8
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "Time Spent Doing I/Os",
+ "tooltip": {
+ "shared": false,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "s",
+ "label": "time",
+ "logBase": 1,
+ "max": null,
+ "min": "0",
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "description": "",
+ "fill": 2,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 0,
+ "y": 59
+ },
+ "hiddenSeries": false,
+ "id": 34,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "hideZero": true,
+ "max": true,
+ "min": true,
+ "rightSide": false,
+ "show": true,
+ "sort": "current",
+ "sortDesc": true,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "maxPerRow": 6,
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [
+ {
+ "alias": "/.*sda_.*/",
+ "color": "#7EB26D"
+ },
+ {
+ "alias": "/.*sdb_.*/",
+ "color": "#EAB839"
+ },
+ {
+ "alias": "/.*sdc_.*/",
+ "color": "#6ED0E0"
+ },
+ {
+ "alias": "/.*sdd_.*/",
+ "color": "#EF843C"
+ },
+ {
+ "alias": "/.*sde_.*/",
+ "color": "#E24D42"
+ },
+ {
+ "alias": "/.*sda1.*/",
+ "color": "#584477"
+ },
+ {
+ "alias": "/.*sda2_.*/",
+ "color": "#BA43A9"
+ },
+ {
+ "alias": "/.*sda3_.*/",
+ "color": "#F4D598"
+ },
+ {
+ "alias": "/.*sdb1.*/",
+ "color": "#0A50A1"
+ },
+ {
+ "alias": "/.*sdb2.*/",
+ "color": "#BF1B00"
+ },
+ {
+ "alias": "/.*sdb3.*/",
+ "color": "#E0752D"
+ },
+ {
+ "alias": "/.*sdc1.*/",
+ "color": "#962D82"
+ },
+ {
+ "alias": "/.*sdc2.*/",
+ "color": "#614D93"
+ },
+ {
+ "alias": "/.*sdc3.*/",
+ "color": "#9AC48A"
+ },
+ {
+ "alias": "/.*sdd1.*/",
+ "color": "#65C5DB"
+ },
+ {
+ "alias": "/.*sdd2.*/",
+ "color": "#F9934E"
+ },
+ {
+ "alias": "/.*sdd3.*/",
+ "color": "#EA6460"
+ },
+ {
+ "alias": "/.*sde1.*/",
+ "color": "#E0F9D7"
+ },
+ {
+ "alias": "/.*sdd2.*/",
+ "color": "#FCEACA"
+ },
+ {
+ "alias": "/.*sde3.*/",
+ "color": "#F9E2D2"
+ }
+ ],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "node_disk_io_now{instance=~\"$node:$port\",job=~\"$job\"}",
+ "interval": "",
+ "intervalFactor": 4,
+ "legendFormat": "{{device}} - IO now",
+ "refId": "A",
+ "step": 8
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "Disk IOs Current in Progress",
+ "tooltip": {
+ "shared": false,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "iops",
+ "label": "I/Os",
+ "logBase": 1,
+ "max": null,
+ "min": "0",
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "description": "",
+ "fill": 2,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 12,
+ "y": 59
+ },
+ "hiddenSeries": false,
+ "id": 301,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "hideZero": true,
+ "max": true,
+ "min": true,
+ "rightSide": false,
+ "show": true,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "maxPerRow": 6,
+ "nullPointMode": "null as zero",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [
+ {
+ "alias": "/.*sda_.*/",
+ "color": "#7EB26D"
+ },
+ {
+ "alias": "/.*sdb_.*/",
+ "color": "#EAB839"
+ },
+ {
+ "alias": "/.*sdc_.*/",
+ "color": "#6ED0E0"
+ },
+ {
+ "alias": "/.*sdd_.*/",
+ "color": "#EF843C"
+ },
+ {
+ "alias": "/.*sde_.*/",
+ "color": "#E24D42"
+ },
+ {
+ "alias": "/.*sda1.*/",
+ "color": "#584477"
+ },
+ {
+ "alias": "/.*sda2_.*/",
+ "color": "#BA43A9"
+ },
+ {
+ "alias": "/.*sda3_.*/",
+ "color": "#F4D598"
+ },
+ {
+ "alias": "/.*sdb1.*/",
+ "color": "#0A50A1"
+ },
+ {
+ "alias": "/.*sdb2.*/",
+ "color": "#BF1B00"
+ },
+ {
+ "alias": "/.*sdb3.*/",
+ "color": "#E0752D"
+ },
+ {
+ "alias": "/.*sdc1.*/",
+ "color": "#962D82"
+ },
+ {
+ "alias": "/.*sdc2.*/",
+ "color": "#614D93"
+ },
+ {
+ "alias": "/.*sdc3.*/",
+ "color": "#9AC48A"
+ },
+ {
+ "alias": "/.*sdd1.*/",
+ "color": "#65C5DB"
+ },
+ {
+ "alias": "/.*sdd2.*/",
+ "color": "#F9934E"
+ },
+ {
+ "alias": "/.*sdd3.*/",
+ "color": "#EA6460"
+ },
+ {
+ "alias": "/.*sde1.*/",
+ "color": "#E0F9D7"
+ },
+ {
+ "alias": "/.*sdd2.*/",
+ "color": "#FCEACA"
+ },
+ {
+ "alias": "/.*sde3.*/",
+ "color": "#F9E2D2"
+ }
+ ],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "irate(node_disk_discards_completed_total{instance=~\"$node:$port\",job=~\"$job\"}[5m])",
+ "interval": "",
+ "intervalFactor": 4,
+ "legendFormat": "{{device}} - Discards completed",
+ "refId": "A",
+ "step": 8
+ },
+ {
+ "expr": "irate(node_disk_discards_merged_total{instance=~\"$node:$port\",job=~\"$job\"}[5m])",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "{{device}} - Discards merged",
+ "refId": "B",
+ "step": 4
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "Disk IOps Discards completed / merged",
+ "tooltip": {
+ "shared": false,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "iops",
+ "label": "IOs",
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ }
+ ],
+ "repeat": null,
+ "title": "Storage Disk",
+ "type": "row"
+ },
+ {
+ "collapsed": true,
+ "datasource": "${DS_PROMETHEUS}",
+ "gridPos": {
+ "h": 1,
+ "w": 24,
+ "x": 0,
+ "y": 67
+ },
+ "id": 271,
+ "panels": [
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "decimals": 3,
+ "description": "",
+ "fill": 2,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 0,
+ "y": 29
+ },
+ "hiddenSeries": false,
+ "id": 43,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "max": true,
+ "min": true,
+ "rightSide": false,
+ "show": true,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "maxPerRow": 6,
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "node_filesystem_avail_bytes{instance=~\"$node:$port\",job=~\"$job\",device!~'rootfs'}",
+ "format": "time_series",
+ "hide": false,
+ "intervalFactor": 2,
+ "legendFormat": "{{mountpoint}} - Available",
+ "metric": "",
+ "refId": "A",
+ "step": 4
+ },
+ {
+ "expr": "node_filesystem_free_bytes{instance=~\"$node:$port\",job=~\"$job\",device!~'rootfs'}",
+ "format": "time_series",
+ "hide": true,
+ "intervalFactor": 2,
+ "legendFormat": "{{mountpoint}} - Free",
+ "refId": "B",
+ "step": 2
+ },
+ {
+ "expr": "node_filesystem_size_bytes{instance=~\"$node:$port\",job=~\"$job\",device!~'rootfs'}",
+ "format": "time_series",
+ "hide": true,
+ "intervalFactor": 2,
+ "legendFormat": "{{mountpoint}} - Size",
+ "refId": "C",
+ "step": 2
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "Filesystem space available",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "bytes",
+ "label": "bytes",
+ "logBase": 1,
+ "max": null,
+ "min": "0",
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "description": "",
+ "fill": 2,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 12,
+ "y": 29
+ },
+ "hiddenSeries": false,
+ "id": 41,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "hideZero": true,
+ "max": true,
+ "min": true,
+ "rightSide": false,
+ "show": true,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "node_filesystem_files_free{instance=~\"$node:$port\",job=~\"$job\",device!~'rootfs'}",
+ "format": "time_series",
+ "hide": false,
+ "intervalFactor": 2,
+ "legendFormat": "{{mountpoint}} - Free file nodes",
+ "refId": "A",
+ "step": 4
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "File Nodes Free",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "short",
+ "label": "file nodes",
+ "logBase": 1,
+ "max": null,
+ "min": "0",
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "description": "",
+ "fill": 2,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 0,
+ "y": 39
+ },
+ "hiddenSeries": false,
+ "id": 28,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "max": true,
+ "min": true,
+ "show": true,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "maxPerRow": 6,
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "node_filefd_maximum{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "intervalFactor": 4,
+ "legendFormat": "Max open files",
+ "refId": "A",
+ "step": 8
+ },
+ {
+ "expr": "node_filefd_allocated{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "intervalFactor": 2,
+ "legendFormat": "Open files",
+ "refId": "B",
+ "step": 4
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "File Descriptor",
+ "tooltip": {
+ "shared": false,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "short",
+ "label": "files",
+ "logBase": 1,
+ "max": null,
+ "min": "0",
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "description": "",
+ "fill": 2,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 12,
+ "y": 39
+ },
+ "hiddenSeries": false,
+ "id": 219,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "hideZero": true,
+ "max": true,
+ "min": true,
+ "rightSide": false,
+ "show": true,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "node_filesystem_files{instance=~\"$node:$port\",job=~\"$job\",device!~'rootfs'}",
+ "format": "time_series",
+ "hide": false,
+ "intervalFactor": 2,
+ "legendFormat": "{{mountpoint}} - File nodes total",
+ "refId": "A",
+ "step": 4
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "File Nodes Size",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "short",
+ "label": "file Nodes",
+ "logBase": 1,
+ "max": null,
+ "min": "0",
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {
+ "/ ReadOnly": "#890F02"
+ },
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "decimals": null,
+ "description": "",
+ "fill": 2,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 0,
+ "y": 49
+ },
+ "hiddenSeries": false,
+ "id": 44,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "hideEmpty": true,
+ "hideZero": true,
+ "max": true,
+ "min": true,
+ "rightSide": false,
+ "show": true,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "maxPerRow": 6,
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "stack": true,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "node_filesystem_readonly{instance=~\"$node:$port\",job=~\"$job\",device!~'rootfs'}",
+ "format": "time_series",
+ "intervalFactor": 2,
+ "legendFormat": "{{mountpoint}} - ReadOnly",
+ "refId": "A",
+ "step": 4
+ },
+ {
+ "expr": "node_filesystem_device_error{instance=~\"$node:$port\",job=~\"$job\",device!~'rootfs'}",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "{{mountpoint}} - Device error",
+ "refId": "B",
+ "step": 4
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "Filesystem in ReadOnly / Error",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "short",
+ "label": "counter",
+ "logBase": 1,
+ "max": "1",
+ "min": "0",
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ }
+ ],
+ "repeat": null,
+ "title": "Storage Filesystem",
+ "type": "row"
+ },
+ {
+ "collapsed": true,
+ "datasource": "${DS_PROMETHEUS}",
+ "gridPos": {
+ "h": 1,
+ "w": 24,
+ "x": 0,
+ "y": 68
+ },
+ "id": 272,
+ "panels": [
+ {
+ "aliasColors": {
+ "receive_packets_eth0": "#7EB26D",
+ "receive_packets_lo": "#E24D42",
+ "transmit_packets_eth0": "#7EB26D",
+ "transmit_packets_lo": "#E24D42"
+ },
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "fieldConfig": {
+ "defaults": {
+ "custom": {}
+ },
+ "overrides": []
+ },
+ "fill": 2,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 0,
+ "y": 109
+ },
+ "hiddenSeries": false,
+ "id": 60,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "max": true,
+ "min": true,
+ "rightSide": false,
+ "show": true,
+ "sideWidth": 300,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [
+ {
+ "alias": "/.*Trans.*/",
+ "transform": "negative-Y"
+ }
+ ],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "irate(node_network_receive_packets_total{instance=~\"$node:$port\",job=~\"$job\"}[5m])",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "{{device}} - Receive",
+ "refId": "A",
+ "step": 4
+ },
+ {
+ "expr": "irate(node_network_transmit_packets_total{instance=~\"$node:$port\",job=~\"$job\"}[5m])",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "{{device}} - Transmit",
+ "refId": "B",
+ "step": 4
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "Network Traffic by Packets",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "pps",
+ "label": "packets out (-) / in (+)",
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "fieldConfig": {
+ "defaults": {
+ "custom": {}
+ },
+ "overrides": []
+ },
+ "fill": 2,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 12,
+ "y": 109
+ },
+ "hiddenSeries": false,
+ "id": 142,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "hideEmpty": false,
+ "hideZero": false,
+ "max": true,
+ "min": true,
+ "rightSide": false,
+ "show": true,
+ "sideWidth": 300,
+ "sort": "current",
+ "sortDesc": true,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [
+ {
+ "alias": "/.*Trans.*/",
+ "transform": "negative-Y"
+ }
+ ],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "irate(node_network_receive_errs_total{instance=~\"$node:$port\",job=~\"$job\"}[5m])",
+ "format": "time_series",
+ "intervalFactor": 2,
+ "legendFormat": "{{device}} - Receive errors",
+ "refId": "A",
+ "step": 4
+ },
+ {
+ "expr": "irate(node_network_transmit_errs_total{instance=~\"$node:$port\",job=~\"$job\"}[5m])",
+ "format": "time_series",
+ "intervalFactor": 2,
+ "legendFormat": "{{device}} - Rransmit errors",
+ "refId": "B",
+ "step": 4
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "Network Traffic Errors",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "pps",
+ "label": "packets out (-) / in (+)",
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "fieldConfig": {
+ "defaults": {
+ "custom": {}
+ },
+ "overrides": []
+ },
+ "fill": 2,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 0,
+ "y": 119
+ },
+ "hiddenSeries": false,
+ "id": 143,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "hideEmpty": false,
+ "hideZero": false,
+ "max": true,
+ "min": true,
+ "rightSide": false,
+ "show": true,
+ "sideWidth": 300,
+ "sort": "current",
+ "sortDesc": true,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [
+ {
+ "alias": "/.*Trans.*/",
+ "transform": "negative-Y"
+ }
+ ],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "irate(node_network_receive_drop_total{instance=~\"$node:$port\",job=~\"$job\"}[5m])",
+ "format": "time_series",
+ "intervalFactor": 2,
+ "legendFormat": "{{device}} - Receive drop",
+ "refId": "A",
+ "step": 4
+ },
+ {
+ "expr": "irate(node_network_transmit_drop_total{instance=~\"$node:$port\",job=~\"$job\"}[5m])",
+ "format": "time_series",
+ "intervalFactor": 2,
+ "legendFormat": "{{device}} - Transmit drop",
+ "refId": "B",
+ "step": 4
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "Network Traffic Drop",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "pps",
+ "label": "packets out (-) / in (+)",
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "fieldConfig": {
+ "defaults": {
+ "custom": {}
+ },
+ "overrides": []
+ },
+ "fill": 2,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 12,
+ "y": 119
+ },
+ "hiddenSeries": false,
+ "id": 141,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "hideEmpty": false,
+ "hideZero": false,
+ "max": true,
+ "min": true,
+ "rightSide": false,
+ "show": true,
+ "sideWidth": 300,
+ "sort": "current",
+ "sortDesc": true,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [
+ {
+ "alias": "/.*Trans.*/",
+ "transform": "negative-Y"
+ }
+ ],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "irate(node_network_receive_compressed_total{instance=~\"$node:$port\",job=~\"$job\"}[5m])",
+ "format": "time_series",
+ "intervalFactor": 2,
+ "legendFormat": "{{device}} - Receive compressed",
+ "refId": "A",
+ "step": 4
+ },
+ {
+ "expr": "irate(node_network_transmit_compressed_total{instance=~\"$node:$port\",job=~\"$job\"}[5m])",
+ "format": "time_series",
+ "intervalFactor": 2,
+ "legendFormat": "{{device}} - Transmit compressed",
+ "refId": "B",
+ "step": 4
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "Network Traffic Compressed",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "pps",
+ "label": "packets out (-) / in (+)",
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "fieldConfig": {
+ "defaults": {
+ "custom": {}
+ },
+ "overrides": []
+ },
+ "fill": 2,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 0,
+ "y": 129
+ },
+ "hiddenSeries": false,
+ "id": 146,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "hideEmpty": false,
+ "hideZero": false,
+ "max": true,
+ "min": true,
+ "rightSide": false,
+ "show": true,
+ "sideWidth": 300,
+ "sort": "current",
+ "sortDesc": true,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [
+ {
+ "alias": "/.*Trans.*/",
+ "transform": "negative-Y"
+ }
+ ],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "irate(node_network_receive_multicast_total{instance=~\"$node:$port\",job=~\"$job\"}[5m])",
+ "format": "time_series",
+ "intervalFactor": 2,
+ "legendFormat": "{{device}} - Receive multicast",
+ "refId": "A",
+ "step": 4
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "Network Traffic Multicast",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "pps",
+ "label": "packets out (-) / in (+)",
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "fieldConfig": {
+ "defaults": {
+ "custom": {}
+ },
+ "overrides": []
+ },
+ "fill": 2,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 12,
+ "y": 129
+ },
+ "hiddenSeries": false,
+ "id": 144,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "hideEmpty": false,
+ "hideZero": false,
+ "max": true,
+ "min": true,
+ "rightSide": false,
+ "show": true,
+ "sideWidth": 300,
+ "sort": "current",
+ "sortDesc": true,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [
+ {
+ "alias": "/.*Trans.*/",
+ "transform": "negative-Y"
+ }
+ ],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "irate(node_network_receive_fifo_total{instance=~\"$node:$port\",job=~\"$job\"}[5m])",
+ "format": "time_series",
+ "intervalFactor": 2,
+ "legendFormat": "{{device}} - Receive fifo",
+ "refId": "A",
+ "step": 4
+ },
+ {
+ "expr": "irate(node_network_transmit_fifo_total{instance=~\"$node:$port\",job=~\"$job\"}[5m])",
+ "format": "time_series",
+ "intervalFactor": 2,
+ "legendFormat": "{{device}} - Transmit fifo",
+ "refId": "B",
+ "step": 4
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "Network Traffic Fifo",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "pps",
+ "label": "packets out (-) / in (+)",
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "fieldConfig": {
+ "defaults": {
+ "custom": {}
+ },
+ "overrides": []
+ },
+ "fill": 2,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 0,
+ "y": 139
+ },
+ "hiddenSeries": false,
+ "id": 145,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "hideEmpty": false,
+ "hideZero": false,
+ "max": true,
+ "min": true,
+ "rightSide": false,
+ "show": true,
+ "sideWidth": 300,
+ "sort": "current",
+ "sortDesc": true,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [
+ {
+ "alias": "/.*Trans.*/",
+ "transform": "negative-Y"
+ }
+ ],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "irate(node_network_receive_frame_total{instance=~\"$node:$port\",job=~\"$job\"}[5m])",
+ "format": "time_series",
+ "hide": false,
+ "intervalFactor": 2,
+ "legendFormat": "{{device}} - Receive frame",
+ "refId": "A",
+ "step": 4
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "Network Traffic Frame",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "pps",
+ "label": "packets out (-) / in (+)",
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "fieldConfig": {
+ "defaults": {
+ "custom": {}
+ },
+ "overrides": []
+ },
+ "fill": 2,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 12,
+ "y": 139
+ },
+ "hiddenSeries": false,
+ "id": 231,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "hideEmpty": false,
+ "hideZero": false,
+ "max": true,
+ "min": true,
+ "rightSide": false,
+ "show": true,
+ "sideWidth": 300,
+ "sort": "current",
+ "sortDesc": true,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "irate(node_network_transmit_carrier_total{instance=~\"$node:$port\",job=~\"$job\"}[5m])",
+ "format": "time_series",
+ "intervalFactor": 2,
+ "legendFormat": "{{device}} - Statistic transmit_carrier",
+ "refId": "A",
+ "step": 4
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "Network Traffic Carrier",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "short",
+ "label": "counter",
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "fieldConfig": {
+ "defaults": {
+ "custom": {}
+ },
+ "overrides": []
+ },
+ "fill": 2,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 0,
+ "y": 149
+ },
+ "hiddenSeries": false,
+ "id": 232,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "hideEmpty": false,
+ "hideZero": false,
+ "max": true,
+ "min": true,
+ "rightSide": false,
+ "show": true,
+ "sideWidth": 300,
+ "sort": "current",
+ "sortDesc": true,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [
+ {
+ "alias": "/.*Trans.*/",
+ "transform": "negative-Y"
+ }
+ ],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "irate(node_network_transmit_colls_total{instance=~\"$node:$port\",job=~\"$job\"}[5m])",
+ "format": "time_series",
+ "intervalFactor": 2,
+ "legendFormat": "{{device}} - Transmit colls",
+ "refId": "A",
+ "step": 4
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "Network Traffic Colls",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "short",
+ "label": "counter",
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "fieldConfig": {
+ "defaults": {
+ "custom": {}
+ },
+ "overrides": []
+ },
+ "fill": 2,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 12,
+ "y": 149
+ },
+ "hiddenSeries": false,
+ "id": 61,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "max": true,
+ "min": true,
+ "rightSide": false,
+ "show": true,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [
+ {
+ "alias": "NF conntrack limit",
+ "color": "#890F02",
+ "fill": 0
+ }
+ ],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "node_nf_conntrack_entries{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "intervalFactor": 2,
+ "legendFormat": "NF conntrack entries",
+ "refId": "A",
+ "step": 4
+ },
+ {
+ "expr": "node_nf_conntrack_entries_limit{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "intervalFactor": 2,
+ "legendFormat": "NF conntrack limit",
+ "refId": "B",
+ "step": 4
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "NF Contrack",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "short",
+ "label": "entries",
+ "logBase": 1,
+ "max": null,
+ "min": "0",
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "fieldConfig": {
+ "defaults": {
+ "custom": {}
+ },
+ "overrides": []
+ },
+ "fill": 2,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 0,
+ "y": 159
+ },
+ "hiddenSeries": false,
+ "id": 230,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "max": true,
+ "min": true,
+ "rightSide": false,
+ "show": true,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "node_arp_entries{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "intervalFactor": 2,
+ "legendFormat": "{{ device }} - ARP entries",
+ "refId": "A",
+ "step": 4
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "ARP Entries",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "short",
+ "label": "Entries",
+ "logBase": 1,
+ "max": null,
+ "min": "0",
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "fieldConfig": {
+ "defaults": {
+ "custom": {}
+ },
+ "overrides": []
+ },
+ "fill": 2,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 12,
+ "y": 159
+ },
+ "hiddenSeries": false,
+ "id": 288,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "max": true,
+ "min": true,
+ "rightSide": false,
+ "show": true,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 1,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "node_network_mtu_bytes{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "intervalFactor": 2,
+ "legendFormat": "{{ device }} - Bytes",
+ "refId": "A",
+ "step": 4
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "MTU",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "decimals": 0,
+ "format": "bytes",
+ "label": "bytes",
+ "logBase": 1,
+ "max": null,
+ "min": "0",
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "fieldConfig": {
+ "defaults": {
+ "custom": {}
+ },
+ "overrides": []
+ },
+ "fill": 2,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 0,
+ "y": 169
+ },
+ "hiddenSeries": false,
+ "id": 280,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "max": true,
+ "min": true,
+ "rightSide": false,
+ "show": true,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 1,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "node_network_speed_bytes{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "intervalFactor": 2,
+ "legendFormat": "{{ device }} - Speed",
+ "refId": "A",
+ "step": 4
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "Speed",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "decimals": 0,
+ "format": "bytes",
+ "label": "bytes",
+ "logBase": 1,
+ "max": null,
+ "min": "0",
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "fieldConfig": {
+ "defaults": {
+ "custom": {}
+ },
+ "overrides": []
+ },
+ "fill": 2,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 12,
+ "y": 169
+ },
+ "hiddenSeries": false,
+ "id": 289,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "max": true,
+ "min": true,
+ "rightSide": false,
+ "show": true,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 1,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "node_network_transmit_queue_length{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "intervalFactor": 2,
+ "legendFormat": "{{ device }} - Interface transmit queue length",
+ "refId": "A",
+ "step": 4
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "Queue Length",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "decimals": 0,
+ "format": "none",
+ "label": "packets",
+ "logBase": 1,
+ "max": null,
+ "min": "0",
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "fieldConfig": {
+ "defaults": {
+ "custom": {}
+ },
+ "overrides": []
+ },
+ "fill": 2,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 0,
+ "y": 179
+ },
+ "hiddenSeries": false,
+ "id": 290,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "hideEmpty": false,
+ "hideZero": false,
+ "max": true,
+ "min": true,
+ "rightSide": false,
+ "show": true,
+ "sideWidth": 300,
+ "sort": "current",
+ "sortDesc": true,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [
+ {
+ "alias": "/.*Dropped.*/",
+ "transform": "negative-Y"
+ }
+ ],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "irate(node_softnet_processed_total{instance=~\"$node:$port\",job=~\"$job\"}[5m])",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "CPU {{cpu}} - Processed",
+ "refId": "A",
+ "step": 4
+ },
+ {
+ "expr": "irate(node_softnet_dropped_total{instance=~\"$node:$port\",job=~\"$job\"}[5m])",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "CPU {{cpu}} - Dropped",
+ "refId": "B",
+ "step": 4
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "Softnet Packets",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "short",
+ "label": "packetes drop (-) / process (+)",
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "fieldConfig": {
+ "defaults": {
+ "custom": {}
+ },
+ "overrides": []
+ },
+ "fill": 2,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 12,
+ "y": 179
+ },
+ "hiddenSeries": false,
+ "id": 310,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "hideEmpty": false,
+ "hideZero": false,
+ "max": true,
+ "min": true,
+ "rightSide": false,
+ "show": true,
+ "sideWidth": 300,
+ "sort": "current",
+ "sortDesc": true,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "irate(node_softnet_times_squeezed_total{instance=~\"$node:$port\",job=~\"$job\"}[5m])",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "CPU {{cpu}} - Squeezed",
+ "refId": "A",
+ "step": 4
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "Softnet Out of Quota",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "short",
+ "label": "counter",
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "fieldConfig": {
+ "defaults": {
+ "custom": {}
+ },
+ "overrides": []
+ },
+ "fill": 2,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 0,
+ "y": 189
+ },
+ "hiddenSeries": false,
+ "id": 309,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "hideEmpty": false,
+ "hideZero": false,
+ "max": true,
+ "min": true,
+ "rightSide": false,
+ "show": true,
+ "sideWidth": 300,
+ "sort": "current",
+ "sortDesc": true,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "node_network_up{operstate=\"up\",instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "intervalFactor": 2,
+ "legendFormat": "{{interface}} - Operational state UP",
+ "refId": "A",
+ "step": 4
+ },
+ {
+ "expr": "node_network_carrier{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "instant": false,
+ "legendFormat": "{{device}} - Physical link state",
+ "refId": "B"
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "Network Operational Status",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "short",
+ "label": "counter",
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ }
+ ],
+ "repeat": null,
+ "title": "Network Traffic",
+ "type": "row"
+ },
+ {
+ "collapsed": true,
+ "datasource": "${DS_PROMETHEUS}",
+ "gridPos": {
+ "h": 1,
+ "w": 24,
+ "x": 0,
+ "y": 69
+ },
+ "id": 273,
+ "panels": [
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "fill": 2,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 0,
+ "y": 13
+ },
+ "hiddenSeries": false,
+ "id": 63,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "hideEmpty": false,
+ "hideZero": false,
+ "max": true,
+ "min": true,
+ "rightSide": false,
+ "show": true,
+ "sideWidth": 300,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "node_sockstat_TCP_alloc{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "TCP_alloc - Allocated sockets",
+ "refId": "A",
+ "step": 240
+ },
+ {
+ "expr": "node_sockstat_TCP_inuse{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "TCP_inuse - Tcp sockets currently in use",
+ "refId": "B",
+ "step": 240
+ },
+ {
+ "expr": "node_sockstat_TCP_mem{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "hide": true,
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "TCP_mem - Used memory for tcp",
+ "refId": "C",
+ "step": 240
+ },
+ {
+ "expr": "node_sockstat_TCP_orphan{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "TCP_orphan - Orphan sockets",
+ "refId": "D",
+ "step": 240
+ },
+ {
+ "expr": "node_sockstat_TCP_tw{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "TCP_tw - Sockets wating close",
+ "refId": "E",
+ "step": 240
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "Sockstat TCP",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "short",
+ "label": "counter",
+ "logBase": 1,
+ "max": null,
+ "min": "0",
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "fill": 2,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 12,
+ "y": 13
+ },
+ "hiddenSeries": false,
+ "id": 124,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "hideEmpty": false,
+ "hideZero": false,
+ "max": true,
+ "min": true,
+ "rightSide": false,
+ "show": true,
+ "sideWidth": 300,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "node_sockstat_UDPLITE_inuse{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "UDPLITE_inuse - Udplite sockets currently in use",
+ "refId": "A",
+ "step": 240
+ },
+ {
+ "expr": "node_sockstat_UDP_inuse{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "UDP_inuse - Udp sockets currently in use",
+ "refId": "B",
+ "step": 240
+ },
+ {
+ "expr": "node_sockstat_UDP_mem{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "UDP_mem - Used memory for udp",
+ "refId": "C",
+ "step": 240
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "Sockstat UDP",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "short",
+ "label": "counter",
+ "logBase": 1,
+ "max": null,
+ "min": "0",
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "fill": 2,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 0,
+ "y": 23
+ },
+ "hiddenSeries": false,
+ "id": 126,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "hideEmpty": false,
+ "hideZero": false,
+ "max": true,
+ "min": true,
+ "rightSide": false,
+ "show": true,
+ "sideWidth": 300,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "node_sockstat_sockets_used{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "Sockets_used - Sockets currently in use",
+ "refId": "A",
+ "step": 240
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "Sockstat Used",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "short",
+ "label": "sockets",
+ "logBase": 1,
+ "max": null,
+ "min": "0",
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "fill": 2,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 12,
+ "y": 23
+ },
+ "hiddenSeries": false,
+ "id": 220,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "hideEmpty": false,
+ "hideZero": false,
+ "max": true,
+ "min": true,
+ "rightSide": false,
+ "show": true,
+ "sideWidth": 300,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "node_sockstat_TCP_mem_bytes{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "mem_bytes - TCP sockets in that state",
+ "refId": "A",
+ "step": 240
+ },
+ {
+ "expr": "node_sockstat_UDP_mem_bytes{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "mem_bytes - UDP sockets in that state",
+ "refId": "B",
+ "step": 240
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "Sockstat Memory Size",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "bytes",
+ "label": "bytes",
+ "logBase": 1,
+ "max": null,
+ "min": "0",
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "fill": 2,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 0,
+ "y": 33
+ },
+ "hiddenSeries": false,
+ "id": 125,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "hideEmpty": false,
+ "hideZero": false,
+ "max": true,
+ "min": true,
+ "rightSide": false,
+ "show": true,
+ "sideWidth": 300,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "node_sockstat_FRAG_inuse{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "FRAG_inuse - Frag sockets currently in use",
+ "refId": "A",
+ "step": 240
+ },
+ {
+ "expr": "node_sockstat_FRAG_memory{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "FRAG_memory - Used memory for frag",
+ "refId": "B",
+ "step": 240
+ },
+ {
+ "expr": "node_sockstat_RAW_inuse{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "RAW_inuse - Raw sockets currently in use",
+ "refId": "C",
+ "step": 240
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "Sockstat FRAG / RAW",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "short",
+ "label": "counter",
+ "logBase": 1,
+ "max": null,
+ "min": "0",
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ }
+ ],
+ "repeat": null,
+ "title": "Network Sockstat",
+ "type": "row"
+ },
+ {
+ "collapsed": true,
+ "datasource": "${DS_PROMETHEUS}",
+ "gridPos": {
+ "h": 1,
+ "w": 24,
+ "x": 0,
+ "y": 70
+ },
+ "id": 274,
+ "panels": [
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "fill": 2,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 0,
+ "y": 32
+ },
+ "height": "",
+ "hiddenSeries": false,
+ "id": 221,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "hideEmpty": false,
+ "hideZero": false,
+ "max": true,
+ "min": true,
+ "rightSide": false,
+ "show": true,
+ "sideWidth": 300,
+ "sort": "current",
+ "sortDesc": true,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "maxPerRow": 12,
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [
+ {
+ "alias": "/.*Out.*/",
+ "transform": "negative-Y"
+ }
+ ],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "irate(node_netstat_IpExt_InOctets{instance=~\"$node:$port\",job=~\"$job\"}[5m])",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "InOctets - Received octets",
+ "refId": "A",
+ "step": 4
+ },
+ {
+ "expr": "irate(node_netstat_IpExt_OutOctets{instance=~\"$node:$port\",job=~\"$job\"}[5m])",
+ "format": "time_series",
+ "intervalFactor": 2,
+ "legendFormat": "OutOctets - Sent octets",
+ "refId": "B",
+ "step": 4
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "Netstat IP In / Out Octets",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "short",
+ "label": "octects out (-) / in (+)",
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "fill": 2,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 12,
+ "y": 32
+ },
+ "height": "",
+ "hiddenSeries": false,
+ "id": 81,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "hideEmpty": false,
+ "hideZero": false,
+ "max": true,
+ "min": true,
+ "rightSide": false,
+ "show": true,
+ "sideWidth": 300,
+ "sort": "current",
+ "sortDesc": true,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "irate(node_netstat_Ip_Forwarding{instance=~\"$node:$port\",job=~\"$job\"}[5m])",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "Forwarding - IP forwarding",
+ "refId": "A",
+ "step": 4
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "Netstat IP Forwarding",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "short",
+ "label": "datagrams",
+ "logBase": 1,
+ "max": null,
+ "min": "0",
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "decimals": null,
+ "fill": 2,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 0,
+ "y": 42
+ },
+ "height": "",
+ "hiddenSeries": false,
+ "id": 115,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "hideZero": false,
+ "max": true,
+ "min": true,
+ "rightSide": false,
+ "show": true,
+ "sort": "current",
+ "sortDesc": true,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "maxPerRow": 12,
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [
+ {
+ "alias": "/.*Out.*/",
+ "transform": "negative-Y"
+ }
+ ],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "irate(node_netstat_Icmp_InMsgs{instance=~\"$node:$port\",job=~\"$job\"}[5m])",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "InMsgs - Messages which the entity received. Note that this counter includes all those counted by icmpInErrors",
+ "refId": "A",
+ "step": 4
+ },
+ {
+ "expr": "irate(node_netstat_Icmp_OutMsgs{instance=~\"$node:$port\",job=~\"$job\"}[5m])",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "OutMsgs - Messages which this entity attempted to send. Note that this counter includes all those counted by icmpOutErrors",
+ "refId": "B",
+ "step": 4
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "ICMP In / Out",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "short",
+ "label": "messages out (-) / in (+)",
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "decimals": null,
+ "fill": 2,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 12,
+ "y": 42
+ },
+ "height": "",
+ "hiddenSeries": false,
+ "id": 50,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "hideZero": false,
+ "max": true,
+ "min": true,
+ "rightSide": false,
+ "show": true,
+ "sort": "current",
+ "sortDesc": true,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "maxPerRow": 12,
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [
+ {
+ "alias": "/.*Out.*/",
+ "transform": "negative-Y"
+ }
+ ],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "irate(node_netstat_Icmp_InErrors{instance=~\"$node:$port\",job=~\"$job\"}[5m])",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "InErrors - Messages which the entity received but determined as having ICMP-specific errors (bad ICMP checksums, bad length, etc.)",
+ "refId": "A",
+ "step": 4
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "ICMP Errors",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "short",
+ "label": "messages out (-) / in (+)",
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "decimals": null,
+ "fill": 2,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 0,
+ "y": 52
+ },
+ "height": "",
+ "hiddenSeries": false,
+ "id": 55,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "hideZero": false,
+ "max": true,
+ "min": true,
+ "rightSide": false,
+ "show": true,
+ "sort": "current",
+ "sortDesc": true,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "maxPerRow": 12,
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [
+ {
+ "alias": "/.*Out.*/",
+ "transform": "negative-Y"
+ },
+ {
+ "alias": "/.*Snd.*/",
+ "transform": "negative-Y"
+ }
+ ],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "irate(node_netstat_Udp_InDatagrams{instance=~\"$node:$port\",job=~\"$job\"}[5m])",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "InDatagrams - Datagrams received",
+ "refId": "A",
+ "step": 4
+ },
+ {
+ "expr": "irate(node_netstat_Udp_OutDatagrams{instance=~\"$node:$port\",job=~\"$job\"}[5m])",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "OutDatagrams - Datagrams sent",
+ "refId": "B",
+ "step": 4
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "UDP In / Out",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "short",
+ "label": "datagrams out (-) / in (+)",
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "fill": 2,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 12,
+ "y": 52
+ },
+ "height": "",
+ "hiddenSeries": false,
+ "id": 109,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "hideZero": false,
+ "max": true,
+ "min": true,
+ "rightSide": false,
+ "show": true,
+ "sort": "current",
+ "sortDesc": true,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "maxPerRow": 12,
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "irate(node_netstat_Udp_InErrors{instance=~\"$node:$port\",job=~\"$job\"}[5m])",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "InErrors - UDP Datagrams that could not be delivered to an application",
+ "refId": "A",
+ "step": 4
+ },
+ {
+ "expr": "irate(node_netstat_Udp_NoPorts{instance=~\"$node:$port\",job=~\"$job\"}[5m])",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "NoPorts - UDP Datagrams received on a port with no listener",
+ "refId": "B",
+ "step": 4
+ },
+ {
+ "expr": "irate(node_netstat_UdpLite_InErrors{instance=~\"$node:$port\",job=~\"$job\"}[5m])",
+ "interval": "",
+ "legendFormat": "InErrors Lite - UDPLite Datagrams that could not be delivered to an application",
+ "refId": "C"
+ },
+ {
+ "expr": "irate(node_netstat_Udp_RcvbufErrors{instance=~\"$node:$port\",job=~\"$job\"}[5m])",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "RcvbufErrors - UDP buffer errors received",
+ "refId": "D",
+ "step": 4
+ },
+ {
+ "expr": "irate(node_netstat_Udp_SndbufErrors{instance=~\"$node:$port\",job=~\"$job\"}[5m])",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "SndbufErrors - UDP buffer errors send",
+ "refId": "E",
+ "step": 4
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "UDP Errors",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "short",
+ "label": "datagrams",
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "decimals": null,
+ "fill": 2,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 0,
+ "y": 62
+ },
+ "height": "",
+ "hiddenSeries": false,
+ "id": 299,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "hideZero": false,
+ "max": true,
+ "min": true,
+ "rightSide": false,
+ "show": true,
+ "sort": "current",
+ "sortDesc": true,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "maxPerRow": 12,
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [
+ {
+ "alias": "/.*Out.*/",
+ "transform": "negative-Y"
+ },
+ {
+ "alias": "/.*Snd.*/",
+ "transform": "negative-Y"
+ }
+ ],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "irate(node_netstat_Tcp_InSegs{instance=~\"$node:$port\",job=~\"$job\"}[5m])",
+ "format": "time_series",
+ "instant": false,
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "InSegs - Segments received, including those received in error. This count includes segments received on currently established connections",
+ "refId": "A",
+ "step": 4
+ },
+ {
+ "expr": "irate(node_netstat_Tcp_OutSegs{instance=~\"$node:$port\",job=~\"$job\"}[5m])",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "OutSegs - Segments sent, including those on current connections but excluding those containing only retransmitted octets",
+ "refId": "B",
+ "step": 4
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "TCP In / Out",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "short",
+ "label": "datagrams out (-) / in (+)",
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "description": "",
+ "fill": 2,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 12,
+ "y": 62
+ },
+ "height": "",
+ "hiddenSeries": false,
+ "id": 104,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "hideEmpty": false,
+ "hideZero": false,
+ "max": true,
+ "min": true,
+ "rightSide": false,
+ "show": true,
+ "sort": "current",
+ "sortDesc": true,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "maxPerRow": 12,
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "irate(node_netstat_TcpExt_ListenOverflows{instance=~\"$node:$port\",job=~\"$job\"}[5m])",
+ "format": "time_series",
+ "hide": false,
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "ListenOverflows - Times the listen queue of a socket overflowed",
+ "refId": "A",
+ "step": 4
+ },
+ {
+ "expr": "irate(node_netstat_TcpExt_ListenDrops{instance=~\"$node:$port\",job=~\"$job\"}[5m])",
+ "format": "time_series",
+ "hide": false,
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "ListenDrops - SYNs to LISTEN sockets ignored",
+ "refId": "B",
+ "step": 4
+ },
+ {
+ "expr": "irate(node_netstat_TcpExt_TCPSynRetrans{instance=~\"$node:$port\",job=~\"$job\"}[5m])",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "TCPSynRetrans - SYN-SYN/ACK retransmits to break down retransmissions in SYN, fast/timeout retransmits",
+ "refId": "C",
+ "step": 4
+ },
+ {
+ "expr": "irate(node_netstat_Tcp_RetransSegs{instance=~\"$node:$port\",job=~\"$job\"}[5m])",
+ "interval": "",
+ "legendFormat": "RetransSegs - Segments retransmitted - that is, the number of TCP segments transmitted containing one or more previously transmitted octets",
+ "refId": "D"
+ },
+ {
+ "expr": "irate(node_netstat_Tcp_InErrs{instance=~\"$node:$port\",job=~\"$job\"}[5m])",
+ "interval": "",
+ "legendFormat": "InErrs - Segments received in error (e.g., bad TCP checksums)",
+ "refId": "E"
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "TCP Errors",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "short",
+ "label": "counter",
+ "logBase": 1,
+ "max": null,
+ "min": "0",
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "fill": 2,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 0,
+ "y": 72
+ },
+ "height": "",
+ "hiddenSeries": false,
+ "id": 85,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "hideZero": false,
+ "max": true,
+ "min": true,
+ "rightSide": false,
+ "show": true,
+ "sortDesc": true,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "maxPerRow": 12,
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [
+ {
+ "alias": "/.*MaxConn *./",
+ "color": "#890F02",
+ "fill": 0
+ }
+ ],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "node_netstat_Tcp_CurrEstab{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "hide": false,
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "CurrEstab - TCP connections for which the current state is either ESTABLISHED or CLOSE- WAIT",
+ "refId": "A",
+ "step": 4
+ },
+ {
+ "expr": "node_netstat_Tcp_MaxConn{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "hide": false,
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "MaxConn - Limit on the total number of TCP connections the entity can support (Dinamic is \"-1\")",
+ "refId": "B",
+ "step": 4
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "TCP Connections",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "short",
+ "label": "connections",
+ "logBase": 1,
+ "max": null,
+ "min": "0",
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "description": "",
+ "fill": 2,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 12,
+ "y": 72
+ },
+ "height": "",
+ "hiddenSeries": false,
+ "id": 91,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "hideEmpty": false,
+ "hideZero": false,
+ "max": true,
+ "min": true,
+ "rightSide": false,
+ "show": true,
+ "sort": "current",
+ "sortDesc": true,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "maxPerRow": 12,
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [
+ {
+ "alias": "/.*Sent.*/",
+ "transform": "negative-Y"
+ }
+ ],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "irate(node_netstat_TcpExt_SyncookiesFailed{instance=~\"$node:$port\",job=~\"$job\"}[5m])",
+ "format": "time_series",
+ "hide": false,
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "SyncookiesFailed - Invalid SYN cookies received",
+ "refId": "A",
+ "step": 4
+ },
+ {
+ "expr": "irate(node_netstat_TcpExt_SyncookiesRecv{instance=~\"$node:$port\",job=~\"$job\"}[5m])",
+ "format": "time_series",
+ "hide": false,
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "SyncookiesRecv - SYN cookies received",
+ "refId": "B",
+ "step": 4
+ },
+ {
+ "expr": "irate(node_netstat_TcpExt_SyncookiesSent{instance=~\"$node:$port\",job=~\"$job\"}[5m])",
+ "format": "time_series",
+ "hide": false,
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "SyncookiesSent - SYN cookies sent",
+ "refId": "C",
+ "step": 4
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "TCP SynCookie",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "short",
+ "label": "counter out (-) / in (+)",
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "fill": 2,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 0,
+ "y": 82
+ },
+ "height": "",
+ "hiddenSeries": false,
+ "id": 82,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "hideZero": false,
+ "max": true,
+ "min": true,
+ "rightSide": false,
+ "show": true,
+ "sortDesc": true,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "maxPerRow": 12,
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "irate(node_netstat_Tcp_ActiveOpens{instance=~\"$node:$port\",job=~\"$job\"}[5m])",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "ActiveOpens - TCP connections that have made a direct transition to the SYN-SENT state from the CLOSED state",
+ "refId": "A",
+ "step": 4
+ },
+ {
+ "expr": "irate(node_netstat_Tcp_PassiveOpens{instance=~\"$node:$port\",job=~\"$job\"}[5m])",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "PassiveOpens - TCP connections that have made a direct transition to the SYN-RCVD state from the LISTEN state",
+ "refId": "B",
+ "step": 4
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "TCP Direct Transition",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "short",
+ "label": "connections",
+ "logBase": 1,
+ "max": null,
+ "min": "0",
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ }
+ ],
+ "repeat": null,
+ "title": "Network Netstat",
+ "type": "row"
+ },
+ {
+ "collapsed": true,
+ "datasource": "${DS_PROMETHEUS}",
+ "gridPos": {
+ "h": 1,
+ "w": 24,
+ "x": 0,
+ "y": 71
+ },
+ "id": 279,
+ "panels": [
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "description": "",
+ "fieldConfig": {
+ "defaults": {
+ "custom": {}
+ },
+ "overrides": []
+ },
+ "fill": 2,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 0,
+ "y": 154
+ },
+ "hiddenSeries": false,
+ "id": 40,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "max": true,
+ "min": true,
+ "show": true,
+ "sort": "current",
+ "sortDesc": true,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "stack": true,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "node_scrape_collector_duration_seconds{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "hide": false,
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "{{collector}} - Scrape duration",
+ "refId": "A",
+ "step": 4
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "Node Exporter Scrape Time",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "s",
+ "label": "seconds",
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": "${DS_PROMETHEUS}",
+ "description": "",
+ "fieldConfig": {
+ "defaults": {
+ "custom": {}
+ },
+ "overrides": []
+ },
+ "fill": 2,
+ "fillGradient": 0,
+ "gridPos": {
+ "h": 10,
+ "w": 12,
+ "x": 12,
+ "y": 154
+ },
+ "hiddenSeries": false,
+ "id": 157,
+ "legend": {
+ "alignAsTable": true,
+ "avg": true,
+ "current": true,
+ "max": true,
+ "min": true,
+ "show": true,
+ "total": false,
+ "values": true
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "nullPointMode": "null",
+ "options": {
+ "dataLinks": []
+ },
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [
+ {
+ "alias": "/.*error.*/",
+ "color": "#F2495C",
+ "transform": "negative-Y"
+ }
+ ],
+ "spaceLength": 10,
+ "stack": true,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "node_scrape_collector_success{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "hide": false,
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "{{collector}} - Scrape success",
+ "refId": "A",
+ "step": 4
+ },
+ {
+ "expr": "node_textfile_scrape_error{instance=~\"$node:$port\",job=~\"$job\"}",
+ "format": "time_series",
+ "hide": false,
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "{{collector}} - Scrape textfile error (1 = true)",
+ "refId": "B",
+ "step": 4
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeRegions": [],
+ "timeShift": null,
+ "title": "Node Exporter Scrape",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "short",
+ "label": "counter",
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": false
+ }
+ ],
+ "yaxis": {
+ "align": false,
+ "alignLevel": null
+ }
+ }
+ ],
+ "repeat": null,
+ "title": "Node Exporter",
+ "type": "row"
+ }
+ ],
+ "refresh": false,
+ "schemaVersion": 25,
+ "style": "dark",
+ "tags": [
+ "linux"
+ ],
+ "templating": {
+ "list": [
+ {
+ "current": {
+ "selected": false,
+ "text": "Prometheus",
+ "value": "Prometheus"
+ },
+ "hide": 0,
+ "includeAll": false,
+ "label": "datasource",
+ "multi": false,
+ "name": "DS_PROMETHEUS",
+ "options": [],
+ "query": "prometheus",
+ "queryValue": "",
+ "refresh": 1,
+ "regex": "",
+ "skipUrlSync": false,
+ "type": "datasource"
+ },
+ {
+ "allValue": null,
+ "current": {
+ "selected": false,
+ "text": "kafka",
+ "value": "kafka"
+ },
+ "datasource": "${DS_PROMETHEUS}",
+ "definition": "",
+ "hide": 0,
+ "includeAll": false,
+ "label": "Job",
+ "multi": false,
+ "name": "job",
+ "options": [],
+ "query": "label_values(node_uname_info, job)",
+ "refresh": 1,
+ "regex": "",
+ "skipUrlSync": false,
+ "sort": 1,
+ "tagValuesQuery": "",
+ "tags": [],
+ "tagsQuery": "",
+ "type": "query",
+ "useTags": false
+ },
+ {
+ "allValue": null,
+ "current": {
+ "selected": false,
+ "text": "ip-10-0-0-15.us-west-2.compute.internal",
+ "value": "ip-10-0-0-15.us-west-2.compute.internal"
+ },
+ "datasource": "${DS_PROMETHEUS}",
+ "definition": "",
+ "hide": 0,
+ "includeAll": false,
+ "label": "Host:",
+ "multi": false,
+ "name": "name",
+ "options": [],
+ "query": "label_values(node_uname_info{job=~\"$job\"}, nodename)",
+ "refresh": 1,
+ "regex": "",
+ "skipUrlSync": false,
+ "sort": 1,
+ "tagValuesQuery": "",
+ "tags": [],
+ "tagsQuery": "",
+ "type": "query",
+ "useTags": false
+ },
+ {
+ "allValue": null,
+ "current": {
+ "selected": false,
+ "text": "10.0.0.15",
+ "value": "10.0.0.15"
+ },
+ "datasource": "${DS_PROMETHEUS}",
+ "definition": "",
+ "hide": 2,
+ "includeAll": false,
+ "label": "Host:",
+ "multi": false,
+ "name": "node",
+ "options": [],
+ "query": "label_values(node_uname_info{nodename=\"$name\"}, instance)",
+ "refresh": 1,
+ "regex": "/([^:]+):.*/",
+ "skipUrlSync": false,
+ "sort": 1,
+ "tagValuesQuery": "",
+ "tags": [],
+ "tagsQuery": "",
+ "type": "query",
+ "useTags": false
+ },
+ {
+ "allValue": null,
+ "current": {
+ "selected": false,
+ "text": "9100",
+ "value": "9100"
+ },
+ "datasource": "${DS_PROMETHEUS}",
+ "definition": "",
+ "hide": 0,
+ "includeAll": false,
+ "label": "Port",
+ "multi": false,
+ "name": "port",
+ "options": [],
+ "query": "label_values(node_uname_info{instance=~\"$node:(.*)\"}, instance)",
+ "refresh": 1,
+ "regex": "/[^:]+:(.*)/",
+ "skipUrlSync": false,
+ "sort": 3,
+ "tagValuesQuery": "",
+ "tags": [],
+ "tagsQuery": "",
+ "type": "query",
+ "useTags": false
+ }
+ ]
+ },
+ "time": {
+ "from": "now-3h",
+ "to": "now"
+ },
+ "timepicker": {
+ "refresh_intervals": [
+ "10s",
+ "30s",
+ "1m",
+ "5m",
+ "15m",
+ "30m",
+ "1h",
+ "2h",
+ "1d"
+ ],
+ "time_options": [
+ "5m",
+ "15m",
+ "1h",
+ "6h",
+ "12h",
+ "24h",
+ "2d",
+ "7d",
+ "30d"
+ ]
+ },
+ "timezone": "browser",
+ "title": "Node Exporter Full",
+ "uid": "rYdddlPWk",
+ "version": 4
+}
\ No newline at end of file
diff --git a/driver-msk/deploy/monitoring/dashboards/zookeeper.json b/driver-msk/deploy/monitoring/dashboards/zookeeper.json
new file mode 100644
index 00000000..d8a4ab43
--- /dev/null
+++ b/driver-msk/deploy/monitoring/dashboards/zookeeper.json
@@ -0,0 +1,953 @@
+{
+ "annotations": {
+ "list": []
+ },
+ "editable": true,
+ "gnetId": null,
+ "graphTooltip": 0,
+ "hideControls": false,
+ "id": 3,
+ "links": [],
+ "rows": [
+ {
+ "collapse": false,
+ "height": 236,
+ "panels": [
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": null,
+ "decimals": 1,
+ "description": "All write operations (create / setData / ...)",
+ "fill": 0,
+ "id": 1,
+ "legend": {
+ "avg": false,
+ "current": false,
+ "max": false,
+ "min": false,
+ "show": false,
+ "total": false,
+ "values": false
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "nullPointMode": "null",
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "span": 6,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "irate(zookeeper_server_requests_latency_ms_count{cluster=~\"$cluster\", kubernetes_pod_name=~\"$server\", type=\"write\"}[30s])",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "{{kubernetes_pod_name}}",
+ "metric": "zookeeper_server_requests_latency_ms_count",
+ "refId": "A",
+ "step": 10
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeShift": null,
+ "title": "Write Requests",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "short",
+ "label": "writes / s",
+ "logBase": 1,
+ "max": null,
+ "min": "0",
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ }
+ ]
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": null,
+ "description": "All read operations (getData / getChildren / ...)",
+ "fill": 0,
+ "id": 2,
+ "legend": {
+ "avg": false,
+ "current": false,
+ "max": false,
+ "min": false,
+ "show": false,
+ "total": false,
+ "values": false
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "nullPointMode": "null",
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "span": 6,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "irate(zookeeper_server_requests_latency_ms_count{cluster=~\"$cluster\", kubernetes_pod_name=~\"$server\", type=\"read\"}[30s])",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "{{kubernetes_pod_name}}",
+ "metric": "zookeeper_server_requests_latency_ms_count",
+ "refId": "A",
+ "step": 10
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeShift": null,
+ "title": "Read Requests",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "short",
+ "label": "reads / s",
+ "logBase": 1,
+ "max": null,
+ "min": "0",
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ }
+ ]
+ }
+ ],
+ "repeat": null,
+ "repeatIteration": null,
+ "repeatRowId": null,
+ "showTitle": false,
+ "title": "Dashboard Row",
+ "titleSize": "h6"
+ },
+ {
+ "collapse": false,
+ "height": 250,
+ "panels": [
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": null,
+ "description": "All write operations (create / setData / ...)",
+ "fill": 0,
+ "id": 4,
+ "legend": {
+ "avg": false,
+ "current": false,
+ "max": false,
+ "min": false,
+ "show": false,
+ "total": false,
+ "values": false
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "nullPointMode": "null as zero",
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "span": 6,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "zookeeper_server_requests_latency_ms{cluster=~\"$cluster\", kubernetes_pod_name=~\"$server\", type=\"write\", quantile=\"0.99\"}",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "{{kubernetes_pod_name}}",
+ "metric": "zookeeper_server_requests_latency_ms_count",
+ "refId": "A",
+ "step": 10
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeShift": null,
+ "title": "Write Latency - 99pct",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "ms",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": "0",
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ }
+ ]
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": null,
+ "decimals": 1,
+ "description": "All write operations (create / setData / ...)",
+ "fill": 0,
+ "id": 5,
+ "legend": {
+ "avg": false,
+ "current": false,
+ "max": false,
+ "min": false,
+ "show": false,
+ "total": false,
+ "values": false
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "nullPointMode": "null",
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "span": 6,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "zookeeper_server_requests_latency_ms{cluster=~\"$cluster\", kubernetes_pod_name=~\"$server\", type=\"read\", quantile=\"0.99\"}",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "{{kubernetes_pod_name}}",
+ "metric": "zookeeper_server_requests_latency_ms_count",
+ "refId": "A",
+ "step": 10
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeShift": null,
+ "title": "Read Latency - 99pct",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "ms",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": "0",
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ }
+ ]
+ }
+ ],
+ "repeat": null,
+ "repeatIteration": null,
+ "repeatRowId": null,
+ "showTitle": false,
+ "title": "Dashboard Row",
+ "titleSize": "h6"
+ },
+ {
+ "collapse": false,
+ "height": 250,
+ "panels": [
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": null,
+ "decimals": 1,
+ "description": "",
+ "fill": 1,
+ "id": 6,
+ "legend": {
+ "avg": false,
+ "current": false,
+ "max": false,
+ "min": false,
+ "show": true,
+ "total": false,
+ "values": false
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "nullPointMode": "null",
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "span": 12,
+ "stack": true,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "sum(irate(zookeeper_server_requests{cluster=~\"$cluster\", kubernetes_pod_name=~\"$server\"}[30s])) by (type)",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "{{type}}",
+ "metric": "zookeeper_server_requests_latency_ms_count",
+ "refId": "A",
+ "step": 4
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeShift": null,
+ "title": "Request Types",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "short",
+ "label": "",
+ "logBase": 1,
+ "max": null,
+ "min": "0",
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ }
+ ]
+ }
+ ],
+ "repeat": null,
+ "repeatIteration": null,
+ "repeatRowId": null,
+ "showTitle": false,
+ "title": "Dashboard Row",
+ "titleSize": "h6"
+ },
+ {
+ "collapse": false,
+ "height": 250,
+ "panels": [
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": null,
+ "description": "",
+ "fill": 0,
+ "id": 7,
+ "legend": {
+ "avg": false,
+ "current": false,
+ "max": false,
+ "min": false,
+ "show": false,
+ "total": false,
+ "values": false
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "nullPointMode": "null",
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "span": 6,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "zookeeper_server_znode_count{cluster=~\"$cluster\", kubernetes_pod_name=~\"$server\"}",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "{{cluster}} - {{kubernetes_pod_name}}",
+ "metric": "zookeeper_server_requests_latency_ms_count",
+ "refId": "A",
+ "step": 10
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeShift": null,
+ "title": "Z-Nodes count",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "short",
+ "label": "",
+ "logBase": 1,
+ "max": null,
+ "min": "0",
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ }
+ ]
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": null,
+ "description": "Size of the data stored in a ZK server",
+ "fill": 0,
+ "id": 8,
+ "legend": {
+ "avg": false,
+ "current": false,
+ "max": false,
+ "min": false,
+ "show": false,
+ "total": false,
+ "values": false
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "nullPointMode": "null",
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "span": 6,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "zookeeper_server_data_size_bytes{cluster=~\"$cluster\", kubernetes_pod_name=~\"$server\"}",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "{{cluster}} - {{kubernetes_pod_name}}",
+ "metric": "zookeeper_server_requests_latency_ms_count",
+ "refId": "A",
+ "step": 10
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeShift": null,
+ "title": "Data set size",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "decbytes",
+ "label": "",
+ "logBase": 1,
+ "max": null,
+ "min": "0",
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ }
+ ]
+ }
+ ],
+ "repeat": null,
+ "repeatIteration": null,
+ "repeatRowId": null,
+ "showTitle": false,
+ "title": "Dashboard Row",
+ "titleSize": "h6"
+ },
+ {
+ "collapse": false,
+ "height": 250,
+ "panels": [
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": null,
+ "description": "Number of opened connections",
+ "fill": 1,
+ "id": 9,
+ "legend": {
+ "avg": false,
+ "current": false,
+ "max": false,
+ "min": false,
+ "show": false,
+ "total": false,
+ "values": false
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "nullPointMode": "null",
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "span": 4,
+ "stack": true,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "zookeeper_server_connections{cluster=~\"$cluster\", kubernetes_pod_name=~\"$server\"}",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "{{cluster}} - {{kubernetes_pod_name}}",
+ "metric": "zookeeper_server_requests_latency_ms_count",
+ "refId": "A",
+ "step": 10
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeShift": null,
+ "title": "Connections",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "short",
+ "label": "",
+ "logBase": 1,
+ "max": null,
+ "min": "0",
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ }
+ ]
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": null,
+ "description": "",
+ "fill": 1,
+ "id": 10,
+ "legend": {
+ "avg": false,
+ "current": false,
+ "max": false,
+ "min": false,
+ "show": false,
+ "total": false,
+ "values": false
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "nullPointMode": "null",
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "span": 4,
+ "stack": true,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "zookeeper_server_watches_count{cluster=~\"$cluster\", kubernetes_pod_name=~\"$server\"}",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "{{cluster}} - {{kubernetes_pod_name}}",
+ "metric": "zookeeper_server_requests_latency_ms_count",
+ "refId": "A",
+ "step": 10
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeShift": null,
+ "title": "Number of watches",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "short",
+ "label": "",
+ "logBase": 1,
+ "max": null,
+ "min": "0",
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ }
+ ]
+ },
+ {
+ "aliasColors": {},
+ "bars": false,
+ "dashLength": 10,
+ "dashes": false,
+ "datasource": null,
+ "description": "",
+ "fill": 0,
+ "id": 11,
+ "legend": {
+ "avg": false,
+ "current": false,
+ "max": false,
+ "min": false,
+ "show": false,
+ "total": false,
+ "values": false
+ },
+ "lines": true,
+ "linewidth": 1,
+ "links": [],
+ "nullPointMode": "null",
+ "percentage": false,
+ "pointradius": 5,
+ "points": false,
+ "renderer": "flot",
+ "seriesOverrides": [],
+ "spaceLength": 10,
+ "span": 4,
+ "stack": false,
+ "steppedLine": false,
+ "targets": [
+ {
+ "expr": "zookeeper_server_ephemerals_count{cluster=~\"$cluster\", kubernetes_pod_name=~\"$server\"}",
+ "format": "time_series",
+ "interval": "",
+ "intervalFactor": 2,
+ "legendFormat": "{{cluster}} - {{kubernetes_pod_name}}",
+ "metric": "zookeeper_server_requests_latency_ms_count",
+ "refId": "A",
+ "step": 10
+ }
+ ],
+ "thresholds": [],
+ "timeFrom": null,
+ "timeShift": null,
+ "title": "Number of ephemerals",
+ "tooltip": {
+ "shared": true,
+ "sort": 0,
+ "value_type": "individual"
+ },
+ "type": "graph",
+ "xaxis": {
+ "buckets": null,
+ "mode": "time",
+ "name": null,
+ "show": true,
+ "values": []
+ },
+ "yaxes": [
+ {
+ "format": "short",
+ "label": "",
+ "logBase": 1,
+ "max": null,
+ "min": "0",
+ "show": true
+ },
+ {
+ "format": "short",
+ "label": null,
+ "logBase": 1,
+ "max": null,
+ "min": null,
+ "show": true
+ }
+ ]
+ }
+ ],
+ "repeat": null,
+ "repeatIteration": null,
+ "repeatRowId": null,
+ "showTitle": false,
+ "title": "Dashboard Row",
+ "titleSize": "h6"
+ }
+ ],
+ "schemaVersion": 14,
+ "style": "dark",
+ "tags": [
+ "zookeeper"
+ ],
+ "templating": {
+ "list": [
+ {
+ "allValue": null,
+ "current": {
+ "text": "us-central",
+ "value": "us-central"
+ },
+ "datasource": "Prometheus",
+ "hide": 0,
+ "includeAll": false,
+ "label": "Cluster",
+ "multi": false,
+ "name": "cluster",
+ "options": [],
+ "query": "{cluster=~\".+\"}",
+ "refresh": 1,
+ "regex": "/.*[^_]cluster=\\\"([^\\\"]+)\\\".*/",
+ "sort": 1,
+ "tagValuesQuery": "",
+ "tags": [],
+ "tagsQuery": "",
+ "type": "query",
+ "useTags": false
+ },
+ {
+ "allValue": null,
+ "current": {
+ "text": "All",
+ "value": "$__all"
+ },
+ "datasource": "Prometheus",
+ "hide": 0,
+ "includeAll": true,
+ "label": "ZK Server",
+ "multi": false,
+ "name": "server",
+ "options": [],
+ "query": "zookeeper_server_requests_latency_ms_count{cluster=~\"$cluster\", kubernetes_pod_name=~\".+\"}",
+ "refresh": 2,
+ "regex": "/.*[^_]kubernetes_pod_name=\\\"([^\\\"]+)\\\".*/",
+ "sort": 1,
+ "tagValuesQuery": "",
+ "tags": [],
+ "tagsQuery": "",
+ "type": "query",
+ "useTags": false
+ }
+ ]
+ },
+ "time": {
+ "from": "now-1h",
+ "to": "now"
+ },
+ "timepicker": {
+ "refresh_intervals": [
+ "5s",
+ "10s",
+ "30s",
+ "1m",
+ "5m",
+ "15m",
+ "30m",
+ "1h",
+ "2h",
+ "1d"
+ ],
+ "time_options": [
+ "5m",
+ "15m",
+ "1h",
+ "6h",
+ "12h",
+ "24h",
+ "2d",
+ "7d",
+ "30d"
+ ]
+ },
+ "timezone": "browser",
+ "title": "ZooKeeper",
+ "version": 11
+}
\ No newline at end of file
diff --git a/driver-msk/deploy/monitoring/grafana.ini b/driver-msk/deploy/monitoring/grafana.ini
new file mode 100644
index 00000000..e28ca88b
--- /dev/null
+++ b/driver-msk/deploy/monitoring/grafana.ini
@@ -0,0 +1,39 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements. See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership. The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied. See the License for the
+# specific language governing permissions and limitations
+# under the License.
+#
+
+[users]
+# disable user signup / registration
+allow_sign_up = false
+
+# Default UI theme ("dark" or "light")
+default_theme = dark
+
+[auth.anonymous]
+# enable anonymous access
+enabled = true
+
+# specify organization name that should be used for unauthenticated users
+org_name = Main Org.
+
+# specify role for unauthenticated users
+org_role = Viewer
+
+[dashboards]
+enabled = true
+path = /var/lib/grafana/dashboards
\ No newline at end of file
diff --git a/driver-msk/deploy/monitoring/jmx_exporter/client.yml b/driver-msk/deploy/monitoring/jmx_exporter/client.yml
new file mode 100644
index 00000000..792f8af8
--- /dev/null
+++ b/driver-msk/deploy/monitoring/jmx_exporter/client.yml
@@ -0,0 +1,22 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements. See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership. The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied. See the License for the
+# specific language governing permissions and limitations
+# under the License.
+#
+
+lowercaseOutputLabelNames: false
+lowercaseOutputName: true
+## Export everything
diff --git a/driver-msk/deploy/monitoring/jmx_exporter/jmx_prometheus_javaagent-0.13.0.jar b/driver-msk/deploy/monitoring/jmx_exporter/jmx_prometheus_javaagent-0.13.0.jar
new file mode 100644
index 00000000..c55a92e5
Binary files /dev/null and b/driver-msk/deploy/monitoring/jmx_exporter/jmx_prometheus_javaagent-0.13.0.jar differ
diff --git a/driver-msk/deploy/monitoring/jmx_exporter/metrics.yml b/driver-msk/deploy/monitoring/jmx_exporter/metrics.yml
new file mode 100644
index 00000000..cbde974b
--- /dev/null
+++ b/driver-msk/deploy/monitoring/jmx_exporter/metrics.yml
@@ -0,0 +1,122 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements. See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership. The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied. See the License for the
+# specific language governing permissions and limitations
+# under the License.
+#
+
+lowercaseOutputName: true
+
+rules:
+ # Special cases and very specific rules
+ - pattern: kafka.server<>Value
+ name: kafka_server_$1_$2
+ type: GAUGE
+ labels:
+ clientId: "$3"
+ topic: "$4"
+ partition: "$5"
+ - pattern: kafka.server<>Value
+ name: kafka_server_$1_$2
+ type: GAUGE
+ labels:
+ clientId: "$3"
+ broker: "$4:$5"
+ - pattern: kafka.coordinator.(\w+)<>Value
+ name: kafka_coordinator_$1_$2_$3
+ type: GAUGE
+
+ # Generic per-second counters with 0-2 key/value pairs
+ - pattern: kafka.(\w+)<>Count
+ name: kafka_$1_$2_$3_total
+ type: COUNTER
+ labels:
+ "$4": "$5"
+ "$6": "$7"
+ - pattern: kafka.(\w+)<>Count
+ name: kafka_$1_$2_$3_total
+ type: COUNTER
+ labels:
+ "$4": "$5"
+ - pattern: kafka.(\w+)<>Count
+ name: kafka_$1_$2_$3_total
+ type: COUNTER
+
+ - pattern: kafka.server<>([a-z-]+)
+ name: kafka_server_quota_$3
+ type: GAUGE
+ labels:
+ resource: "$1"
+ clientId: "$2"
+
+ - pattern: kafka.server<>([a-z-]+)
+ name: kafka_server_quota_$4
+ type: GAUGE
+ labels:
+ resource: "$1"
+ user: "$2"
+ clientId: "$3"
+
+ # Generic gauges with 0-2 key/value pairs
+ - pattern: kafka.(\w+)<>Value
+ name: kafka_$1_$2_$3
+ type: GAUGE
+ labels:
+ "$4": "$5"
+ "$6": "$7"
+ - pattern: kafka.(\w+)<>Value
+ name: kafka_$1_$2_$3
+ type: GAUGE
+ labels:
+ "$4": "$5"
+ - pattern: kafka.(\w+)<>Value
+ name: kafka_$1_$2_$3
+ type: GAUGE
+
+ # Emulate Prometheus 'Summary' metrics for the exported 'Histogram's.
+ #
+ # Note that these are missing the '_sum' metric!
+ - pattern: kafka.(\w+)<>Count
+ name: kafka_$1_$2_$3_count
+ type: COUNTER
+ labels:
+ "$4": "$5"
+ "$6": "$7"
+ - pattern: kafka.(\w+)<>(\d+)thPercentile
+ name: kafka_$1_$2_$3
+ type: GAUGE
+ labels:
+ "$4": "$5"
+ "$6": "$7"
+ quantile: "0.$8"
+ - pattern: kafka.(\w+)<>Count
+ name: kafka_$1_$2_$3_count
+ type: COUNTER
+ labels:
+ "$4": "$5"
+ - pattern: kafka.(\w+)<>(\d+)thPercentile
+ name: kafka_$1_$2_$3
+ type: GAUGE
+ labels:
+ "$4": "$5"
+ quantile: "0.$6"
+ - pattern: kafka.(\w+)<>Count
+ name: kafka_$1_$2_$3_count
+ type: COUNTER
+ - pattern: kafka.(\w+)<>(\d+)thPercentile
+ name: kafka_$1_$2_$3
+ type: GAUGE
+ labels:
+ quantile: "0.$4"
diff --git a/driver-msk/deploy/monitoring/jmx_exporter/zookeeper.yml b/driver-msk/deploy/monitoring/jmx_exporter/zookeeper.yml
new file mode 100644
index 00000000..22b82b72
--- /dev/null
+++ b/driver-msk/deploy/monitoring/jmx_exporter/zookeeper.yml
@@ -0,0 +1,123 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements. See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership. The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied. See the License for the
+# specific language governing permissions and limitations
+# under the License.
+#
+
+lowercaseOutputLabelNames: false
+lowercaseOutputName: true
+## Export everything
+# rules:
+# # Special cases and very specific rules
+# - pattern: kafka.server<>Value
+# name: kafka_server_$1_$2
+# type: GAUGE
+# labels:
+# clientId: "$3"
+# topic: "$4"
+# partition: "$5"
+# - pattern: kafka.server<>Value
+# name: kafka_server_$1_$2
+# type: GAUGE
+# labels:
+# clientId: "$3"
+# broker: "$4:$5"
+# - pattern: kafka.coordinator.(\w+)<>Value
+# name: kafka_coordinator_$1_$2_$3
+# type: GAUGE
+
+# # Generic per-second counters with 0-2 key/value pairs
+# - pattern: kafka.(\w+)<>Count
+# name: kafka_$1_$2_$3_total
+# type: COUNTER
+# labels:
+# "$4": "$5"
+# "$6": "$7"
+# - pattern: kafka.(\w+)<>Count
+# name: kafka_$1_$2_$3_total
+# type: COUNTER
+# labels:
+# "$4": "$5"
+# - pattern: kafka.(\w+)<>Count
+# name: kafka_$1_$2_$3_total
+# type: COUNTER
+
+# - pattern: kafka.server<>([a-z-]+)
+# name: kafka_server_quota_$3
+# type: GAUGE
+# labels:
+# resource: "$1"
+# clientId: "$2"
+
+# - pattern: kafka.server<>([a-z-]+)
+# name: kafka_server_quota_$4
+# type: GAUGE
+# labels:
+# resource: "$1"
+# user: "$2"
+# clientId: "$3"
+
+# # Generic gauges with 0-2 key/value pairs
+# - pattern: kafka.(\w+)<>Value
+# name: kafka_$1_$2_$3
+# type: GAUGE
+# labels:
+# "$4": "$5"
+# "$6": "$7"
+# - pattern: kafka.(\w+)<>Value
+# name: kafka_$1_$2_$3
+# type: GAUGE
+# labels:
+# "$4": "$5"
+# - pattern: kafka.(\w+)<>Value
+# name: kafka_$1_$2_$3
+# type: GAUGE
+
+# # Emulate Prometheus 'Summary' metrics for the exported 'Histogram's.
+# #
+# # Note that these are missing the '_sum' metric!
+# - pattern: kafka.(\w+)<>Count
+# name: kafka_$1_$2_$3_count
+# type: COUNTER
+# labels:
+# "$4": "$5"
+# "$6": "$7"
+# - pattern: kafka.(\w+)<>(\d+)thPercentile
+# name: kafka_$1_$2_$3
+# type: GAUGE
+# labels:
+# "$4": "$5"
+# "$6": "$7"
+# quantile: "0.$8"
+# - pattern: kafka.(\w+)<>Count
+# name: kafka_$1_$2_$3_count
+# type: COUNTER
+# labels:
+# "$4": "$5"
+# - pattern: kafka.(\w+)<>(\d+)thPercentile
+# name: kafka_$1_$2_$3
+# type: GAUGE
+# labels:
+# "$4": "$5"
+# quantile: "0.$6"
+# - pattern: kafka.(\w+)<>Count
+# name: kafka_$1_$2_$3_count
+# type: COUNTER
+# - pattern: kafka.(\w+)<>(\d+)thPercentile
+# name: kafka_$1_$2_$3
+# type: GAUGE
+# labels:
+# quantile: "0.$4"
diff --git a/driver-msk/deploy/monitoring/start.sh b/driver-msk/deploy/monitoring/start.sh
new file mode 100755
index 00000000..8cc5a499
--- /dev/null
+++ b/driver-msk/deploy/monitoring/start.sh
@@ -0,0 +1,39 @@
+#!/bin/bash
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements. See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership. The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied. See the License for the
+# specific language governing permissions and limitations
+# under the License.
+#
+
+
+echo 'Starting Grafana...'
+
+/run.sh "$@" &
+
+AddDataSource() {
+ curl 'http://admin:admin@localhost:3000/api/datasources' \
+ -X POST \
+ -H 'Content-Type: application/json;charset=UTF-8' \
+ --data-binary \
+ "{\"name\":\"Prometheus\",\"type\":\"prometheus\",\"url\":\"$PROMETHEUS_URL\",\"access\":\"proxy\",\"isDefault\":true}"
+}
+
+until AddDataSource; do
+ echo 'Configuring Grafana...'
+ sleep 1
+done
+ echo 'Done!'
+wait
\ No newline at end of file
diff --git a/driver-msk/deploy/provision-msk.tf b/driver-msk/deploy/provision-msk.tf
new file mode 100644
index 00000000..69b42557
--- /dev/null
+++ b/driver-msk/deploy/provision-msk.tf
@@ -0,0 +1,267 @@
+provider "aws" {
+ region = "${var.region}"
+}
+
+provider "random" {
+}
+
+variable "public_key_path" {}
+
+variable "key_name" {
+ default = "kafka-benchmark-key"
+ description = "Desired name prefix for the AWS key pair"
+}
+
+variable "region" {}
+
+variable "ami" {}
+
+variable "profile" {}
+
+variable "kafka_version" {}
+
+variable "instance_types" {
+ type = map(string)
+}
+
+variable "num_instances" {
+ type = map(number)
+}
+
+data "aws_availability_zones" "azs" {
+ state = "available"
+}
+
+resource "random_id" "hash" {
+ byte_length = 8
+}
+
+resource "aws_vpc" "vpc" {
+ cidr_block = "192.168.0.0/22"
+}
+
+# Create an internet gateway to give our subnet access to the outside world
+resource "aws_internet_gateway" "msk-bench" {
+ vpc_id = "${aws_vpc.vpc.id}"
+}
+
+# Grant the VPC internet access on its main route table
+resource "aws_route" "internet_access" {
+ route_table_id = "${aws_vpc.vpc.main_route_table_id}"
+ destination_cidr_block = "0.0.0.0/0"
+ gateway_id = "${aws_internet_gateway.msk-bench.id}"
+}
+
+resource "aws_subnet" "subnet_az1" {
+ availability_zone = data.aws_availability_zones.azs.names[0]
+ cidr_block = "192.168.0.0/24"
+ vpc_id = aws_vpc.vpc.id
+}
+
+resource "aws_subnet" "subnet_az2" {
+ availability_zone = data.aws_availability_zones.azs.names[1]
+ cidr_block = "192.168.1.0/24"
+ vpc_id = aws_vpc.vpc.id
+}
+
+resource "aws_subnet" "subnet_az3" {
+ availability_zone = data.aws_availability_zones.azs.names[2]
+ cidr_block = "192.168.2.0/24"
+ vpc_id = aws_vpc.vpc.id
+}
+
+#resource "aws_subnet" "subnet_clients" {
+# availability_zone = data.aws_availability_zones.azs.names[0]
+# cidr_block = "192.168.3.0/24"
+# vpc_id = aws_vpc.vpc.id
+# map_public_ip_on_launch = true
+#}
+
+resource "aws_security_group" "sg" {
+ vpc_id = aws_vpc.vpc.id
+ name = "omb-sg-${random_id.hash.hex}"
+
+ # SSH access from anywhere
+ ingress {
+ from_port = 22
+ to_port = 22
+ protocol = "tcp"
+ cidr_blocks = ["0.0.0.0/0"]
+ }
+
+ # All ports open within the VPC
+ ingress {
+ from_port = 0
+ to_port = 65535
+ protocol = "tcp"
+ cidr_blocks = ["192.168.0.0/22"]
+ }
+
+ # outbound internet access
+ egress {
+ from_port = 0
+ to_port = 0
+ protocol = "-1"
+ cidr_blocks = ["0.0.0.0/0"]
+ }
+
+ ingress {
+ from_port = 0
+ to_port = 0
+ protocol = "-1"
+ security_groups = [aws_security_group.sg-client.id]
+ }
+}
+
+resource "aws_security_group" "sg-client" {
+ vpc_id = aws_vpc.vpc.id
+ name = "omb-client-sg-${random_id.hash.hex}"
+
+ # SSH access from anywhere
+ ingress {
+ from_port = 22
+ to_port = 22
+ protocol = "tcp"
+ cidr_blocks = ["0.0.0.0/0"]
+ }
+
+ # All ports open within the VPC
+ ingress {
+ from_port = 0
+ to_port = 65535
+ protocol = "tcp"
+ cidr_blocks = ["192.168.0.0/22"]
+ }
+
+ # All ports open to this machine
+ ingress {
+ from_port = 0
+ to_port = 65535
+ protocol = "tcp"
+ cidr_blocks = ["${chomp(data.http.myip.body)}/32"]
+ }
+
+ # Prometheus/Dashboard access
+ ingress {
+ from_port = 3000
+ to_port = 3000
+ protocol = "tcp"
+ cidr_blocks = ["0.0.0.0/0"]
+ }
+
+ # outbound internet access
+ egress {
+ from_port = 0
+ to_port = 0
+ protocol = "-1"
+ cidr_blocks = ["0.0.0.0/0"]
+ }
+
+ tags = {
+ Name = "Benchmark-Security-Group-${random_id.hash.hex}"
+ }
+}
+
+resource "aws_msk_cluster" "bench" {
+ cluster_name = "bench"
+ kafka_version = var.kafka_version
+ number_of_broker_nodes = "${var.num_instances["broker"]}"
+
+ broker_node_group_info {
+ instance_type = "${var.instance_types["broker"]}"
+
+ ebs_volume_size = 4000
+ client_subnets = [
+ aws_subnet.subnet_az1.id,
+ aws_subnet.subnet_az2.id,
+ aws_subnet.subnet_az3.id,
+ ]
+ security_groups = [aws_security_group.sg.id]
+ }
+
+ encryption_info {
+ encryption_in_transit {
+ client_broker = "PLAINTEXT"
+ in_cluster = false
+ }
+ }
+
+ open_monitoring {
+ prometheus {
+ jmx_exporter {
+ enabled_in_broker = true
+ }
+ node_exporter {
+ enabled_in_broker = true
+ }
+ }
+ }
+
+ tags = {
+ owner = "user"
+ }
+}
+
+###
+# Get public IP of this machine
+data "http" "myip" {
+ url = "http://ipv4.icanhazip.com"
+}
+
+resource "aws_key_pair" "auth" {
+ key_name = "${var.key_name}-${random_id.hash.hex}"
+ public_key = "${file(var.public_key_path)}"
+}
+
+resource "aws_instance" "client" {
+ ami = "${var.ami}"
+ instance_type = "${var.instance_types["client"]}"
+ key_name = "${aws_key_pair.auth.id}"
+ subnet_id = aws_subnet.subnet_az1.id
+ associate_public_ip_address = true
+ vpc_security_group_ids = [aws_security_group.sg-client.id]
+ count = "${var.num_instances["client"]}"
+ monitoring = true
+
+ tags = {
+ Name = "kafka-client-${count.index}"
+ }
+
+ root_block_device {
+ volume_size = 100
+ }
+}
+
+output "clients" {
+ value = {
+ for instance in aws_instance.client :
+ instance.public_ip => instance.private_ip
+ }
+}
+
+output "zookeeperServers" {
+ value = aws_msk_cluster.bench.zookeeper_connect_string
+}
+
+output "boostrapServers" {
+ value = aws_msk_cluster.bench.bootstrap_brokers
+}
+
+output "zookeeper" {
+ value = {
+ for instance in "${split (",", aws_msk_cluster.bench.zookeeper_connect_string)}" :
+ "${element (split (":",instance),0)}" => "${element (split (":",instance),0)}"
+ }
+}
+
+output "brokers" {
+ value = {
+ for instance in "${split (",", aws_msk_cluster.bench.bootstrap_brokers)}" :
+ "${element (split (":",instance),0)}" => "${element (split (":",instance),0)}"
+ }
+}
+
+
+#output "client_ssh_host" {
+# value = "${aws_instance.client.0.public_ip}"
+#}
diff --git a/driver-msk/deploy/templates/benchmark-worker.service b/driver-msk/deploy/templates/benchmark-worker.service
new file mode 100644
index 00000000..0a75fa8b
--- /dev/null
+++ b/driver-msk/deploy/templates/benchmark-worker.service
@@ -0,0 +1,14 @@
+[Unit]
+Description=Benchmark worker
+After=network.target
+
+[Service]
+ExecStart=/opt/benchmark/bin/benchmark-worker
+WorkingDirectory=/opt/benchmark
+RestartSec=1s
+Restart=on-failure
+Type=simple
+LimitNOFILE=300000
+
+[Install]
+WantedBy=multi-user.target
diff --git a/driver-msk/deploy/templates/chrony.conf b/driver-msk/deploy/templates/chrony.conf
new file mode 100644
index 00000000..0e68c66a
--- /dev/null
+++ b/driver-msk/deploy/templates/chrony.conf
@@ -0,0 +1,54 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements. See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership. The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied. See the License for the
+# specific language governing permissions and limitations
+# under the License.
+#
+
+# Use public servers from the pool.ntp.org project.
+# Please consider joining the pool (http://www.pool.ntp.org/join.html).
+server 169.254.169.123 prefer iburst
+
+# Record the rate at which the system clock gains/losses time.
+driftfile /var/lib/chrony/drift
+
+# Allow the system clock to be stepped in the first three updates
+# if its offset is larger than 1 second.
+makestep 1.0 3
+
+# Enable kernel synchronization of the real-time clock (RTC).
+rtcsync
+
+# Enable hardware timestamping on all interfaces that support it.
+#hwtimestamp *
+
+# Increase the minimum number of selectable sources required to adjust
+# the system clock.
+#minsources 2
+
+# Allow NTP client access from local network.
+#allow 192.168.0.0/16
+
+# Serve time even if not synchronized to a time source.
+#local stratum 10
+
+# Specify file containing keys for NTP authentication.
+#keyfile /etc/chrony.keys
+
+# Specify directory for log files.
+logdir /var/log/chrony
+
+# Select which information is logged.
+#log measurements statistics tracking
diff --git a/driver-msk/deploy/templates/kafka-dashboard.service b/driver-msk/deploy/templates/kafka-dashboard.service
new file mode 100644
index 00000000..8b00e459
--- /dev/null
+++ b/driver-msk/deploy/templates/kafka-dashboard.service
@@ -0,0 +1,16 @@
+[Unit]
+Description=Kafka Dashboard
+After=docker.service
+Requires=docker.service
+After=prometheus.service
+Requires=prometheus.service
+
+[Service]
+ExecStartPre=/usr/bin/docker pull aloknnikhil/omb-kafka-dashboard:latest
+ExecStart=/usr/bin/docker run --restart=always --name=systemd_kafka_dashboard -p3000:3000 -e PROMETHEUS_URL=http://{{ hostvars[groups['prometheus'][0]].private_ip }}:9090/ aloknnikhil/omb-kafka-dashboard:latest
+ExecStop=/usr/bin/docker stop systemd_kafka_dashboard
+ExecStopPost=/usr/bin/docker rm -f systemd_kafka_dashboard
+ExecReload=/usr/bin/docker restart systemd_kafka_dashboard
+
+[Install]
+WantedBy=multi-user.target
diff --git a/driver-msk/deploy/templates/kafka.service b/driver-msk/deploy/templates/kafka.service
new file mode 100644
index 00000000..3ed876d3
--- /dev/null
+++ b/driver-msk/deploy/templates/kafka.service
@@ -0,0 +1,19 @@
+[Unit]
+Description=Kafka
+After=network.target
+
+[Service]
+ExecStart=/opt/kafka/bin/kafka-server-start.sh config/server.properties
+Environment=JMX_PORT=9999
+Environment="KAFKA_OPTS=-javaagent:/opt/kafka/jmx_prometheus_javaagent-0.13.0.jar=8080:/opt/kafka/metrics.yml"
+Environment="KAFKA_JMX_OPTS=-Dcom.sun.management.jmxremote=true -Dcom.sun.management.jmxremote.authenticate=false -Dcom.sun.management.jmxremote.ssl=false -Djava.rmi.server.hostname=prod-kafka-001 -Djava.net.preferIPv4Stack=true"
+Environment='KAFKA_HEAP_OPTS=-Xms6g -Xmx6g -XX:MetaspaceSize=96m'
+Environment='KAFKA_JVM_PERFORMANCE_OPTS=-server -XX:+UseG1GC -XX:MaxGCPauseMillis=20 -XX:InitiatingHeapOccupancyPercent=35 -XX:G1HeapRegionSize=16M -XX:MinMetaspaceFreeRatio=50 -XX:MaxMetaspaceFreeRatio=80 -Djava.awt.headless=true"
+WorkingDirectory=/opt/kafka
+RestartSec=1s
+Restart=on-failure
+Type=simple
+LimitNOFILE=300000
+
+[Install]
+WantedBy=multi-user.target
diff --git a/driver-msk/deploy/templates/myid b/driver-msk/deploy/templates/myid
new file mode 100644
index 00000000..6b8ddbb2
--- /dev/null
+++ b/driver-msk/deploy/templates/myid
@@ -0,0 +1 @@
+{{ zid }}
diff --git a/driver-msk/deploy/templates/prometheus.service b/driver-msk/deploy/templates/prometheus.service
new file mode 100644
index 00000000..8f1effac
--- /dev/null
+++ b/driver-msk/deploy/templates/prometheus.service
@@ -0,0 +1,13 @@
+[Unit]
+Description=Prometheus server
+After=network.target
+
+[Service]
+WorkingDirectory=/opt/prometheus
+ExecStart=/opt/prometheus/prometheus \
+ --web.enable-admin-api \
+ --storage.tsdb.path=/opt/prometheus/data \
+ --config.file=/opt/prometheus/prometheus.yml
+
+[Install]
+WantedBy=multi-user.target
\ No newline at end of file
diff --git a/driver-msk/deploy/templates/prometheus.yml b/driver-msk/deploy/templates/prometheus.yml
new file mode 100644
index 00000000..13462e8c
--- /dev/null
+++ b/driver-msk/deploy/templates/prometheus.yml
@@ -0,0 +1,83 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements. See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership. The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied. See the License for the
+# specific language governing permissions and limitations
+# under the License.
+#
+
+# my global config
+global:
+ scrape_interval: 15s # Set the scrape interval to every 15 seconds. Default is every 1 minute.
+ evaluation_interval: 15s # Evaluate rules every 15 seconds. The default is every 1 minute.
+ # scrape_timeout is set to the global default (10s).
+
+# Alertmanager configuration
+alerting:
+ alertmanagers:
+ - static_configs:
+ - targets:
+ # - alertmanager:9093
+
+# Load rules once and periodically evaluate them according to the global 'evaluation_interval'.
+rule_files:
+ # - "first_rules.yml"
+ # - "second_rules.yml"
+
+# A scrape configuration containing exactly one endpoint to scrape:
+# Here it's Prometheus itself.
+scrape_configs:
+ # The job name is added as a label `job=` to any timeseries scraped from this config.
+ - job_name: 'kafka'
+
+ # metrics_path defaults to '/metrics'
+ # scheme defaults to 'http'.
+
+ static_configs:
+ - targets:
+{% for worker in groups['kafka'] %}
+ - {{ hostvars[worker].private_ip }}:8080
+{% endfor %}
+ labels:
+ group: 'broker'
+ - targets:
+{% for worker in groups['client'] %}
+ - {{ hostvars[worker].private_ip }}:8081
+{% endfor %}
+ labels:
+ group: 'client'
+ - targets:
+{% for worker in groups['kafka'] %}
+ - {{ hostvars[worker].private_ip }}:9100
+{% endfor %}
+ labels:
+ group: 'broker-os'
+ - targets:
+{% for worker in groups['zookeeper'] %}
+ - {{ hostvars[worker].private_ip }}:7070
+{% endfor %}
+ labels:
+ group: 'zookeeper-jmx'
+ - targets:
+{% for worker in groups['zookeeper'] %}
+ - {{ hostvars[worker].private_ip }}:8000
+{% endfor %}
+ labels:
+ group: 'zookeeper'
+ - targets:
+{% for worker in groups['client'] %}
+ - {{ hostvars[worker].private_ip }}:9090
+{% endfor %}
+ labels:
+ group: 'client - jmx'
diff --git a/driver-msk/deploy/templates/server.properties b/driver-msk/deploy/templates/server.properties
new file mode 100644
index 00000000..d5ec0f61
--- /dev/null
+++ b/driver-msk/deploy/templates/server.properties
@@ -0,0 +1,38 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements. See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership. The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied. See the License for the
+# specific language governing permissions and limitations
+# under the License.
+#
+
+
+broker.id={{ brokerId }}
+
+advertised.listeners=PLAINTEXT://{{ privateIp }}:9092
+
+log.dirs=/mnt/data-1,/mnt/data-2
+
+zookeeper.connect={{ zookeeperServers }}
+
+num.replica.fetchers=16
+
+message.max.bytes=10485760
+
+replica.fetch.max.bytes=10485760
+
+num.network.threads=16
+
+num.io.threads=16
+
diff --git a/driver-rabbitmq/rabbitmq.yaml b/driver-msk/deploy/templates/workers.yaml
similarity index 80%
rename from driver-rabbitmq/rabbitmq.yaml
rename to driver-msk/deploy/templates/workers.yaml
index 7ba41ff6..c9ef4e45 100644
--- a/driver-rabbitmq/rabbitmq.yaml
+++ b/driver-msk/deploy/templates/workers.yaml
@@ -18,10 +18,7 @@
#
-name: RabbitMQ
-driverClass: io.openmessaging.benchmark.driver.rabbitmq.RabbitMqBenchmarkDriver
-
-# RabbitMq client specific configurations
-
-brokerAddress: localhost
-messagePersistence: false
\ No newline at end of file
+workers:
+{% for worker in groups['client'] %}
+ - http://{{ hostvars[worker].private_ip }}:8080
+{% endfor %}
diff --git a/driver-msk/deploy/templates/zookeeper.properties b/driver-msk/deploy/templates/zookeeper.properties
new file mode 100644
index 00000000..b33cad81
--- /dev/null
+++ b/driver-msk/deploy/templates/zookeeper.properties
@@ -0,0 +1,49 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements. See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership. The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied. See the License for the
+# specific language governing permissions and limitations
+# under the License.
+#
+
+# The number of milliseconds of each tick
+tickTime=2000
+# The number of ticks that the initial
+# synchronization phase can take
+initLimit=10
+# The number of ticks that can pass between
+# sending a request and getting an acknowledgement
+syncLimit=5
+# the directory where the snapshot is stored.
+dataDir=data/zookeeper
+# the port at which the clients will connect
+clientPort=2181
+# the maximum number of client connections.
+# increase this if you need to handle more clients
+#maxClientCnxns=60
+#
+# Be sure to read the maintenance section of the
+# administrator guide before turning on autopurge.
+#
+# http://zookeeper.apache.org/doc/current/zookeeperAdmin.html#sc_maintenance
+#
+# The number of snapshots to retain in dataDir
+autopurge.snapRetainCount=3
+# Purge task interval in hours
+# Set to "0" to disable auto purge feature
+autopurge.purgeInterval=1
+
+{% for zk in groups['zookeeper'] %}
+server.{{ hostvars[zk].zid }}={{ hostvars[zk].private_ip }}:2888:3888
+{% endfor %}
diff --git a/driver-msk/deploy/templates/zookeeper.service b/driver-msk/deploy/templates/zookeeper.service
new file mode 100644
index 00000000..13dbf8e9
--- /dev/null
+++ b/driver-msk/deploy/templates/zookeeper.service
@@ -0,0 +1,14 @@
+[Unit]
+Description=ZooKeeper
+After=network.target
+
+[Service]
+Environment="EXTRA_ARGS=-javaagent:/opt/kafka/jmx_prometheus_javaagent-0.13.0.jar=7070:/opt/kafka/zookeeper.yml"
+ExecStart=/opt/kafka/bin/zookeeper-server-start.sh config/zookeeper.properties
+WorkingDirectory=/opt/kafka
+RestartSec=1s
+Restart=on-failure
+Type=simple
+
+[Install]
+WantedBy=multi-user.target
diff --git a/driver-msk/deploy/terraform.tfvars b/driver-msk/deploy/terraform.tfvars
new file mode 100644
index 00000000..acae1045
--- /dev/null
+++ b/driver-msk/deploy/terraform.tfvars
@@ -0,0 +1,15 @@
+public_key_path = "~/.ssh/id_rsa.pub"
+region = "us-east-2"
+ami = "ami-03d64741867e7bb94"
+profile = "default"
+kafka_version = "2.6.2"
+
+instance_types = {
+ "client" = "m5.4xlarge"
+ "broker" = "kafka.m5.2xlarge"
+}
+
+num_instances = {
+ "client" = 2
+ "broker" = 9
+}
diff --git a/driver-msk/kafka-0.yaml b/driver-msk/kafka-0.yaml
new file mode 100644
index 00000000..7b9082f2
--- /dev/null
+++ b/driver-msk/kafka-0.yaml
@@ -0,0 +1,40 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements. See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership. The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied. See the License for the
+# specific language governing permissions and limitations
+# under the License.
+#
+
+name: Kafka
+driverClass: io.openmessaging.benchmark.driver.kafka.KafkaBenchmarkDriver
+
+# Kafka client-specific configuration
+replicationFactor: 3
+reset: true
+
+topicConfig: |
+ min.insync.replicas=2
+
+commonConfig: |
+ bootstrap.servers=localhost:9092
+
+producerConfig: |
+ acks=0
+ linger.ms=1
+ batch.size=131072
+
+consumerConfig: |
+ auto.offset.reset=earliest
+ enable.auto.commit=false
diff --git a/driver-msk/kafka-1.yaml b/driver-msk/kafka-1.yaml
new file mode 100644
index 00000000..5abf8242
--- /dev/null
+++ b/driver-msk/kafka-1.yaml
@@ -0,0 +1,40 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements. See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership. The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied. See the License for the
+# specific language governing permissions and limitations
+# under the License.
+#
+
+name: Kafka
+driverClass: io.openmessaging.benchmark.driver.kafka.KafkaBenchmarkDriver
+
+# Kafka client-specific configuration
+replicationFactor: 3
+reset: true
+
+topicConfig: |
+ min.insync.replicas=2
+
+commonConfig: |
+ bootstrap.servers=localhost:9092
+
+producerConfig: |
+ acks=1
+ linger.ms=1
+ batch.size=131072
+
+consumerConfig: |
+ auto.offset.reset=earliest
+ enable.auto.commit=false
diff --git a/driver-msk/kafka-all.yaml b/driver-msk/kafka-all.yaml
new file mode 100644
index 00000000..c78f99c0
--- /dev/null
+++ b/driver-msk/kafka-all.yaml
@@ -0,0 +1,41 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements. See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership. The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied. See the License for the
+# specific language governing permissions and limitations
+# under the License.
+#
+
+name: Kafka
+driverClass: io.openmessaging.benchmark.driver.kafka.KafkaBenchmarkDriver
+
+# Kafka client-specific configuration
+replicationFactor: 3
+reset: true
+
+topicConfig: |
+ min.insync.replicas=2
+
+commonConfig: |
+ bootstrap.servers=localhost:9092
+
+producerConfig: |
+ acks=all
+ linger.ms=100
+ batch.size=1048576
+
+consumerConfig: |
+ auto.offset.reset=earliest
+ enable.auto.commit=false
+ max.partition.fetch.bytes=10485760
diff --git a/driver-msk/kafka-exactly-once.yaml b/driver-msk/kafka-exactly-once.yaml
new file mode 100644
index 00000000..a7407661
--- /dev/null
+++ b/driver-msk/kafka-exactly-once.yaml
@@ -0,0 +1,43 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements. See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership. The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied. See the License for the
+# specific language governing permissions and limitations
+# under the License.
+#
+
+name: Kafka-exactly-once
+driverClass: io.openmessaging.benchmark.driver.kafka.KafkaBenchmarkDriver
+
+# Kafka client-specific configuration
+replicationFactor: 3
+reset: true
+
+topicConfig: |
+ min.insync.replicas=2
+
+commonConfig: |
+ bootstrap.servers=localhost:9092
+
+producerConfig: |
+ enable.idempotence=true
+ max.in.flight.requests.per.connection=1
+ retries=2147483647
+ acks=all
+ linger.ms=1
+ batch.size=131072
+
+consumerConfig: |
+ auto.offset.reset=earliest
+ enable.auto.commit=false
diff --git a/driver-msk/kafka-group-all.yaml b/driver-msk/kafka-group-all.yaml
new file mode 100644
index 00000000..b8944d36
--- /dev/null
+++ b/driver-msk/kafka-group-all.yaml
@@ -0,0 +1,42 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements. See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership. The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied. See the License for the
+# specific language governing permissions and limitations
+# under the License.
+#
+
+name: Kafka
+driverClass: io.openmessaging.benchmark.driver.kafka.KafkaBenchmarkDriver
+
+# Kafka client-specific configuration
+replicationFactor: 3
+reset: true
+
+topicConfig: |
+ min.insync.replicas=2
+
+commonConfig: |
+ bootstrap.servers=localhost:9092
+ request.timeout.ms=120000
+producerConfig: |
+ acks=all
+ linger.ms=10
+ batch.size=1048576
+
+consumerConfig: |
+ group.id=benchGroup
+ auto.offset.reset=earliest
+ enable.auto.commit=false
+ max.partition.fetch.bytes=1048576
diff --git a/driver-msk/kafka-sync-0.yaml b/driver-msk/kafka-sync-0.yaml
new file mode 100644
index 00000000..a4e03180
--- /dev/null
+++ b/driver-msk/kafka-sync-0.yaml
@@ -0,0 +1,44 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements. See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership. The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied. See the License for the
+# specific language governing permissions and limitations
+# under the License.
+#
+
+# Kafka driver profile that is configuring to sync all
+# published messages to disk to achieve durability
+name: Kafka
+driverClass: io.openmessaging.benchmark.driver.kafka.KafkaBenchmarkDriver
+
+# Kafka client-specific configuration
+replicationFactor: 3
+reset: true
+
+topicConfig: |
+ min.insync.replicas=2
+ flush.messages=1
+ flush.ms=0
+
+commonConfig: |
+ bootstrap.servers=localhost:9092
+
+producerConfig: |
+ acks=0
+ linger.ms=1
+ batch.size=131072
+
+consumerConfig: |
+ auto.offset.reset=earliest
+ enable.auto.commit=false
diff --git a/driver-msk/kafka-sync-1.yaml b/driver-msk/kafka-sync-1.yaml
new file mode 100644
index 00000000..3bf6d7d1
--- /dev/null
+++ b/driver-msk/kafka-sync-1.yaml
@@ -0,0 +1,44 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements. See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership. The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied. See the License for the
+# specific language governing permissions and limitations
+# under the License.
+#
+
+# Kafka driver profile that is configuring to sync all
+# published messages to disk to achieve durability
+name: Kafka
+driverClass: io.openmessaging.benchmark.driver.kafka.KafkaBenchmarkDriver
+
+# Kafka client-specific configuration
+replicationFactor: 3
+reset: true
+
+topicConfig: |
+ min.insync.replicas=2
+ flush.messages=1
+ flush.ms=0
+
+commonConfig: |
+ bootstrap.servers=localhost:9092
+
+producerConfig: |
+ acks=1
+ linger.ms=1
+ batch.size=131072
+
+consumerConfig: |
+ auto.offset.reset=earliest
+ enable.auto.commit=false
diff --git a/driver-msk/kafka-sync-all.yaml b/driver-msk/kafka-sync-all.yaml
new file mode 100644
index 00000000..a4fb4955
--- /dev/null
+++ b/driver-msk/kafka-sync-all.yaml
@@ -0,0 +1,44 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements. See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership. The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied. See the License for the
+# specific language governing permissions and limitations
+# under the License.
+#
+
+# Kafka driver profile that is configuring to sync all
+# published messages to disk to achieve durability
+name: Kafka
+driverClass: io.openmessaging.benchmark.driver.kafka.KafkaBenchmarkDriver
+
+# Kafka client-specific configuration
+replicationFactor: 3
+reset: true
+
+topicConfig: |
+ min.insync.replicas=2
+ flush.messages=1
+ flush.ms=0
+
+commonConfig: |
+ bootstrap.servers=localhost:9092
+
+producerConfig: |
+ acks=all
+ linger.ms=1
+ batch.size=131072
+
+consumerConfig: |
+ auto.offset.reset=earliest
+ enable.auto.commit=false
diff --git a/driver-msk/kafka-sync-group-all-1ms.yaml b/driver-msk/kafka-sync-group-all-1ms.yaml
new file mode 100644
index 00000000..fcda5030
--- /dev/null
+++ b/driver-msk/kafka-sync-group-all-1ms.yaml
@@ -0,0 +1,41 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements. See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership. The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied. See the License for the
+# specific language governing permissions and limitations
+# under the License.
+#
+
+name: Kafka
+driverClass: io.openmessaging.benchmark.driver.kafka.KafkaBenchmarkDriver
+# Kafka client-specific configuration
+replicationFactor: 3
+reset: true
+
+topicConfig: |
+ min.insync.replicas=2
+ flush.messages=1
+ flush.ms=0
+commonConfig: |
+ bootstrap.servers=localhost:9092
+ request.timeout.ms=120000
+producerConfig: |
+ acks=all
+ linger.ms=1
+ batch.size=131072
+consumerConfig: |
+ group.id=benchGroup
+ auto.offset.reset=earliest
+ enable.auto.commit=false
+ max.partition.fetch.bytes=131072
diff --git a/driver-msk/kafka-sync-group-all.yaml b/driver-msk/kafka-sync-group-all.yaml
new file mode 100644
index 00000000..dbd91136
--- /dev/null
+++ b/driver-msk/kafka-sync-group-all.yaml
@@ -0,0 +1,42 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements. See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership. The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied. See the License for the
+# specific language governing permissions and limitations
+# under the License.
+#
+
+name: Kafka
+driverClass: io.openmessaging.benchmark.driver.kafka.KafkaBenchmarkDriver
+# Kafka client-specific configuration
+replicationFactor: 3
+reset: true
+
+# Configuration (flush.messages; flush.ms) enabled only for tests that fsync every message
+topicConfig: |
+ min.insync.replicas=2
+ flush.messages=1
+ flush.ms=0
+commonConfig: |
+ bootstrap.servers=localhost:9092
+ request.timeout.ms=120000
+producerConfig: |
+ acks=all
+ linger.ms=10
+ batch.size=1048576
+consumerConfig: |
+ group.id=benchGroup
+ auto.offset.reset=earliest
+ enable.auto.commit=false
+ max.partition.fetch.bytes=1048576
diff --git a/driver-msk/pom.xml b/driver-msk/pom.xml
new file mode 100644
index 00000000..9c044ae8
--- /dev/null
+++ b/driver-msk/pom.xml
@@ -0,0 +1,70 @@
+
+
+ 4.0.0
+
+ io.openmessaging.benchmark
+ messaging-benchmark
+ 0.0.1-SNAPSHOT
+ ..
+
+ driver-kafka
+
+
+
+
+
+
+ Kafka Public Repo
+ https://repository.apache.org/content/groups/public/
+
+
+
+
+
+ ${project.groupId}
+ driver-api
+ ${project.version}
+
+
+
+ org.apache.kafka
+ kafka-clients
+ 2.6.0
+
+
+
+
+
+
+
\ No newline at end of file
diff --git a/driver-msk/src/main/java/io/openmessaging/benchmark/driver/kafka/Config.java b/driver-msk/src/main/java/io/openmessaging/benchmark/driver/kafka/Config.java
new file mode 100644
index 00000000..da59b963
--- /dev/null
+++ b/driver-msk/src/main/java/io/openmessaging/benchmark/driver/kafka/Config.java
@@ -0,0 +1,33 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package io.openmessaging.benchmark.driver.kafka;
+
+public class Config {
+ public short replicationFactor;
+
+ public boolean reset;
+
+ public String topicConfig;
+
+ public String commonConfig;
+
+ public String producerConfig;
+
+ public String consumerConfig;
+}
diff --git a/driver-msk/src/main/java/io/openmessaging/benchmark/driver/kafka/KafkaBenchmarkConsumer.java b/driver-msk/src/main/java/io/openmessaging/benchmark/driver/kafka/KafkaBenchmarkConsumer.java
new file mode 100644
index 00000000..c5a4f13f
--- /dev/null
+++ b/driver-msk/src/main/java/io/openmessaging/benchmark/driver/kafka/KafkaBenchmarkConsumer.java
@@ -0,0 +1,84 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package io.openmessaging.benchmark.driver.kafka;
+
+import java.time.Duration;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.kafka.clients.consumer.ConsumerRecord;
+import org.apache.kafka.clients.consumer.ConsumerRecords;
+import org.apache.kafka.clients.consumer.KafkaConsumer;
+import org.apache.kafka.clients.consumer.OffsetAndMetadata;
+import org.apache.kafka.common.TopicPartition;
+
+import io.openmessaging.benchmark.driver.BenchmarkConsumer;
+import io.openmessaging.benchmark.driver.ConsumerCallback;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class KafkaBenchmarkConsumer implements BenchmarkConsumer {
+
+ private static final Logger log = LoggerFactory.getLogger(KafkaBenchmarkConsumer.class);
+
+ private final KafkaConsumer consumer;
+
+ private final ExecutorService executor;
+ private final Future> consumerTask;
+ private volatile boolean closing = false;
+
+ public KafkaBenchmarkConsumer(KafkaConsumer consumer, ConsumerCallback callback) {
+ this.consumer = consumer;
+ this.executor = Executors.newSingleThreadExecutor();
+
+ this.consumerTask = this.executor.submit(() -> {
+ while (!closing) {
+ try {
+ ConsumerRecords records = consumer.poll(Duration.ofMillis(100));
+ Map offsetMap = new HashMap<>();
+ for (ConsumerRecord record : records) {
+ callback.messageReceived(record.value(), TimeUnit.MILLISECONDS.toNanos(record.timestamp()));
+ offsetMap.put(new TopicPartition(record.topic(), record.partition()),
+ new OffsetAndMetadata(record.offset()));
+ }
+
+ if (!records.isEmpty()) {
+ // Async commit all messages polled so far
+ consumer.commitAsync(offsetMap, null);
+ }
+ } catch (Exception e) {
+ log.error("exception occur while consuming message", e);
+ }
+ }
+ });
+ }
+
+ @Override
+ public void close() throws Exception {
+ closing = true;
+ executor.shutdown();
+ consumerTask.get();
+ consumer.close();
+ }
+
+}
diff --git a/driver-msk/src/main/java/io/openmessaging/benchmark/driver/kafka/KafkaBenchmarkDriver.java b/driver-msk/src/main/java/io/openmessaging/benchmark/driver/kafka/KafkaBenchmarkDriver.java
new file mode 100644
index 00000000..8d708c31
--- /dev/null
+++ b/driver-msk/src/main/java/io/openmessaging/benchmark/driver/kafka/KafkaBenchmarkDriver.java
@@ -0,0 +1,183 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package io.openmessaging.benchmark.driver.kafka;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.StringReader;
+import java.util.*;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+
+import org.apache.bookkeeper.stats.StatsLogger;
+import org.apache.kafka.clients.admin.AdminClient;
+import org.apache.kafka.clients.admin.DeleteTopicsResult;
+import org.apache.kafka.clients.admin.ListTopicsResult;
+import org.apache.kafka.clients.admin.NewTopic;
+import org.apache.kafka.clients.consumer.ConsumerConfig;
+import org.apache.kafka.clients.consumer.KafkaConsumer;
+import org.apache.kafka.clients.producer.KafkaProducer;
+import org.apache.kafka.clients.producer.ProducerConfig;
+import org.apache.kafka.common.serialization.ByteArrayDeserializer;
+import org.apache.kafka.common.serialization.ByteArraySerializer;
+import org.apache.kafka.common.serialization.StringDeserializer;
+import org.apache.kafka.common.serialization.StringSerializer;
+
+import com.fasterxml.jackson.databind.DeserializationFeature;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.dataformat.yaml.YAMLFactory;
+
+import io.openmessaging.benchmark.driver.BenchmarkConsumer;
+import io.openmessaging.benchmark.driver.BenchmarkDriver;
+import io.openmessaging.benchmark.driver.BenchmarkProducer;
+import io.openmessaging.benchmark.driver.ConsumerCallback;
+
+public class KafkaBenchmarkDriver implements BenchmarkDriver {
+
+ private Config config;
+
+ private List producers = Collections.synchronizedList(new ArrayList<>());
+ private List consumers = Collections.synchronizedList(new ArrayList<>());
+
+ private Properties topicProperties;
+ private Properties producerProperties;
+ private Properties consumerProperties;
+
+ private AdminClient admin;
+
+ @Override
+ public void initialize(File configurationFile, StatsLogger statsLogger) throws IOException {
+ config = mapper.readValue(configurationFile, Config.class);
+
+ Properties commonProperties = new Properties();
+ commonProperties.load(new StringReader(config.commonConfig));
+
+ producerProperties = new Properties();
+ commonProperties.forEach((key, value) -> producerProperties.put(key, value));
+ producerProperties.load(new StringReader(config.producerConfig));
+ producerProperties.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, StringSerializer.class.getName());
+ producerProperties.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class.getName());
+
+ consumerProperties = new Properties();
+ commonProperties.forEach((key, value) -> consumerProperties.put(key, value));
+ consumerProperties.load(new StringReader(config.consumerConfig));
+ consumerProperties.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class.getName());
+ consumerProperties.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class.getName());
+
+ topicProperties = new Properties();
+ topicProperties.load(new StringReader(config.topicConfig));
+
+ admin = AdminClient.create(commonProperties);
+
+ if (config.reset) {
+ // List existing topics
+ ListTopicsResult result = admin.listTopics();
+ try {
+ Set topics = result.names().get();
+ // Delete all existing topics
+ DeleteTopicsResult deletes = admin.deleteTopics(topics);
+ deletes.all().get();
+ } catch (InterruptedException | ExecutionException e) {
+ e.printStackTrace();
+ throw new IOException(e);
+ }
+ }
+ }
+
+ @Override
+ public String getTopicNamePrefix() {
+ return "test-topic";
+ }
+
+ @SuppressWarnings({ "rawtypes", "unchecked" })
+ @Override
+ public CompletableFuture createTopic(String topic, int partitions) {
+ return CompletableFuture.runAsync(() -> {
+ try {
+ NewTopic newTopic = new NewTopic(topic, partitions, config.replicationFactor);
+ newTopic.configs(new HashMap<>((Map) topicProperties));
+ admin.createTopics(Arrays.asList(newTopic)).all().get();
+ } catch (InterruptedException | ExecutionException e) {
+ throw new RuntimeException(e);
+ }
+ });
+ }
+
+ @Override
+ public CompletableFuture notifyTopicCreation(String topic, int partitions) {
+ // No-op
+ return CompletableFuture.completedFuture(null);
+ }
+
+ @Override
+ public CompletableFuture createProducer(String topic) {
+ KafkaProducer kafkaProducer = new KafkaProducer<>(producerProperties);
+ BenchmarkProducer benchmarkProducer = new KafkaBenchmarkProducer(kafkaProducer, topic);
+ try {
+ // Add to producer list to close later
+ producers.add(benchmarkProducer);
+ return CompletableFuture.completedFuture(benchmarkProducer);
+ } catch (Throwable t) {
+ kafkaProducer.close();
+ CompletableFuture future = new CompletableFuture<>();
+ future.completeExceptionally(t);
+ return future;
+ }
+ }
+
+ @Override
+ public CompletableFuture createConsumer(String topic, String subscriptionName,
+ Optional partition, ConsumerCallback consumerCallback) {
+ Properties properties = new Properties();
+ consumerProperties.forEach((key, value) -> properties.put(key, value));
+ properties.put(ConsumerConfig.GROUP_ID_CONFIG, subscriptionName);
+ KafkaConsumer kafkaConsumer = new KafkaConsumer<>(properties);
+ try {
+ // Subscribe
+ kafkaConsumer.subscribe(Arrays.asList(topic));
+
+ // Start polling
+ BenchmarkConsumer benchmarkConsumer = new KafkaBenchmarkConsumer(kafkaConsumer, consumerCallback);
+
+ // Add to consumer list to close later
+ consumers.add(benchmarkConsumer);
+ return CompletableFuture.completedFuture(benchmarkConsumer);
+ } catch (Throwable t) {
+ kafkaConsumer.close();
+ CompletableFuture future = new CompletableFuture<>();
+ future.completeExceptionally(t);
+ return future;
+ }
+ }
+
+ @Override
+ public void close() throws Exception {
+ for (BenchmarkProducer producer : producers) {
+ producer.close();
+ }
+
+ for (BenchmarkConsumer consumer : consumers) {
+ consumer.close();
+ }
+ admin.close();
+ }
+
+ private static final ObjectMapper mapper = new ObjectMapper(new YAMLFactory())
+ .configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false);
+}
diff --git a/driver-msk/src/main/java/io/openmessaging/benchmark/driver/kafka/KafkaBenchmarkProducer.java b/driver-msk/src/main/java/io/openmessaging/benchmark/driver/kafka/KafkaBenchmarkProducer.java
new file mode 100644
index 00000000..dcd487b8
--- /dev/null
+++ b/driver-msk/src/main/java/io/openmessaging/benchmark/driver/kafka/KafkaBenchmarkProducer.java
@@ -0,0 +1,61 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package io.openmessaging.benchmark.driver.kafka;
+
+import java.util.Optional;
+import java.util.concurrent.CompletableFuture;
+
+import org.apache.kafka.clients.producer.KafkaProducer;
+import org.apache.kafka.clients.producer.ProducerRecord;
+
+import io.openmessaging.benchmark.driver.BenchmarkProducer;
+
+public class KafkaBenchmarkProducer implements BenchmarkProducer {
+
+ private final KafkaProducer