-
Notifications
You must be signed in to change notification settings - Fork 1k
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: add max task usage and num stateful tasks metrics #8549
Changes from 6 commits
bb3fdd5
cfff094
ceae440
f6d3059
9ed5a55
8458505
8f13c11
f6a2a0f
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -27,8 +27,10 @@ | |
import java.util.HashMap; | ||
import java.util.List; | ||
import java.util.Map; | ||
import java.util.Optional; | ||
import java.util.Set; | ||
import java.util.concurrent.ConcurrentHashMap; | ||
import java.util.concurrent.atomic.AtomicInteger; | ||
import java.util.function.Supplier; | ||
import java.util.regex.Matcher; | ||
import java.util.regex.Pattern; | ||
|
@@ -49,8 +51,9 @@ public class StorageUtilizationMetricsReporter implements MetricsReporter { | |
private static final String METRIC_GROUP = "ksqldb_utilization"; | ||
|
||
private Map<String, Map<String, TaskStorageMetric>> metricsSeen; | ||
private Metrics metricRegistry; | ||
private static Metrics metricRegistry; | ||
private static Map<String, String> customTags = new HashMap<>(); | ||
private static AtomicInteger numberStatefulTasks = new AtomicInteger(0); | ||
|
||
public StorageUtilizationMetricsReporter() { | ||
} | ||
|
@@ -82,7 +85,10 @@ public static void configureShared( | |
metricRegistry.metricName("node_storage_used_bytes", METRIC_GROUP, customTags); | ||
final MetricName nodePct = | ||
metricRegistry.metricName("storage_utilization", METRIC_GROUP, customTags); | ||
|
||
final MetricName maxTaskPerNode = | ||
metricRegistry.metricName("max_task_storage_used_bytes", METRIC_GROUP, customTags); | ||
final MetricName numStatefulTasks = | ||
metricRegistry.metricName("num_stateful_tasks", METRIC_GROUP, customTags); | ||
metricRegistry.addMetric( | ||
nodeAvailable, | ||
(Gauge<Long>) (config, now) -> baseDir.getFreeSpace() | ||
|
@@ -101,6 +107,14 @@ public static void configureShared( | |
(((double) baseDir.getTotalSpace() - (double) baseDir.getFreeSpace()) | ||
/ (double) baseDir.getTotalSpace()) | ||
); | ||
metricRegistry.addMetric( | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. actually getting NPEs from these two, it seems like There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. updated the gauges to handle the if the items are null, moving them to
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Not sure I understand the above issue: what object the NPE is referring to? The metric names like There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. The NPE was when the gauge was being sampled, so when |
||
maxTaskPerNode, | ||
(Gauge<BigInteger>) (config, now) -> (getMaxTaskUsage()) | ||
); | ||
metricRegistry.addMetric( | ||
numStatefulTasks, | ||
(Gauge<Integer>) (config, now) -> (numberStatefulTasks.get()) | ||
); | ||
} | ||
|
||
@Override | ||
|
@@ -176,6 +190,7 @@ private synchronized void handleNewSstFilesSizeMetric( | |
final TaskStorageMetric newMetric; | ||
// We haven't seen a metric for this query's task before | ||
if (!metricsSeen.get(queryId).containsKey(taskId)) { | ||
numberStatefulTasks.getAndIncrement(); | ||
// create a new task level metric to track state store storage usage | ||
newMetric = new TaskStorageMetric( | ||
metricRegistry.metricName( | ||
|
@@ -205,6 +220,7 @@ private synchronized void handleRemovedSstFileSizeMetric( | |
) { | ||
// remove storage metric for this task | ||
taskMetric.remove(metric); | ||
numberStatefulTasks.getAndDecrement(); | ||
if (taskMetric.metrics.size() == 0) { | ||
// no more storage metrics for this task, can remove task gauge | ||
metricRegistry.removeMetric(taskMetric.metricName); | ||
|
@@ -227,6 +243,24 @@ private BigInteger computeQueryMetric(final String queryId) { | |
} | ||
return queryMetricSum; | ||
} | ||
|
||
public static synchronized BigInteger getMaxTaskUsage() { | ||
if (metricRegistry == null) { | ||
return BigInteger.ZERO; | ||
} | ||
final Collection<KafkaMetric> taskMetrics = metricRegistry | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. This is kind of expensive. Once this is shipped can you just see what happens to the cpu of an idle instance? I don't think it will make much difference because it only runs every 30 seconds. But better to check anyway. Alternatively we could keep a static set of the tasks and a max that's updated on every new or removed task. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. idle instance as in an instance with no queries or instance with queries but no data? I can check after shipping, it's definitely ... unideal, but storing the tasks / max had a longer LOE so went with this first. Would the static max update as the underlying task metrics do? I wanted to make sure we were getting the max of all the most recent gauge values There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
Yep. An idle cluster will still have loads of metrics, so seeing if the cpu usage for an idle cluster is higher than normal is a good test. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
Good point - the space used is not known at the time the metric is added. If the current impl turns out too expensive we can just maintain a static mapping of task to sst file used metrics and then take the max every time the metric is sampled (rather than filtering over every metric like we're doing here). |
||
.metrics() | ||
.entrySet() | ||
.stream() | ||
.filter(e -> e.getKey().name().contains("task_storage_used_bytes")) | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. put the metric name in a constant |
||
.collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)) | ||
.values(); | ||
final Optional<BigInteger> maxOfTaskMetrics = taskMetrics | ||
.stream() | ||
.map(e -> (BigInteger) e.metricValue()) | ||
.reduce(BigInteger::max); | ||
return maxOfTaskMetrics.orElse(BigInteger.ZERO); | ||
} | ||
|
||
private synchronized Collection<Supplier<BigInteger>> getGaugesForQuery(final String queryId) { | ||
return metricsSeen.get(queryId).values().stream() | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -3,7 +3,10 @@ | |
import static org.hamcrest.MatcherAssert.assertThat; | ||
import static org.hamcrest.Matchers.equalTo; | ||
import static org.hamcrest.Matchers.greaterThan; | ||
import static org.hamcrest.Matchers.greaterThanOrEqualTo; | ||
import static org.junit.Assert.assertEquals; | ||
import static org.junit.Assert.assertThrows; | ||
import static org.junit.Assert.assertTrue; | ||
import static org.mockito.ArgumentMatchers.any; | ||
import static org.mockito.ArgumentMatchers.argThat; | ||
import static org.mockito.Mockito.mock; | ||
|
@@ -16,12 +19,16 @@ | |
import java.io.File; | ||
import java.io.IOException; | ||
import java.math.BigInteger; | ||
import java.util.Collections; | ||
import java.util.Map; | ||
import java.util.concurrent.atomic.AtomicInteger; | ||
|
||
import org.apache.kafka.common.MetricName; | ||
import org.apache.kafka.common.metrics.Gauge; | ||
import org.apache.kafka.common.metrics.KafkaMetric; | ||
import org.apache.kafka.common.metrics.MetricValueProvider; | ||
import org.apache.kafka.common.metrics.Metrics; | ||
import org.codehaus.janino.Java; | ||
import org.junit.Before; | ||
import org.junit.Test; | ||
import org.junit.runner.RunWith; | ||
|
@@ -35,6 +42,7 @@ public class StorageUtilizationMetricsReporterTest { | |
|
||
private static final String KAFKA_METRIC_NAME = "total-sst-files-size"; | ||
private static final String KAFKA_METRIC_GROUP = "streams-metric"; | ||
private static final String KSQL_METRIC_GROUP = "ksqldb_utilization"; | ||
private static final String THREAD_ID = "_confluent_blahblah_query_CTAS_TEST_1-blahblah"; | ||
private static final String TRANSIENT_THREAD_ID = "_confluent_blahblah_transient_blahblah_4-blahblah"; | ||
private static final String TASK_STORAGE_METRIC = "task_storage_used_bytes"; | ||
|
@@ -85,12 +93,18 @@ public void shouldAddNodeMetricsOnConfigure() throws IOException { | |
final Object storageUsedValue = storageUsedGauge.value(null, 0); | ||
final Gauge<?> pctUsedGauge = verifyAndGetRegisteredMetric("storage_utilization", BASE_TAGS); | ||
final Object pctUsedValue = pctUsedGauge.value(null, 0); | ||
|
||
final Gauge<?> maxTaskUsageGauge = verifyAndGetRegisteredMetric("max_task_storage_used_bytes", BASE_TAGS); | ||
final Object maxTaskUsageValue = maxTaskUsageGauge.value(null, 0); | ||
final Gauge<?> numStatefulTasksGauge = verifyAndGetRegisteredMetric("num_stateful_tasks", BASE_TAGS); | ||
final Object numStatefulTasksValue = numStatefulTasksGauge.value(null, 0); | ||
|
||
// Then: | ||
assertThat((Long) storageFreeValue, greaterThan(0L)); | ||
assertThat((Long) storageTotalValue, greaterThan(0L)); | ||
assertThat((Long) storageUsedValue, greaterThan(0L)); | ||
assertThat((Double) pctUsedValue, greaterThan(0.0)); | ||
assertThat((BigInteger) maxTaskUsageValue, greaterThanOrEqualTo(BigInteger.ZERO)); | ||
assertEquals((int) numStatefulTasksValue, 0); | ||
} | ||
|
||
@Test | ||
|
@@ -253,6 +267,70 @@ public void shouldIgnoreNonSSTMetrics() { | |
// Then: | ||
assertThrows(AssertionError.class, () -> verifyAndGetRegisteredMetric(TASK_STORAGE_METRIC, TASK_ONE_TAGS)); | ||
} | ||
|
||
@Test | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. can you add a case for max task when there are 0 tasks? There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. yup |
||
public void shouldRecordMaxTaskUsage() { | ||
// Given: | ||
KafkaMetric m1 = mockMetric( | ||
KSQL_METRIC_GROUP, | ||
TASK_STORAGE_METRIC, | ||
BigInteger.valueOf(2), | ||
ImmutableMap.of("task-id", "t1")); | ||
KafkaMetric m2 = mockMetric( | ||
KSQL_METRIC_GROUP, | ||
TASK_STORAGE_METRIC, | ||
BigInteger.valueOf(5), | ||
ImmutableMap.of("task-id", "t2")); | ||
MetricName n1 = m1.metricName(); | ||
MetricName n2 = m2.metricName(); | ||
when(metrics.metrics()).thenReturn(ImmutableMap.of(n1, m1, n2, m2)); | ||
|
||
// When: | ||
listener.metricChange(m1); | ||
listener.metricChange(m2); | ||
|
||
// Then: | ||
BigInteger maxVal = StorageUtilizationMetricsReporter.getMaxTaskUsage(); | ||
assertTrue(maxVal.equals(BigInteger.valueOf(5))); | ||
} | ||
|
||
@Test | ||
public void shouldRecordMaxTaskUsageWithNoTasks() { | ||
// Given: | ||
when(metrics.metrics()).thenReturn(Collections.EMPTY_MAP); | ||
|
||
// When: | ||
|
||
// Then: | ||
BigInteger maxVal = StorageUtilizationMetricsReporter.getMaxTaskUsage(); | ||
assertTrue(maxVal.equals(BigInteger.valueOf(0))); | ||
} | ||
|
||
@Test | ||
public void shouldRecordNumStatefulTasks() { | ||
// Given: | ||
final File f = new File("/tmp/storage-test/"); | ||
listener.configureShared(f, metrics, BASE_TAGS); | ||
final Gauge<?> numStatefulTasksGauge = verifyAndGetRegisteredMetric("num_stateful_tasks", BASE_TAGS); | ||
|
||
// When: | ||
listener.metricChange(mockMetric( | ||
KAFKA_METRIC_GROUP, | ||
KAFKA_METRIC_NAME, | ||
BigInteger.valueOf(2), | ||
ImmutableMap.of("store-id", "s1", "task-id", "t2", "thread-id", TRANSIENT_THREAD_ID)) | ||
); | ||
listener.metricChange(mockMetric( | ||
KAFKA_METRIC_GROUP, | ||
KAFKA_METRIC_NAME, | ||
BigInteger.valueOf(5), | ||
ImmutableMap.of("store-id", "s2", "task-id", "t1", "thread-id", TRANSIENT_THREAD_ID)) | ||
); | ||
|
||
// Then: | ||
final Object numStatefulTasksValue = numStatefulTasksGauge.value(null, 0); | ||
assertEquals((int) numStatefulTasksValue, 2); | ||
} | ||
|
||
private KafkaMetric mockMetric( | ||
final String group, final String name, Object value, final Map<String, String> tags) { | ||
|
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
This should not be static - just leave it local to each member. We intentionally made it not-static to avoid re-using the instance across test runs.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Is there a way to still reference it from a static context?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I didn't change this because I'm not sure how to handle it,
configureShared
is called from a static context so it has to be static, meaninggetMaxTaskUsage
has to be static which callsmetricRegistry.metrics() which intellij is also telling me needs to be static to be called from this context. I can create a helper function to get
metricRegistry.metrics()` but it seems like it would have the same issueThere was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
configureShared
is passed aMetricsRegistry
instance that you can register the metrics in.There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Ah makes sense, just updated