Skip to content

Commit

Permalink
feat: add aggregated rocksdb metrics (#6354)
Browse files Browse the repository at this point in the history
* feat: add aggregated rocksdb metrics

This patch adds metrics that return aggregates of the rocksdb metrics
added by KIP-607. Specifically, this particular PR adds the following metics:

num-running-compactions-total: the total number of running compactions
estimate-num-keys-total: an estimate of the total number of rocksdb keys
block-cache-usage-total: total memory usage of all block cache
block-cache-pinned-usage-total: total memory used by pinned blocks
estimate-table-readers-mem-total: estimate of the total table readers mem

ksqlDB registers for notification about new rocksdb metrics by creating a
MetricsReporter implementation called RocksDBMetricCollector. The metrics
system calls into MetricsReporter.metricChange when a new metric is added.
RocksDBMetricCollector looks out for rocksdb property metrics it cares about
and tracks them under the relevant aggregates. Each aggregate is registered
with the ksql metrics context on the first instantiation of
RocksDBMetricCollector.

Metrics are computed lazily when read, and are rate-limited to a configurable
interval. The interval is set using the property
ksql.rocksdb.metrics.update.interval.seconds
  • Loading branch information
rodesai authored Oct 16, 2020
1 parent ed3ca5e commit ecc6625
Show file tree
Hide file tree
Showing 3 changed files with 553 additions and 0 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -35,6 +35,7 @@
import io.confluent.ksql.execution.streams.KSPlanBuilder;
import io.confluent.ksql.execution.streams.materialization.KsqlMaterializationFactory;
import io.confluent.ksql.execution.streams.materialization.ks.KsMaterializationFactory;
import io.confluent.ksql.execution.streams.metrics.RocksDBMetricsCollector;
import io.confluent.ksql.function.FunctionRegistry;
import io.confluent.ksql.logging.processing.ProcessingLogContext;
import io.confluent.ksql.logging.processing.ProcessingLogger;
Expand Down Expand Up @@ -306,6 +307,11 @@ private Map<String, Object> buildStreamsProperties(
StreamsConfig.producerPrefix(ProducerConfig.INTERCEPTOR_CLASSES_CONFIG),
ProducerCollector.class.getCanonicalName()
);
updateListProperty(
newStreamsProperties,
StreamsConfig.METRIC_REPORTER_CLASSES_CONFIG,
RocksDBMetricsCollector.class.getName()
);
return newStreamsProperties;
}

Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,278 @@
/*
* Copyright 2020 Confluent Inc.
*
* Licensed under the Confluent Community License (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.confluent.io/confluent-community-license
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
* WARRANTIES OF ANY KIND, either express or implied. See the License for the
* specific language governing permissions and limitations under the License.
*/

package io.confluent.ksql.execution.streams.metrics;

import com.google.common.annotations.VisibleForTesting;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
import io.confluent.ksql.metrics.MetricCollectors;
import java.math.BigInteger;
import java.time.Instant;
import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
import java.util.LinkedList;
import java.util.List;
import java.util.Map;
import java.util.Objects;
import java.util.Set;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.atomic.AtomicReference;
import java.util.function.BinaryOperator;
import java.util.function.Supplier;
import java.util.stream.Collectors;
import org.apache.kafka.common.MetricName;
import org.apache.kafka.common.config.AbstractConfig;
import org.apache.kafka.common.config.ConfigDef;
import org.apache.kafka.common.config.ConfigDef.Importance;
import org.apache.kafka.common.config.ConfigDef.Type;
import org.apache.kafka.common.metrics.Gauge;
import org.apache.kafka.common.metrics.KafkaMetric;
import org.apache.kafka.common.metrics.Metrics;
import org.apache.kafka.common.metrics.MetricsReporter;
import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

