-
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-8980: Refactor state-store-level streams metrics #7584
Changes from all commits
ea7d9f3
b324b39
486ed6b
f54db7d
430d725
e5cd569
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 |
---|---|---|
|
@@ -35,9 +35,10 @@ | |
import org.apache.kafka.streams.processor.internals.RecordBatchingStateRestoreCallback; | ||
import org.apache.kafka.streams.processor.internals.RecordCollector; | ||
import org.apache.kafka.streams.processor.internals.RecordQueue; | ||
import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl; | ||
import org.apache.kafka.streams.state.StoreBuilder; | ||
import org.apache.kafka.streams.state.ValueAndTimestamp; | ||
import org.apache.kafka.streams.state.internals.metrics.Sensors; | ||
import org.apache.kafka.streams.state.internals.metrics.StateStoreMetrics; | ||
|
||
import java.nio.ByteBuffer; | ||
import java.util.Collection; | ||
|
@@ -61,6 +62,7 @@ public final class InMemoryTimeOrderedKeyValueBuffer<K, V> implements TimeOrdere | |
new RecordHeaders(new Header[] {new RecordHeader("v", new byte[] {(byte) 1})}); | ||
private static final RecordHeaders V_2_CHANGELOG_HEADERS = | ||
new RecordHeaders(new Header[] {new RecordHeader("v", new byte[] {(byte) 2})}); | ||
private static final String METRIC_SCOPE = "in-memory-suppression"; | ||
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. I had to add this metric scope (a.k.a. store type) for the metrics tags. Now the tags would
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. What's the tags look like before? 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. Before it was:
|
||
|
||
private final Map<Bytes, BufferKey> index = new HashMap<>(); | ||
private final TreeMap<BufferKey, BufferValue> sortedMap = new TreeMap<>(); | ||
|
@@ -78,6 +80,9 @@ public final class InMemoryTimeOrderedKeyValueBuffer<K, V> implements TimeOrdere | |
private String changelogTopic; | ||
private Sensor bufferSizeSensor; | ||
private Sensor bufferCountSensor; | ||
private StreamsMetricsImpl streamsMetrics; | ||
private String threadId; | ||
private String taskId; | ||
|
||
private volatile boolean open; | ||
|
||
|
@@ -181,10 +186,25 @@ public void setSerdesIfNull(final Serde<K> keySerde, final Serde<V> valueSerde) | |
|
||
@Override | ||
public void init(final ProcessorContext context, final StateStore root) { | ||
taskId = context.taskId().toString(); | ||
final InternalProcessorContext internalProcessorContext = (InternalProcessorContext) context; | ||
|
||
bufferSizeSensor = Sensors.createBufferSizeSensor(this, internalProcessorContext); | ||
bufferCountSensor = Sensors.createBufferCountSensor(this, internalProcessorContext); | ||
streamsMetrics = internalProcessorContext.metrics(); | ||
|
||
threadId = Thread.currentThread().getName(); | ||
bufferSizeSensor = StateStoreMetrics.suppressionBufferSizeSensor( | ||
threadId, | ||
taskId, | ||
METRIC_SCOPE, | ||
storeName, | ||
streamsMetrics | ||
); | ||
bufferCountSensor = StateStoreMetrics.suppressionBufferCountSensor( | ||
threadId, | ||
taskId, | ||
METRIC_SCOPE, | ||
storeName, | ||
streamsMetrics | ||
); | ||
Comment on lines
+194
to
+207
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. Metrics for suppression buffers are fetched in this way now. 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. Sounds good, I think not relying on context but passing in parameters explicitly is fine (if that's what you mean has changed here). 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. I wanted just to highlight the most important parts of the PR. Actually, I would prefer to add the thread ID to the context and use the context for the metrics. I have already tried to do it, but I ran into a 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. +1 for adding the thread ID to the context |
||
|
||
context.register(root, (RecordBatchingStateRestoreCallback) this::restoreBatch); | ||
if (loggingEnabled) { | ||
|
@@ -210,6 +230,7 @@ public void close() { | |
memBufferSize = 0; | ||
minTimestamp = Long.MAX_VALUE; | ||
updateBufferMetrics(); | ||
streamsMetrics.removeAllStoreLevelSensors(threadId, taskId, storeName); | ||
} | ||
|
||
@Override | ||
|
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.
Corrected mistake in naming.