-
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?
Conversation
@@ -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()); |
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
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 do not understand this. According to the KIP and the code in ThreadMetrics
the type of the metric is numeric.
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.
We also added a state
metric for stream threads that reports the current state of the thread as a string.
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.
Made a pass. Might be good if @cadonna could take a look too.
} 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 comment
The reason will be displayed to describe this comment to others. Learn more.
Might be better to use another if
and final else
throw an exception as safe guard? Would also highlight that we need to update this code, in case we add a new level?
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 comment
The reason will be displayed to describe this comment to others. Learn more.
Why not use a switch
statement?
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 thought Java 11 had switch expressions but it's Java 13+ so I was a little disappointed, but now looking at again the code will still benefit from using switch
@@ -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 comment
The reason will be displayed to describe this comment to others. Learn more.
nit: threadId
should be in the next line by itself
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.
nit: we usually use 4 spaces indentation.
ThreadMetrics.addThreadStateTelemetryMetric(threadId, | ||
streamsMetrics, | ||
(metricConfig, now) -> this.state().ordinal()); | ||
ThreadMetrics.addThreadStateMetric(threadId, |
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.
as above
(metricConfig, now) -> this.state().ordinal()); | ||
ThreadMetrics.addThreadStateMetric(threadId, | ||
streamsMetrics, | ||
(metricConfig, now) -> this.state().name().toLowerCase(Locale.getDefault())); |
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.
For the corresponding client metric we just use
ClientMetrics.addStateMetric(streamsMetrics, (metricsConfig, now) -> state);
Why so "complicated" -- I am also ok to update the code for the client metric. But both should be the same?
(Or maybe keep ... -> state
and add a "fancy" toString()
overload to both enum
(for client and thread) which model the state?
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.
The KIP specifies lower-case string but I like the other approach so I'll update it.
final StreamsMetricsImpl streamsMetrics, | ||
final Gauge<Integer> threadStateProvider) { | ||
streamsMetrics.addThreadLevelMutableMetric( | ||
THREAD_STATE, |
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.
nit: indention too deep (should only be 4 whitespace, not 8 -- should be an IDE setting)
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.
Ack, I think I need to copy settings from someone this is a common occurance with my PRs
final StreamsMetricsImpl streamsMetrics, | ||
final Gauge<String> threadStateProvider) { | ||
streamsMetrics.addThreadLevelMutableMetric( | ||
STATE, |
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.
as above
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 comment
The reason will be displayed to describe this comment to others. Learn more.
Nit: line too long. How about
final List<String> expectedInstanceMetrics = Arrays.asList(
"org.apache.kafka.stream.client.state",
"org.apache.kafka.stream.alive.stream.threads",
"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 comment
The reason will be displayed to describe this comment to others. Learn more.
Thanks for the PR, @bbejeck !
Could you please also update MetricsIntegrationTest
with the new metrics?
public void shouldAddThreadStateTelemetryMetric() { | ||
final Gauge<Integer> threadStateProvider = (streamsMetrics, startTime) -> StreamThread.State.RUNNING.ordinal(); | ||
ThreadMetrics.addThreadStateTelemetryMetric( | ||
"threadId", |
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.
There is a constant for a test thread ID:
"threadId", | |
THREAD_ID, |
verify(streamsMetrics).addThreadLevelMutableMetric( | ||
"thread-state", | ||
"The current state of the thread", | ||
"threadId", |
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.
"threadId", | |
THREAD_ID, |
public void shouldAddThreadStateJMXMetric() { | ||
final Gauge<String> threadStateProvider = (streamsMetrics, startTime) -> StreamThread.State.RUNNING.name().toLowerCase(Locale.getDefault()); | ||
ThreadMetrics.addThreadStateMetric( | ||
"threadId", |
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.
"threadId", | |
THREAD_ID, |
verify(streamsMetrics).addThreadLevelMutableMetric( | ||
"state", | ||
"The current state of the thread", | ||
"threadId", |
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.
"threadId", | |
THREAD_ID, |
@@ -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 comment
The reason will be displayed to describe this comment to others. Learn more.
nit: we usually use 4 spaces indentation.
.recordLevel(Sensor.RecordingLevel.forName(config.getString(StreamsConfig.METRICS_RECORDING_LEVEL_CONFIG))) | ||
.recordLevel(Sensor.RecordingLevel.forName(config.getString(METRICS_RECORDING_LEVEL_CONFIG))) |
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.
Why did you remove the StreamsConfig.
prefix? For all other config names, we use the prefix.
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.
Unintended, I let IntelliJ do the driving on that one, I'll revert
@@ -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()); |
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 do not understand this. According to the KIP and the code in ThreadMetrics
the type of the metric is numeric.
} | ||
|
||
@Test | ||
public void shouldAddThreadStateJMXMetric() { |
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.
nit:
public void shouldAddThreadStateJMXMetric() { | |
public void shouldAddThreadStateJmxMetric() { |
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.
LGTM. Assuming build passes.
Implementation of KIP-1091 adding operator metrics to Kafka Streams
Updated existing tests to validate added metrics
Committer Checklist (excluded from commit message)