Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

feat: update storage utilization metrics to start when app is initialized #8095

Merged
merged 2 commits into from
Sep 8, 2021
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -41,9 +41,12 @@
import org.apache.kafka.common.metrics.Metrics;
import org.apache.kafka.common.metrics.MetricsContext;
import org.apache.kafka.common.metrics.MetricsReporter;
import org.apache.kafka.streams.StreamsConfig;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

public class StorageUtilizationMetricsReporter implements MetricsReporter {
private static final Logger LOGGER
= LoggerFactory.getLogger(StorageUtilizationMetricsReporter.class);
private static final String METRIC_GROUP = "ksqldb_utilization";

private final Map<String, Map<String, TaskStorageMetric>> metricsSeen;
Expand All @@ -67,23 +70,13 @@ public void init(final List<KafkaMetric> list) {

@Override
public void configure(final Map<String, ?> map) {
final String dir;
if (map.containsKey(StreamsConfig.STATE_DIR_CONFIG)) {
dir = map.get(StreamsConfig.STATE_DIR_CONFIG).toString();
} else {
dir = StreamsConfig
.configDef()
.defaultValues()
.get(StreamsConfig.STATE_DIR_CONFIG)
.toString();
}
configureShared(new File(dir), this.metricRegistry);
}

private static void configureShared(final File baseDir, final Metrics metricRegistry) {
public static void configureShared(final File baseDir, final Metrics metricRegistry) {
if (registeredNodeMetrics.getAndSet(true)) {
return;
}
LOGGER.info("Adding node level storage usage gauges");
final MetricName nodeAvailable =
metricRegistry.metricName("node_storage_free_bytes", METRIC_GROUP);
final MetricName nodeTotal =
Expand Down Expand Up @@ -169,6 +162,7 @@ private synchronized void handleNewSstFilesSizeMetric(
final String taskId,
final String queryId
) {
LOGGER.debug("Updating disk usage metrics");
// if we haven't seen a task for this query yet
if (!metricsSeen.containsKey(queryId)) {
metricRegistry.addMetric(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -60,7 +60,7 @@ public void setUp() throws IOException {
final File f = new File("/tmp/storage-test/");
f.getParentFile().mkdirs();
f.createNewFile();
listener.configure(ImmutableMap.of("state.dir", "/tmp/storage-test/"));
listener.configureShared(f, metrics);
}

@After
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -43,6 +43,7 @@
import io.confluent.ksql.function.MutableFunctionRegistry;
import io.confluent.ksql.function.UserFunctionLoader;
import io.confluent.ksql.internal.PullQueryExecutorMetrics;
import io.confluent.ksql.internal.StorageUtilizationMetricsReporter;
import io.confluent.ksql.logging.processing.ProcessingLogConfig;
import io.confluent.ksql.logging.processing.ProcessingLogContext;
import io.confluent.ksql.logging.processing.ProcessingLogServerUtils;
Expand Down Expand Up @@ -699,6 +700,16 @@ static KsqlRestApplication buildApplication(

final SpecificQueryIdGenerator specificQueryIdGenerator =
new SpecificQueryIdGenerator();

final String stateDir = ksqlConfig.getKsqlStreamConfigProps().getOrDefault(
StreamsConfig.STATE_DIR_CONFIG,
StreamsConfig.configDef().defaultValues().get(StreamsConfig.STATE_DIR_CONFIG))
.toString();

StorageUtilizationMetricsReporter.configureShared(
new File(stateDir),
MetricCollectors.getMetrics()
);

final KsqlEngine ksqlEngine = new KsqlEngine(
serviceContext,
Expand Down