public class RocksDBMetricsCollector implements MetricsReporter {
private static final Logger LOGGER = LoggerFactory.getLogger(RocksDBMetricsCollector.class);

static final String KSQL_ROCKSDB_METRICS_GROUP = "ksql-rocksdb-aggregates";
static final String NUMBER_OF_RUNNING_COMPACTIONS = "num-running-compactions";
static final String BLOCK_CACHE_USAGE = "block-cache-usage";
static final String BLOCK_CACHE_PINNED_USAGE = "block-cache-pinned-usage";
static final String ESTIMATE_NUM_KEYS = "estimate-num-keys";
static final String ESTIMATE_TABLE_READERS_MEM = "estimate-table-readers-mem";
static final String UPDATE_INTERVAL_CONFIG = "ksql.rocksdb.metrics.update.interval.seconds";
private static final int UPDATE_INTERVAL_DEFAULT = 15;

private static final ConfigDef CONFIG_DEF = new ConfigDef()
.define(
UPDATE_INTERVAL_CONFIG,
Type.INT,
UPDATE_INTERVAL_DEFAULT,
Importance.LOW,
"minimum interval between computations of a metric value"
);

private static final Object lock = new Object();

private static Map<String, Collection<AggregatedMetric<?>>> registeredMetrics = null;
private final Metrics metrics;

public RocksDBMetricsCollector() {
this(MetricCollectors.getMetrics());
}

@VisibleForTesting
RocksDBMetricsCollector(final Metrics metrics) {
this.metrics = Objects.requireNonNull(metrics);
}

@Override
public void configure(final Map<String, ?> map) {
final AbstractConfig config = new AbstractConfig(CONFIG_DEF, map);
configureShared(config, metrics);
}

@Override
public Set<String> reconfigurableConfigs() {
return Collections.emptySet();
}

@Override
public void init(final List<KafkaMetric> initial) {
initial.forEach(this::metricChange);
}

@Override
public void metricChange(final KafkaMetric metric) {
if (!metric.metricName().group().equals(StreamsMetricsImpl.STATE_STORE_LEVEL_GROUP)) {
return;
}
metricRemoval(metric);
final Collection<AggregatedMetric<?>> registered
= registeredMetrics.get(metric.metricName().name());
if (registered == null) {
return;
}
registered.forEach(r -> r.add(metric));
}

@Override
public void metricRemoval(final KafkaMetric metric) {
final MetricName metricName = metric.metricName();
if (!metricName.group().equals(StreamsMetricsImpl.STATE_STORE_LEVEL_GROUP)) {
return;
}
final Collection<AggregatedMetric<?>> registered
= registeredMetrics.get(metricName.name());
if (registered == null) {
return;
}
registered.forEach(r -> r.remove(metricName));
}

@VisibleForTesting
static void reset() {
registeredMetrics = null;
}

@Override
public void close() {
}

public static void update() {
registeredMetrics.values().stream()
.flatMap(Collection::stream)
.forEach(AggregatedMetric::update);
}

private static void registerBigIntTotal(
final int interval,
final Map<String, Collection<AggregatedMetric<?>>> registeredMetrics,
final String name,
final Metrics metrics
) {
registeredMetrics.putIfAbsent(name, new LinkedList<>());
final AggregatedMetric<BigInteger> registered = new AggregatedMetric<>(
BigInteger.class,
BigInteger::add,
BigInteger.ZERO,
new Interval(interval)
);
registeredMetrics.get(name).add(registered);
final MetricName metricName = metrics.metricName(name + "-total", KSQL_ROCKSDB_METRICS_GROUP);
metrics.addMetric(metricName, (Gauge<BigInteger>) (c, t) -> registered.getValue());
}

private static void registerBigIntMax(
final int interval,
final Map<String, Collection<AggregatedMetric<?>>> registeredMetrics,
final String name,
final Metrics metrics
) {
registeredMetrics.putIfAbsent(name, new LinkedList<>());
final AggregatedMetric<BigInteger> registered = new AggregatedMetric<>(
BigInteger.class,
BigInteger::max,
BigInteger.ZERO,
new Interval(interval)
);
registeredMetrics.get(name).add(registered);
final MetricName metricName = metrics.metricName(name + "-max", KSQL_ROCKSDB_METRICS_GROUP);
metrics.addMetric(metricName, (Gauge<BigInteger>) (c, t) -> registered.getValue());
}

static final class AggregatedMetric<T> {
private final Class<T> clazz;
private final BinaryOperator<T> aggregator;
private final T identity;
private final Interval interval;
private final Map<MetricName, KafkaMetric> metrics = new ConcurrentHashMap<>();
private volatile T value;

private AggregatedMetric(
final Class<T> clazz,
final BinaryOperator<T> aggregator,
final T identity,
final Interval interval
) {
this.clazz = Objects.requireNonNull(clazz, "clazz");
this.aggregator = Objects.requireNonNull(aggregator, "aggregator");
this.identity = Objects.requireNonNull(identity, "identity");
this.value = identity;
this.interval = interval;
}

private void add(final KafkaMetric metric) {
metrics.put(metric.metricName(), metric);
}

private void remove(final MetricName name) {
metrics.remove(name);
}

private T getValue() {
if (interval.check()) {
value = update();
}
return value;
}

private T update() {
T current = identity;
for (final KafkaMetric metric : metrics.values()) {
final Object value = metric.metricValue();
if (!clazz.isInstance(value)) {
LOGGER.debug(
"Skipping metric update due to unexpected value type returned by {}",
metric.metricName().toString()
);
return identity;
}
current = aggregator.apply(current, clazz.cast(value));
}
return current;
}
}

static final class Interval {
private final int intervalSeconds;
private final AtomicReference<Instant> last;
private final Supplier<Instant> clock;

private Interval(final int intervalSeconds) {
this(intervalSeconds, Instant::now);
}

Interval(final int intervalSeconds, final Supplier<Instant> clock) {
this.intervalSeconds = intervalSeconds;
this.clock = Objects.requireNonNull(clock, "clock");
this.last = new AtomicReference<>(Instant.EPOCH);
}

boolean check() {
final Instant now = clock.get();
return last.accumulateAndGet(
now,
(l, n) -> n.isAfter(l.plusSeconds(intervalSeconds)) ? n : l
) == now;
}
}

private static void configureShared(final AbstractConfig config, final Metrics metrics) {
synchronized (lock) {
if (registeredMetrics != null) {
return;
}
final int interval = config.getInt(UPDATE_INTERVAL_CONFIG);
final Map<String, Collection<AggregatedMetric<?>>> builder = new HashMap<>();
registerBigIntTotal(interval, builder, NUMBER_OF_RUNNING_COMPACTIONS, metrics);
registerBigIntTotal(interval, builder, BLOCK_CACHE_USAGE, metrics);
registerBigIntMax(interval, builder, BLOCK_CACHE_USAGE, metrics);
registerBigIntTotal(interval, builder, BLOCK_CACHE_PINNED_USAGE, metrics);
registerBigIntMax(interval, builder, BLOCK_CACHE_PINNED_USAGE, metrics);
registerBigIntTotal(interval, builder, ESTIMATE_NUM_KEYS, metrics);
registerBigIntTotal(interval, builder, ESTIMATE_TABLE_READERS_MEM, metrics);
registeredMetrics = ImmutableMap.copyOf(
builder.entrySet()
.stream()
.collect(Collectors.toMap(Map.Entry::getKey, e -> ImmutableList.copyOf(e.getValue())))
);
}
}
}
Loading

0 comments on commit ecc6625

Please sign in to comment.