-
Notifications
You must be signed in to change notification settings - Fork 14.9k
KAFKA-18615: StreamThread *-ratio metrics suffer from sampling bias #21160
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
Merged
Merged
Changes from 7 commits
Commits
Show all changes
10 commits
Select commit
Hold shift + click to select a range
cad7aba
windowed-sum
aliehsaeedii 8f92e78
record ratio metrics during window
aliehsaeedii 067cced
fix initialization
aliehsaeedii 2a2234b
address reviews
aliehsaeedii 66a76ad
merge with trunk
aliehsaeedii f22c176
minor optimization
aliehsaeedii 7b5d1b0
fix utests + upgrade doc
aliehsaeedii ae241dc
address nits
aliehsaeedii 31a2093
move explanations to docs/streams/upgrade-guide.html
aliehsaeedii 8781105
doc moved to 4.3
aliehsaeedii File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change | ||||||||
|---|---|---|---|---|---|---|---|---|---|---|
|
|
@@ -42,7 +42,9 @@ | |||||||||
| import org.apache.kafka.common.errors.UnsupportedVersionException; | ||||||||||
| import org.apache.kafka.common.internals.KafkaFutureImpl; | ||||||||||
| import org.apache.kafka.common.message.StreamsGroupHeartbeatResponseData; | ||||||||||
| import org.apache.kafka.common.metrics.MetricConfig; | ||||||||||
| import org.apache.kafka.common.metrics.Sensor; | ||||||||||
| import org.apache.kafka.common.metrics.stats.WindowedSum; | ||||||||||
| import org.apache.kafka.common.requests.StreamsGroupHeartbeatResponse; | ||||||||||
| import org.apache.kafka.common.serialization.ByteArrayDeserializer; | ||||||||||
| import org.apache.kafka.common.utils.LogContext; | ||||||||||
|
|
@@ -375,6 +377,15 @@ public boolean isStartingRunningOrPartitionAssigned() { | |||||||||
| private final boolean eosEnabled; | ||||||||||
| private final boolean processingThreadsEnabled; | ||||||||||
|
|
||||||||||
| private final WindowedSum pollLatencyWindowedSum = new WindowedSum(); | ||||||||||
| private final WindowedSum totalCommitLatencyWindowedSum = new WindowedSum(); | ||||||||||
| private final WindowedSum processLatencyWindowedSum = new WindowedSum(); | ||||||||||
| private final WindowedSum punctuateLatencyWindowedSum = new WindowedSum(); | ||||||||||
| private final WindowedSum runOnceLatencyWindowedSum = new WindowedSum(); | ||||||||||
| private final MetricConfig metricsConfig; | ||||||||||
|
|
||||||||||
| private boolean latencyWindowsInitialized = false; | ||||||||||
|
|
||||||||||
| private volatile long fetchDeadlineClientInstanceId = -1; | ||||||||||
| private volatile KafkaFutureImpl<Uuid> mainConsumerInstanceIdFuture = new KafkaFutureImpl<>(); | ||||||||||
| private volatile KafkaFutureImpl<Uuid> restoreConsumerInstanceIdFuture = new KafkaFutureImpl<>(); | ||||||||||
|
|
@@ -788,6 +799,7 @@ public StreamThread(final Time time, | |||||||||
| this.shutdownErrorHook = shutdownErrorHook; | ||||||||||
| this.streamsUncaughtExceptionHandler = streamsUncaughtExceptionHandler; | ||||||||||
| this.cacheResizer = cacheResizer; | ||||||||||
| this.metricsConfig = streamsMetrics.metricsRegistry().config(); | ||||||||||
|
|
||||||||||
| // The following sensors are created here but their references are not stored in this object, since within | ||||||||||
| // this object they are not recorded. The sensors are created here so that the stream threads starts with all | ||||||||||
|
|
@@ -887,6 +899,7 @@ public void run() { | |||||||||
| boolean cleanRun = false; | ||||||||||
| try { | ||||||||||
| taskManager.init(); | ||||||||||
| initLatencyWindowsIfNeeded(System.currentTimeMillis()); | ||||||||||
aliehsaeedii marked this conversation as resolved.
Outdated
Show resolved
Hide resolved
|
||||||||||
| cleanRun = runLoop(); | ||||||||||
| } catch (final Throwable e) { | ||||||||||
| failedStreamThreadSensor.record(); | ||||||||||
|
|
@@ -1275,11 +1288,13 @@ void runOnceWithoutProcessingThreads() { | |||||||||
|
|
||||||||||
| now = time.milliseconds(); | ||||||||||
| final long runOnceLatency = now - startMs; | ||||||||||
|
|
||||||||||
| recordWindowedSum(now, pollLatency, totalCommitLatency, totalProcessLatency, totalPunctuateLatency, runOnceLatency); | ||||||||||
| recordRatio(now, pollLatencyWindowedSum, pollRatioSensor); | ||||||||||
| recordRatio(now, totalCommitLatencyWindowedSum, commitRatioSensor); | ||||||||||
| recordRatio(now, processLatencyWindowedSum, processRatioSensor); | ||||||||||
| recordRatio(now, punctuateLatencyWindowedSum, punctuateRatioSensor); | ||||||||||
| processRecordsSensor.record(totalProcessed, now); | ||||||||||
| processRatioSensor.record((double) totalProcessLatency / runOnceLatency, now); | ||||||||||
| punctuateRatioSensor.record((double) totalPunctuateLatency / runOnceLatency, now); | ||||||||||
| pollRatioSensor.record((double) pollLatency / runOnceLatency, now); | ||||||||||
| commitRatioSensor.record((double) totalCommitLatency / runOnceLatency, now); | ||||||||||
|
|
||||||||||
| final long timeSinceLastLog = now - lastLogSummaryMs; | ||||||||||
| if (logSummaryIntervalMs > 0 && timeSinceLastLog > logSummaryIntervalMs) { | ||||||||||
|
|
@@ -1362,8 +1377,10 @@ void runOnceWithProcessingThreads() { | |||||||||
|
|
||||||||||
| now = time.milliseconds(); | ||||||||||
| final long runOnceLatency = now - startMs; | ||||||||||
| pollRatioSensor.record((double) pollLatency / runOnceLatency, now); | ||||||||||
| commitRatioSensor.record((double) totalCommitLatency / runOnceLatency, now); | ||||||||||
|
|
||||||||||
| recordWindowedSum(now, pollLatency, totalCommitLatency, 0, 0, runOnceLatency); | ||||||||||
| recordRatio(now, pollLatencyWindowedSum, pollRatioSensor); | ||||||||||
| recordRatio(now, totalCommitLatencyWindowedSum, commitRatioSensor); | ||||||||||
|
|
||||||||||
| if (logSummaryIntervalMs > 0 && now - lastLogSummaryMs > logSummaryIntervalMs) { | ||||||||||
| log.info("Committed {} total tasks since the last update", totalCommittedSinceLastSummary); | ||||||||||
|
|
@@ -2103,4 +2120,46 @@ Admin adminClient() { | |||||||||
| Optional<StreamsRebalanceData> streamsRebalanceData() { | ||||||||||
| return streamsRebalanceData; | ||||||||||
| } | ||||||||||
|
|
||||||||||
| /** | ||||||||||
| * Initialize both WindowedSum instances at exactly the same timestamp so | ||||||||||
| * their windows are aligned from the very beginning. | ||||||||||
| */ | ||||||||||
| private void initLatencyWindowsIfNeeded(final long now) { | ||||||||||
| if (!latencyWindowsInitialized) { | ||||||||||
| // Start both windows at the same instant with a zero record | ||||||||||
| pollLatencyWindowedSum.record(metricsConfig, 0.0, now); | ||||||||||
| this.totalCommitLatencyWindowedSum.record(metricsConfig, 0, now); | ||||||||||
| this.processLatencyWindowedSum.record(metricsConfig, 0, now); | ||||||||||
| this.punctuateLatencyWindowedSum.record(metricsConfig, 0, now); | ||||||||||
aliehsaeedii marked this conversation as resolved.
Outdated
Show resolved
Hide resolved
|
||||||||||
| runOnceLatencyWindowedSum.record(metricsConfig, 0.0, now); | ||||||||||
| latencyWindowsInitialized = true; | ||||||||||
| } | ||||||||||
| } | ||||||||||
|
|
||||||||||
| private void recordWindowedSum(final long now, | ||||||||||
| final double pollLatency, | ||||||||||
| final double totalCommitLatency, | ||||||||||
| final double processLatency, | ||||||||||
| final double punctuateLatency, | ||||||||||
| final double runOnceLatency) { | ||||||||||
| this.pollLatencyWindowedSum.record(metricsConfig, pollLatency, now); | ||||||||||
| this.totalCommitLatencyWindowedSum.record(metricsConfig, totalCommitLatency, now); | ||||||||||
| this.processLatencyWindowedSum.record(metricsConfig, processLatency, now); | ||||||||||
| this.punctuateLatencyWindowedSum.record(metricsConfig, punctuateLatency, now); | ||||||||||
| this.runOnceLatencyWindowedSum.record(metricsConfig, runOnceLatency, now); | ||||||||||
aliehsaeedii marked this conversation as resolved.
Outdated
Show resolved
Hide resolved
|
||||||||||
| } | ||||||||||
|
|
||||||||||
| private void recordRatio(final long now, final WindowedSum windowedSum, final Sensor ratioSensor) { | ||||||||||
| final double runOnceLatencyWindow = | ||||||||||
| runOnceLatencyWindowedSum.measure(metricsConfig, now); | ||||||||||
|
|
||||||||||
| if (runOnceLatencyWindow > 0.0) { | ||||||||||
| final double latencyWindow = | ||||||||||
| windowedSum.measure(metricsConfig, now); | ||||||||||
| ratioSensor.record(latencyWindow / runOnceLatencyWindow); | ||||||||||
|
Comment on lines
+2158
to
+2160
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.
Suggested change
|
||||||||||
| } else { | ||||||||||
| ratioSensor.record(0.0, now); | ||||||||||
| } | ||||||||||
| } | ||||||||||
| } | ||||||||||
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
Uh oh!
There was an error while loading. Please reload this page.