-
Notifications
You must be signed in to change notification settings - Fork 14k
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
KAFKA-17561: KIP-1091 add operator metrics #17820
base: trunk
Are you sure you want to change the base?
Changes from all commits
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 |
---|---|---|
|
@@ -180,15 +180,16 @@ public void shouldPushMetricsToBroker(final String recordingLevel) throws Except | |
final String name = mn.name().replace('-', '.'); | ||
final String group = mn.group().replace("-metrics", "").replace('-', '.'); | ||
return "org.apache.kafka." + group + "." + name; | ||
}).sorted().collect(Collectors.toList()); | ||
}).filter(name -> !name.equals("org.apache.kafka.stream.thread.state"))// telemetry reporter filters out string metrics | ||
.sorted().collect(Collectors.toList()); | ||
final List<String> actualMetrics = new ArrayList<>(TelemetryPlugin.SUBSCRIBED_METRICS.get(mainConsumerInstanceId)); | ||
assertEquals(expectedMetrics, actualMetrics); | ||
|
||
TestUtils.waitForCondition(() -> !TelemetryPlugin.SUBSCRIBED_METRICS.get(adminInstanceId).isEmpty(), | ||
30_000, | ||
"Never received subscribed metrics"); | ||
final List<String> actualInstanceMetrics = TelemetryPlugin.SUBSCRIBED_METRICS.get(adminInstanceId); | ||
final List<String> expectedInstanceMetrics = Arrays.asList("org.apache.kafka.stream.alive.stream.threads", "org.apache.kafka.stream.failed.stream.threads"); | ||
final List<String> expectedInstanceMetrics = Arrays.asList("org.apache.kafka.stream.alive.stream.threads", "org.apache.kafka.stream.client.state", "org.apache.kafka.stream.failed.stream.threads", "org.apache.kafka.stream.recording.level"); | ||
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. Nit: line too long. How about
|
||
assertEquals(expectedInstanceMetrics, actualInstanceMetrics); | ||
|
||
TestUtils.waitForCondition(() -> TelemetryPlugin.processId != null, | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -110,6 +110,7 @@ | |
import java.util.function.Consumer; | ||
import java.util.function.Supplier; | ||
|
||
import static org.apache.kafka.streams.StreamsConfig.METRICS_RECORDING_LEVEL_CONFIG; | ||
import static org.apache.kafka.streams.errors.StreamsUncaughtExceptionHandler.StreamThreadExceptionResponse.SHUTDOWN_CLIENT; | ||
import static org.apache.kafka.streams.internals.ApiUtils.prepareMillisCheckFailMsgPrefix; | ||
import static org.apache.kafka.streams.internals.ApiUtils.validateMillisecondDuration; | ||
|
@@ -987,6 +988,8 @@ private KafkaStreams(final TopologyMetadata topologyMetadata, | |
ClientMetrics.addApplicationIdMetric(streamsMetrics, applicationConfigs.getString(StreamsConfig.APPLICATION_ID_CONFIG)); | ||
ClientMetrics.addTopologyDescriptionMetric(streamsMetrics, (metricsConfig, now) -> this.topologyMetadata.topologyDescriptionString()); | ||
ClientMetrics.addStateMetric(streamsMetrics, (metricsConfig, now) -> state); | ||
ClientMetrics.addClientStateTelemetryMetric(streamsMetrics, (metricsConfig, now) -> state.ordinal()); | ||
ClientMetrics.addClientRecordingLevelMetric(streamsMetrics, calculateMetricsRecordingLevel()); | ||
threads = Collections.synchronizedList(new LinkedList<>()); | ||
ClientMetrics.addNumAliveStreamThreadMetric(streamsMetrics, (metricsConfig, now) -> numLiveStreamThreads()); | ||
|
||
|
@@ -1067,7 +1070,7 @@ private StreamThread createAndAddStreamThread(final long cacheSizePerThread, fin | |
private static Metrics createMetrics(final StreamsConfig config, final Time time, final String clientId) { | ||
final MetricConfig metricConfig = new MetricConfig() | ||
.samples(config.getInt(StreamsConfig.METRICS_NUM_SAMPLES_CONFIG)) | ||
.recordLevel(Sensor.RecordingLevel.forName(config.getString(StreamsConfig.METRICS_RECORDING_LEVEL_CONFIG))) | ||
.recordLevel(Sensor.RecordingLevel.forName(config.getString(METRICS_RECORDING_LEVEL_CONFIG))) | ||
.timeWindow(config.getLong(StreamsConfig.METRICS_SAMPLE_WINDOW_MS_CONFIG), TimeUnit.MILLISECONDS); | ||
final List<MetricsReporter> reporters = CommonClientConfigs.metricsReporters(clientId, config); | ||
|
||
|
@@ -1250,6 +1253,20 @@ private Optional<String> removeStreamThread(final long timeoutMs) throws Timeout | |
return Optional.empty(); | ||
} | ||
|
||
private int calculateMetricsRecordingLevel() { | ||
final int recordingLevel; | ||
final String recordingLevelString = applicationConfigs.getString(METRICS_RECORDING_LEVEL_CONFIG); | ||
if (recordingLevelString.equals("INFO")) { | ||
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. Why not use a |
||
recordingLevel = 0; | ||
} else if (recordingLevelString.equals("DEBUG")) { | ||
recordingLevel = 1; | ||
} else { | ||
// Must be TRACE level | ||
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. Might be better to use another |
||
recordingLevel = 2; | ||
} | ||
return recordingLevel; | ||
} | ||
|
||
/* | ||
* Takes a snapshot and counts the number of stream threads which are not in PENDING_SHUTDOWN or DEAD | ||
* | ||
|
@@ -1334,7 +1351,7 @@ private ScheduledExecutorService setupStateDirCleaner() { | |
|
||
private static ScheduledExecutorService maybeCreateRocksDBMetricsRecordingService(final String clientId, | ||
final StreamsConfig config) { | ||
if (RecordingLevel.forName(config.getString(StreamsConfig.METRICS_RECORDING_LEVEL_CONFIG)) == RecordingLevel.DEBUG) { | ||
if (RecordingLevel.forName(config.getString(METRICS_RECORDING_LEVEL_CONFIG)) == RecordingLevel.DEBUG) { | ||
return Executors.newSingleThreadScheduledExecutor(r -> { | ||
final Thread thread = new Thread(r, clientId + "-RocksDBMetricsRecordingTrigger"); | ||
thread.setDaemon(true); | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -68,6 +68,7 @@ | |
import java.util.Comparator; | ||
import java.util.HashMap; | ||
import java.util.HashSet; | ||
import java.util.Locale; | ||
import java.util.Map; | ||
import java.util.Optional; | ||
import java.util.Queue; | ||
|
@@ -614,6 +615,12 @@ public StreamThread(final Time time, | |
streamsMetrics, | ||
time.milliseconds() | ||
); | ||
ThreadMetrics.addThreadStateTelemetryMetric(threadId, | ||
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. nit: |
||
streamsMetrics, | ||
(metricConfig, now) -> this.state().ordinal()); | ||
ThreadMetrics.addThreadStateMetric(threadId, | ||
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. as above |
||
streamsMetrics, | ||
(metricConfig, now) -> this.state().name().toLowerCase(Locale.getDefault())); | ||
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. For the corresponding client metric we just use
Why so "complicated" -- I am also ok to update the code for the client metric. But both should be the same? (Or maybe keep |
||
ThreadMetrics.addThreadBlockedTimeMetric( | ||
threadId, | ||
new StreamThreadTotalBlockedTime( | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -16,6 +16,7 @@ | |
*/ | ||
package org.apache.kafka.streams.processor.internals.metrics; | ||
|
||
import org.apache.kafka.common.metrics.Gauge; | ||
import org.apache.kafka.common.metrics.Sensor; | ||
import org.apache.kafka.common.metrics.Sensor.RecordingLevel; | ||
import org.apache.kafka.streams.processor.internals.StreamThreadTotalBlockedTime; | ||
|
@@ -44,7 +45,9 @@ private ThreadMetrics() {} | |
private static final String CREATE_TASK = "task-created"; | ||
private static final String CLOSE_TASK = "task-closed"; | ||
private static final String BLOCKED_TIME = "blocked-time-ns-total"; | ||
private static final String STATE = "state"; | ||
private static final String THREAD_START_TIME = "thread-start-time"; | ||
private static final String THREAD_STATE = "thread-state"; | ||
|
||
private static final String COMMIT_DESCRIPTION = "calls to commit"; | ||
private static final String COMMIT_TOTAL_DESCRIPTION = TOTAL_DESCRIPTION + COMMIT_DESCRIPTION; | ||
|
@@ -88,6 +91,8 @@ private ThreadMetrics() {} | |
"The total time the thread spent blocked on kafka in nanoseconds"; | ||
private static final String THREAD_START_TIME_DESCRIPTION = | ||
"The time that the thread was started"; | ||
private static final String THREAD_STATE_DESCRIPTION = | ||
"The current state of the thread"; | ||
|
||
public static Sensor createTaskSensor(final String threadId, | ||
final StreamsMetricsImpl streamsMetrics) { | ||
|
@@ -290,6 +295,30 @@ public static void addThreadStartTimeMetric(final String threadId, | |
); | ||
} | ||
|
||
public static void addThreadStateTelemetryMetric(final String threadId, | ||
final StreamsMetricsImpl streamsMetrics, | ||
final Gauge<Integer> threadStateProvider) { | ||
streamsMetrics.addThreadLevelMutableMetric( | ||
THREAD_STATE, | ||
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. nit: indention too deep (should only be 4 whitespace, not 8 -- should be an IDE setting) |
||
THREAD_STATE_DESCRIPTION, | ||
threadId, | ||
threadStateProvider | ||
); | ||
} | ||
|
||
public static void addThreadStateMetric(final String threadId, | ||
final StreamsMetricsImpl streamsMetrics, | ||
final Gauge<String> threadStateProvider) { | ||
streamsMetrics.addThreadLevelMutableMetric( | ||
STATE, | ||
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. as above |
||
THREAD_STATE_DESCRIPTION, | ||
threadId, | ||
threadStateProvider | ||
); | ||
} | ||
|
||
|
||
|
||
public static void addThreadBlockedTimeMetric(final String threadId, | ||
final StreamThreadTotalBlockedTime blockedTime, | ||
final StreamsMetricsImpl streamsMetrics) { | ||
|
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.
Telemetry filters out string metrics, so I remove it here from the expected metrics list