diff --git a/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java b/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java index b9bc586c096e1..5a00fe39f6cad 100644 --- a/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java +++ b/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java @@ -37,6 +37,7 @@ import org.apache.kafka.streams.errors.ProcessorStateException; import org.apache.kafka.streams.errors.StreamsException; import org.apache.kafka.streams.internals.ApiUtils; +import org.apache.kafka.streams.internals.metrics.ClientMetrics; import org.apache.kafka.streams.kstream.KStream; import org.apache.kafka.streams.kstream.KTable; import org.apache.kafka.streams.kstream.Produced; @@ -53,6 +54,7 @@ import org.apache.kafka.streams.processor.internals.StreamThread; import org.apache.kafka.streams.processor.internals.StreamsMetadataState; import org.apache.kafka.streams.processor.internals.ThreadStateTransitionValidator; +import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl; import org.apache.kafka.streams.state.HostInfo; import org.apache.kafka.streams.state.QueryableStoreType; import org.apache.kafka.streams.state.StreamsMetadata; @@ -144,6 +146,7 @@ public class KafkaStreams implements AutoCloseable { private final ScheduledExecutorService rocksDBMetricsRecordingService; private final QueryableStoreProvider queryableStoreProvider; private final Admin adminClient; + private final StreamsMetricsImpl streamsMetrics; GlobalStreamThread globalStreamThread; private KafkaStreams.StateListener stateListener; @@ -672,6 +675,16 @@ private KafkaStreams(final InternalTopologyBuilder internalTopologyBuilder, Collections.singletonMap(StreamsConfig.CLIENT_ID_CONFIG, clientId)); reporters.add(new JmxReporter(JMX_PREFIX)); metrics = new Metrics(metricConfig, reporters, time); + streamsMetrics = + new StreamsMetricsImpl(metrics, clientId, config.getString(StreamsConfig.BUILT_IN_METRICS_VERSION_CONFIG)); + streamsMetrics.setRocksDBMetricsRecordingTrigger(rocksDBMetricsRecordingTrigger); + ClientMetrics.addVersionMetric(streamsMetrics); + ClientMetrics.addCommitIdMetric(streamsMetrics); + ClientMetrics.addApplicationIdMetric(streamsMetrics, config.getString(StreamsConfig.APPLICATION_ID_CONFIG)); + ClientMetrics.addTopologyDescriptionMetric(streamsMetrics, internalTopologyBuilder.describe().toString()); + ClientMetrics.addStateMetric(streamsMetrics, (metricsConfig, now) -> state); + log.info("Kafka Streams version: {}", ClientMetrics.version()); + log.info("Kafka Streams commit ID: {}", ClientMetrics.commitId()); // re-write the physical topology according to the config internalTopologyBuilder.rewriteTopology(config); @@ -712,11 +725,10 @@ private KafkaStreams(final InternalTopologyBuilder internalTopologyBuilder, clientSupplier.getGlobalConsumer(config.getGlobalConsumerConfigs(clientId)), stateDirectory, cacheSizePerThread, - metrics, + streamsMetrics, time, globalThreadId, - delegatingStateRestoreListener, - rocksDBMetricsRecordingTrigger + delegatingStateRestoreListener ); globalThreadState = globalStreamThread.state(); } @@ -734,14 +746,13 @@ private KafkaStreams(final InternalTopologyBuilder internalTopologyBuilder, adminClient, processId, clientId, - metrics, + streamsMetrics, time, streamsMetadataState, cacheSizePerThread, stateDirectory, delegatingStateRestoreListener, i + 1); - threads[i].setRocksDBMetricsRecordingTrigger(rocksDBMetricsRecordingTrigger); threadState.put(threads[i].getId(), threads[i].state()); storeProviders.add(new StreamThreadStateStoreProvider(threads[i])); } @@ -928,6 +939,7 @@ private boolean close(final long timeoutMs) { adminClient.close(); + streamsMetrics.removeAllClientLevelMetrics(); metrics.close(); setState(State.NOT_RUNNING); }, "kafka-streams-close-thread"); diff --git a/streams/src/main/java/org/apache/kafka/streams/internals/metrics/ClientMetrics.java b/streams/src/main/java/org/apache/kafka/streams/internals/metrics/ClientMetrics.java new file mode 100644 index 0000000000000..564e06241883d --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/internals/metrics/ClientMetrics.java @@ -0,0 +1,116 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.streams.internals.metrics; + +import org.apache.kafka.common.metrics.Gauge; +import org.apache.kafka.common.metrics.Sensor.RecordingLevel; +import org.apache.kafka.streams.KafkaStreams.State; +import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.InputStream; +import java.util.Properties; + +public class ClientMetrics { + private ClientMetrics() {} + + private static final Logger log = LoggerFactory.getLogger(ClientMetrics.class); + private static final String VERSION = "version"; + private static final String COMMIT_ID = "commit-id"; + private static final String APPLICATION_ID = "application-id"; + private static final String TOPOLOGY_DESCRIPTION = "topology-description"; + private static final String STATE = "state"; + private static final String VERSION_FROM_FILE; + private static final String COMMIT_ID_FROM_FILE; + private static final String DEFAULT_VALUE = "unknown"; + + static { + final Properties props = new Properties(); + try (InputStream resourceStream = ClientMetrics.class.getResourceAsStream( + "/kafka/kafka-streams-version.properties")) { + + props.load(resourceStream); + } catch (final Exception exception) { + log.warn("Error while loading kafka-streams-version.properties", exception); + } + VERSION_FROM_FILE = props.getProperty("version", DEFAULT_VALUE).trim(); + COMMIT_ID_FROM_FILE = props.getProperty("commitId", DEFAULT_VALUE).trim(); + } + + private static final String VERSION_DESCRIPTION = "The version of the Kafka Streams client"; + private static final String COMMIT_ID_DESCRIPTION = "The version control commit ID of the Kafka Streams client"; + private static final String APPLICATION_ID_DESCRIPTION = "The application ID of the Kafka Streams client"; + private static final String TOPOLOGY_DESCRIPTION_DESCRIPTION = + "The description of the topology executed in the Kafka Streams client"; + private static final String STATE_DESCRIPTION = "The state of the Kafka Streams client"; + + public static String version() { + return VERSION_FROM_FILE; + } + + public static String commitId() { + return COMMIT_ID_FROM_FILE; + } + + public static void addVersionMetric(final StreamsMetricsImpl streamsMetrics) { + streamsMetrics.addClientLevelImmutableMetric( + VERSION, + VERSION_DESCRIPTION, + RecordingLevel.INFO, + VERSION_FROM_FILE + ); + } + + public static void addCommitIdMetric(final StreamsMetricsImpl streamsMetrics) { + streamsMetrics.addClientLevelImmutableMetric( + COMMIT_ID, + COMMIT_ID_DESCRIPTION, + RecordingLevel.INFO, + COMMIT_ID_FROM_FILE + ); + } + + public static void addApplicationIdMetric(final StreamsMetricsImpl streamsMetrics, final String applicationId) { + streamsMetrics.addClientLevelImmutableMetric( + APPLICATION_ID, + APPLICATION_ID_DESCRIPTION, + RecordingLevel.INFO, + applicationId + ); + } + + public static void addTopologyDescriptionMetric(final StreamsMetricsImpl streamsMetrics, + final String topologyDescription) { + streamsMetrics.addClientLevelImmutableMetric( + TOPOLOGY_DESCRIPTION, + TOPOLOGY_DESCRIPTION_DESCRIPTION, + RecordingLevel.INFO, + topologyDescription + ); + } + + public static void addStateMetric(final StreamsMetricsImpl streamsMetrics, + final Gauge stateProvider) { + streamsMetrics.addClientLevelMutableMetric( + STATE, + STATE_DESCRIPTION, + RecordingLevel.INFO, + stateProvider + ); + } +} diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamAggregate.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamAggregate.java index ca7266f5aba35..7a0d1409466aa 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamAggregate.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamAggregate.java @@ -67,7 +67,7 @@ private class KStreamAggregateProcessor extends AbstractProcessor { public void init(final ProcessorContext context) { super.init(context); metrics = (StreamsMetricsImpl) context.metrics(); - skippedRecordsSensor = ThreadMetrics.skipRecordSensor(metrics); + skippedRecordsSensor = ThreadMetrics.skipRecordSensor(Thread.currentThread().getName(), metrics); store = (TimestampedKeyValueStore) context.getStateStore(storeName); tupleForwarder = new TimestampedTupleForwarder<>( store, diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoin.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoin.java index 97debbc802195..8f22b08e2f5c7 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoin.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoin.java @@ -65,7 +65,7 @@ private class KStreamKStreamJoinProcessor extends AbstractProcessor { public void init(final ProcessorContext context) { super.init(context); metrics = (StreamsMetricsImpl) context.metrics(); - skippedRecordsSensor = ThreadMetrics.skipRecordSensor(metrics); + skippedRecordsSensor = ThreadMetrics.skipRecordSensor(Thread.currentThread().getName(), metrics); otherWindow = (WindowStore) context.getStateStore(otherWindowName); } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKTableJoinProcessor.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKTableJoinProcessor.java index 92fd4d52e5ae2..3123d770ccb20 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKTableJoinProcessor.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKTableJoinProcessor.java @@ -52,7 +52,7 @@ class KStreamKTableJoinProcessor extends AbstractProcessor { public void init(final ProcessorContext context) { super.init(context); metrics = (StreamsMetricsImpl) context.metrics(); - skippedRecordsSensor = ThreadMetrics.skipRecordSensor(metrics); + skippedRecordsSensor = ThreadMetrics.skipRecordSensor(Thread.currentThread().getName(), metrics); store = (TimestampedKeyValueStore) context.getStateStore(storeName); tupleForwarder = new TimestampedTupleForwarder<>( store, diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSessionWindowAggregate.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSessionWindowAggregate.java index cb117b70ec92c..57557014a8fc5 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSessionWindowAggregate.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSessionWindowAggregate.java @@ -94,7 +94,7 @@ public void init(final ProcessorContext context) { internalProcessorContext = (InternalProcessorContext) context; metrics = (StreamsMetricsImpl) context.metrics(); lateRecordDropSensor = Sensors.lateRecordDropSensor(internalProcessorContext); - skippedRecordsSensor = ThreadMetrics.skipRecordSensor(metrics); + skippedRecordsSensor = ThreadMetrics.skipRecordSensor(Thread.currentThread().getName(), metrics); store = (SessionStore) context.getStateStore(storeName); tupleForwarder = new SessionTupleForwarder<>(store, context, new SessionCacheFlushListener<>(context), sendOldValues); diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamWindowAggregate.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamWindowAggregate.java index 2983a3a9c7ea9..923237626c40f 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamWindowAggregate.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamWindowAggregate.java @@ -92,7 +92,7 @@ public void init(final ProcessorContext context) { metrics = internalProcessorContext.metrics(); lateRecordDropSensor = Sensors.lateRecordDropSensor(internalProcessorContext); - skippedRecordsSensor = ThreadMetrics.skipRecordSensor(metrics); + skippedRecordsSensor = ThreadMetrics.skipRecordSensor(Thread.currentThread().getName(), metrics); windowStore = (TimestampedWindowStore) context.getStateStore(storeName); tupleForwarder = new TimestampedTupleForwarder<>( windowStore, diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableInnerJoin.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableInnerJoin.java index 005ea809b3fe7..561fa4e30f006 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableInnerJoin.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableInnerJoin.java @@ -78,7 +78,7 @@ private class KTableKTableJoinProcessor extends AbstractProcessor> public void init(final ProcessorContext context) { super.init(context); metrics = (StreamsMetricsImpl) context.metrics(); - skippedRecordsSensor = ThreadMetrics.skipRecordSensor(metrics); + skippedRecordsSensor = ThreadMetrics.skipRecordSensor(Thread.currentThread().getName(), metrics); valueGetter.init(context); } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableLeftJoin.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableLeftJoin.java index 4bd6af99d6db5..ff88bd6bb7f6b 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableLeftJoin.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableLeftJoin.java @@ -77,7 +77,7 @@ private class KTableKTableLeftJoinProcessor extends AbstractProcessor { public void init(final ProcessorContext context) { super.init(context); metrics = (StreamsMetricsImpl) context.metrics(); - skippedRecordsSensor = ThreadMetrics.skipRecordSensor(metrics); + skippedRecordsSensor = ThreadMetrics.skipRecordSensor(Thread.currentThread().getName(), metrics); if (queryableName != null) { store = (TimestampedKeyValueStore) context.getStateStore(queryableName); tupleForwarder = new TimestampedTupleForwarder<>( diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/ForeignJoinSubscriptionProcessorSupplier.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/ForeignJoinSubscriptionProcessorSupplier.java index 614b91f071e5e..521e2f0ccaef5 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/ForeignJoinSubscriptionProcessorSupplier.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/ForeignJoinSubscriptionProcessorSupplier.java @@ -64,7 +64,8 @@ private final class KTableKTableJoinProcessor extends AbstractProcessor tags = metrics.tagMap( - "task-id", context.taskId().toString() + threadId, + "task-id", + context.taskId().toString() ); sensor.add( new MetricName( @@ -86,8 +99,10 @@ public static Sensor recordLatenessSensor(final InternalProcessorContext context public static Sensor suppressionEmitSensor(final InternalProcessorContext context) { final StreamsMetricsImpl metrics = context.metrics(); + final String threadId = Thread.currentThread().getName(); final Sensor sensor = metrics.nodeLevelSensor( + threadId, context.taskId().toString(), context.currentNode().name(), "suppression-emit", @@ -95,8 +110,11 @@ public static Sensor suppressionEmitSensor(final InternalProcessorContext contex ); final Map tags = metrics.tagMap( - "task-id", context.taskId().toString(), - PROCESSOR_NODE_ID_TAG, context.currentNode().name() + threadId, + "task-id", + context.taskId().toString(), + PROCESSOR_NODE_ID_TAG, + context.currentNode().name() ); sensor.add( diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalStateUpdateTask.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalStateUpdateTask.java index d6f60e351334e..4d27e82ba2a59 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalStateUpdateTask.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalStateUpdateTask.java @@ -70,7 +70,7 @@ public Map initialize() { source, deserializationExceptionHandler, logContext, - ThreadMetrics.skipRecordSensor(processorContext.metrics()) + ThreadMetrics.skipRecordSensor(Thread.currentThread().getName(), processorContext.metrics()) ) ); } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalStreamThread.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalStreamThread.java index 5248aa4f8ab9c..923480fa5b479 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalStreamThread.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalStreamThread.java @@ -23,7 +23,6 @@ import org.apache.kafka.common.Metric; import org.apache.kafka.common.MetricName; import org.apache.kafka.common.TopicPartition; -import org.apache.kafka.common.metrics.Metrics; import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.Time; import org.apache.kafka.common.utils.Utils; @@ -33,7 +32,6 @@ import org.apache.kafka.streams.processor.StateRestoreListener; import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl; import org.apache.kafka.streams.state.internals.ThreadCache; -import org.apache.kafka.streams.state.internals.metrics.RocksDBMetricsRecordingTrigger; import org.slf4j.Logger; import java.io.IOException; @@ -180,27 +178,21 @@ public GlobalStreamThread(final ProcessorTopology topology, final Consumer globalConsumer, final StateDirectory stateDirectory, final long cacheSizeBytes, - final Metrics metrics, + final StreamsMetricsImpl streamsMetrics, final Time time, final String threadClientId, - final StateRestoreListener stateRestoreListener, - final RocksDBMetricsRecordingTrigger rocksDBMetricsRecordingTrigger) { + final StateRestoreListener stateRestoreListener) { super(threadClientId); this.time = time; this.config = config; this.topology = topology; this.globalConsumer = globalConsumer; this.stateDirectory = stateDirectory; - streamsMetrics = new StreamsMetricsImpl( - metrics, - threadClientId, - config.getString(StreamsConfig.BUILT_IN_METRICS_VERSION_CONFIG) - ); - streamsMetrics.setRocksDBMetricsRecordingTrigger(rocksDBMetricsRecordingTrigger); + this.streamsMetrics = streamsMetrics; this.logPrefix = String.format("global-stream-thread [%s] ", threadClientId); this.logContext = new LogContext(logPrefix); this.log = logContext.logger(getClass()); - this.cache = new ThreadCache(logContext, cacheSizeBytes, streamsMetrics); + this.cache = new ThreadCache(logContext, cacheSizeBytes, this.streamsMetrics); this.stateRestoreListener = stateRestoreListener; } @@ -286,7 +278,7 @@ public void run() { setState(State.DEAD); log.warn("Error happened during initialization of the global state store; this thread has shutdown"); - streamsMetrics.removeAllThreadLevelSensors(); + streamsMetrics.removeAllThreadLevelSensors(getName()); return; } @@ -310,7 +302,7 @@ public void run() { log.error("Failed to close state maintainer due to the following error:", e); } - streamsMetrics.removeAllThreadLevelSensors(); + streamsMetrics.removeAllThreadLevelSensors(getName()); setState(DEAD); diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorNode.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorNode.java index bc66edee307c7..6e10e83830056 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorNode.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorNode.java @@ -34,6 +34,7 @@ import static org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl.PROCESSOR_NODE_ID_TAG; import static org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl.PROCESSOR_NODE_METRICS_GROUP; import static org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl.addAvgAndMaxLatencyToSensor; +import static org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl.addInvocationRateAndCountToSensor; public class ProcessorNode { @@ -166,13 +167,27 @@ private static final class NodeMetrics { private NodeMetrics(final StreamsMetricsImpl metrics, final String processorNodeName, final ProcessorContext context) { this.metrics = metrics; + final String threadId = Thread.currentThread().getName(); final String taskName = context.taskId().toString(); - final Map tagMap = metrics.tagMap("task-id", context.taskId().toString(), PROCESSOR_NODE_ID_TAG, processorNodeName); - final Map allTagMap = metrics.tagMap("task-id", context.taskId().toString(), PROCESSOR_NODE_ID_TAG, "all"); + final Map tagMap = metrics.tagMap( + threadId, + "task-id", + context.taskId().toString(), + PROCESSOR_NODE_ID_TAG, + processorNodeName + ); + final Map allTagMap = metrics.tagMap( + threadId, + "task-id", + context.taskId().toString(), + PROCESSOR_NODE_ID_TAG, + "all" + ); nodeProcessTimeSensor = createTaskAndNodeLatencyAndThroughputSensors( "process", metrics, + threadId, taskName, processorNodeName, allTagMap, @@ -182,6 +197,7 @@ private NodeMetrics(final StreamsMetricsImpl metrics, final String processorNode nodePunctuateTimeSensor = createTaskAndNodeLatencyAndThroughputSensors( "punctuate", metrics, + threadId, taskName, processorNodeName, allTagMap, @@ -191,6 +207,7 @@ private NodeMetrics(final StreamsMetricsImpl metrics, final String processorNode nodeCreationSensor = createTaskAndNodeLatencyAndThroughputSensors( "create", metrics, + threadId, taskName, processorNodeName, allTagMap, @@ -201,6 +218,7 @@ private NodeMetrics(final StreamsMetricsImpl metrics, final String processorNode nodeDestructionSensor = createTaskAndNodeLatencyAndThroughputSensors( "destroy", metrics, + threadId, taskName, processorNodeName, allTagMap, @@ -210,6 +228,7 @@ private NodeMetrics(final StreamsMetricsImpl metrics, final String processorNode sourceNodeForwardSensor = createTaskAndNodeLatencyAndThroughputSensors( "forward", metrics, + threadId, taskName, processorNodeName, allTagMap, @@ -221,24 +240,35 @@ private NodeMetrics(final StreamsMetricsImpl metrics, final String processorNode } private void removeAllSensors() { - metrics.removeAllNodeLevelSensors(taskName, processorNodeName); + metrics.removeAllNodeLevelSensors(Thread.currentThread().getName(), taskName, processorNodeName); } private static Sensor createTaskAndNodeLatencyAndThroughputSensors(final String operation, final StreamsMetricsImpl metrics, + final String threadId, final String taskName, final String processorNodeName, final Map taskTags, final Map nodeTags) { - final Sensor parent = metrics.taskLevelSensor(taskName, operation, Sensor.RecordingLevel.DEBUG); + final Sensor parent = metrics.taskLevelSensor( + threadId, + taskName, + operation, + Sensor.RecordingLevel.DEBUG + ); addAvgAndMaxLatencyToSensor(parent, PROCESSOR_NODE_METRICS_GROUP, taskTags, operation); - StreamsMetricsImpl - .addInvocationRateAndCountToSensor(parent, PROCESSOR_NODE_METRICS_GROUP, taskTags, operation); + addInvocationRateAndCountToSensor(parent, PROCESSOR_NODE_METRICS_GROUP, taskTags, operation); - final Sensor sensor = metrics.nodeLevelSensor(taskName, processorNodeName, operation, Sensor.RecordingLevel.DEBUG, parent); + final Sensor sensor = metrics.nodeLevelSensor( + threadId, + taskName, + processorNodeName, + operation, + Sensor.RecordingLevel.DEBUG, + parent + ); addAvgAndMaxLatencyToSensor(sensor, PROCESSOR_NODE_METRICS_GROUP, nodeTags, operation); - StreamsMetricsImpl - .addInvocationRateAndCountToSensor(sensor, PROCESSOR_NODE_METRICS_GROUP, nodeTags, operation); + addInvocationRateAndCountToSensor(sensor, PROCESSOR_NODE_METRICS_GROUP, nodeTags, operation); return sensor; } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordQueue.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordQueue.java index 70eb770a283bf..05849c57fd8c5 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordQueue.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordQueue.java @@ -62,7 +62,8 @@ public class RecordQueue { this.fifoQueue = new ArrayDeque<>(); this.timestampExtractor = timestampExtractor; this.processorContext = processorContext; - skipRecordsSensor = ThreadMetrics.skipRecordSensor(processorContext.metrics()); + skipRecordsSensor = + ThreadMetrics.skipRecordSensor(Thread.currentThread().getName(), processorContext.metrics()); recordDeserializer = new RecordDeserializer( source, deserializationExceptionHandler, diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StandbyTask.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StandbyTask.java index 071b7d667bbbc..f10c25bc5a737 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StandbyTask.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StandbyTask.java @@ -63,7 +63,8 @@ public class StandbyTask extends AbstractTask { final StateDirectory stateDirectory) { super(id, partitions, topology, consumer, changelogReader, true, stateDirectory, config); - closeTaskSensor = metrics.threadLevelSensor("task-closed", Sensor.RecordingLevel.INFO); + closeTaskSensor = metrics + .threadLevelSensor(Thread.currentThread().getName(), "task-closed", Sensor.RecordingLevel.INFO); processorContext = new StandbyContextImpl(id, config, stateMgr, metrics); final Set changelogTopicNames = new HashSet<>(topology.storeToChangelogTopic().values()); diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java index ccf522813f362..40466b388706d 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java @@ -96,19 +96,25 @@ protected static final class TaskMetrics { final StreamsMetricsImpl metrics; final Sensor taskCommitTimeSensor; final Sensor taskEnforcedProcessSensor; + private final String threadId; private final String taskName; - TaskMetrics(final TaskId id, final StreamsMetricsImpl metrics) { - taskName = id.toString(); + TaskMetrics(final String threadId, + final TaskId taskId, + final StreamsMetricsImpl metrics) { + this.threadId = threadId; + taskName = taskId.toString(); this.metrics = metrics; final String group = "stream-task-metrics"; // first add the global operation metrics if not yet, with the global tags only - final Sensor parent = ThreadMetrics.commitOverTasksSensor(metrics); + final Sensor parent = ThreadMetrics.commitOverTasksSensor(threadId, metrics); // add the operation metrics with additional tags - final Map tagMap = metrics.tagMap("task-id", taskName); - taskCommitTimeSensor = metrics.taskLevelSensor(taskName, "commit", Sensor.RecordingLevel.DEBUG, parent); + final Map tagMap = + metrics.tagMap(Thread.currentThread().getName(), "task-id", taskName); + taskCommitTimeSensor = + metrics.taskLevelSensor(threadId, taskName, "commit", Sensor.RecordingLevel.DEBUG, parent); taskCommitTimeSensor.add( new MetricName("commit-latency-avg", group, "The average latency of commit operation.", tagMap), new Avg() @@ -127,7 +133,13 @@ protected static final class TaskMetrics { ); // add the metrics for enforced processing - taskEnforcedProcessSensor = metrics.taskLevelSensor(taskName, "enforced-processing", Sensor.RecordingLevel.DEBUG, parent); + taskEnforcedProcessSensor = metrics.taskLevelSensor( + threadId, + taskName, + "enforced-processing", + Sensor.RecordingLevel.DEBUG, + parent + ); taskEnforcedProcessSensor.add( new MetricName("enforced-processing-rate", group, "The average number of occurrence of enforced-processing operation per second.", tagMap), new Rate(TimeUnit.SECONDS, new WindowedCount()) @@ -140,7 +152,7 @@ protected static final class TaskMetrics { } void removeAllSensors() { - metrics.removeAllTaskLevelSensors(taskName); + metrics.removeAllTaskLevelSensors(threadId, taskName); } } @@ -179,9 +191,10 @@ public StreamTask(final TaskId id, this.time = time; this.producerSupplier = producerSupplier; this.producer = producerSupplier.get(); - this.taskMetrics = new TaskMetrics(id, streamsMetrics); + final String threadId = Thread.currentThread().getName(); + this.taskMetrics = new TaskMetrics(threadId, id, streamsMetrics); - closeTaskSensor = ThreadMetrics.closeTaskSensor(streamsMetrics); + closeTaskSensor = ThreadMetrics.closeTaskSensor(threadId, streamsMetrics); final ProductionExceptionHandler productionExceptionHandler = config.defaultProductionExceptionHandler(); @@ -190,7 +203,7 @@ public StreamTask(final TaskId id, id.toString(), logContext, productionExceptionHandler, - ThreadMetrics.skipRecordSensor(streamsMetrics)); + ThreadMetrics.skipRecordSensor(threadId, streamsMetrics)); } else { this.recordCollector = recordCollector; } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java index e33c88724b0d0..29e1bc7050ee4 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java @@ -29,7 +29,6 @@ import org.apache.kafka.common.Metric; import org.apache.kafka.common.MetricName; import org.apache.kafka.common.TopicPartition; -import org.apache.kafka.common.metrics.Metrics; import org.apache.kafka.common.metrics.Sensor; import org.apache.kafka.common.serialization.ByteArrayDeserializer; import org.apache.kafka.common.utils.LogContext; @@ -46,7 +45,6 @@ import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl; import org.apache.kafka.streams.processor.internals.metrics.ThreadMetrics; import org.apache.kafka.streams.state.internals.ThreadCache; -import org.apache.kafka.streams.state.internals.metrics.RocksDBMetricsRecordingTrigger; import org.slf4j.Logger; import java.time.Duration; @@ -315,7 +313,7 @@ public void close() {} static class TaskCreator extends AbstractTaskCreator { private final ThreadCache cache; private final KafkaClientSupplier clientSupplier; - private final String threadClientId; + private final String threadId; private final Producer threadProducer; private final Sensor createTaskSensor; @@ -328,7 +326,7 @@ static class TaskCreator extends AbstractTaskCreator { final Time time, final KafkaClientSupplier clientSupplier, final Producer threadProducer, - final String threadClientId, + final String threadId, final Logger log) { super( builder, @@ -341,8 +339,8 @@ static class TaskCreator extends AbstractTaskCreator { this.cache = cache; this.clientSupplier = clientSupplier; this.threadProducer = threadProducer; - this.threadClientId = threadClientId; - createTaskSensor = ThreadMetrics.createTaskSensor(streamsMetrics); + this.threadId = threadId; + createTaskSensor = ThreadMetrics.createTaskSensor(threadId, streamsMetrics); } @Override @@ -368,7 +366,7 @@ StreamTask createTask(final Consumer consumer, private Producer createProducer(final TaskId id) { // eos if (threadProducer == null) { - final Map producerConfigs = config.getProducerConfigs(getTaskProducerClientId(threadClientId, id)); + final Map producerConfigs = config.getProducerConfigs(getTaskProducerClientId(threadId, id)); log.info("Creating producer client for task {}", id); producerConfigs.put(ProducerConfig.TRANSACTIONAL_ID_CONFIG, applicationId + "-" + id); return clientSupplier.getProducer(producerConfigs); @@ -398,6 +396,7 @@ static class StandbyTaskCreator extends AbstractTaskCreator { final StateDirectory stateDirectory, final ChangelogReader storeChangelogReader, final Time time, + final String threadId, final Logger log) { super( builder, @@ -407,7 +406,7 @@ static class StandbyTaskCreator extends AbstractTaskCreator { storeChangelogReader, time, log); - createTaskSensor = ThreadMetrics.createTaskSensor(streamsMetrics); + createTaskSensor = ThreadMetrics.createTaskSensor(threadId, streamsMetrics); } @Override @@ -480,21 +479,21 @@ public static StreamThread create(final InternalTopologyBuilder builder, final Admin adminClient, final UUID processId, final String clientId, - final Metrics metrics, + final StreamsMetricsImpl streamsMetrics, final Time time, final StreamsMetadataState streamsMetadataState, final long cacheSizeBytes, final StateDirectory stateDirectory, final StateRestoreListener userStateRestoreListener, final int threadIdx) { - final String threadClientId = clientId + "-StreamThread-" + threadIdx; + final String threadId = clientId + "-StreamThread-" + threadIdx; - final String logPrefix = String.format("stream-thread [%s] ", threadClientId); + final String logPrefix = String.format("stream-thread [%s] ", threadId); final LogContext logContext = new LogContext(logPrefix); final Logger log = logContext.logger(StreamThread.class); log.info("Creating restore consumer client"); - final Map restoreConsumerConfigs = config.getRestoreConsumerConfigs(getRestoreConsumerClientId(threadClientId)); + final Map restoreConsumerConfigs = config.getRestoreConsumerConfigs(getRestoreConsumerClientId(threadId)); final Consumer restoreConsumer = clientSupplier.getRestoreConsumer(restoreConsumerConfigs); final Duration pollTime = Duration.ofMillis(config.getLong(StreamsConfig.POLL_MS_CONFIG)); final StoreChangelogReader changelogReader = new StoreChangelogReader(restoreConsumer, pollTime, userStateRestoreListener, logContext); @@ -502,17 +501,11 @@ public static StreamThread create(final InternalTopologyBuilder builder, Producer threadProducer = null; final boolean eosEnabled = StreamsConfig.EXACTLY_ONCE.equals(config.getString(StreamsConfig.PROCESSING_GUARANTEE_CONFIG)); if (!eosEnabled) { - final Map producerConfigs = config.getProducerConfigs(getThreadProducerClientId(threadClientId)); + final Map producerConfigs = config.getProducerConfigs(getThreadProducerClientId(threadId)); log.info("Creating shared producer client"); threadProducer = clientSupplier.getProducer(producerConfigs); } - final StreamsMetricsImpl streamsMetrics = new StreamsMetricsImpl( - metrics, - threadClientId, - config.getString(StreamsConfig.BUILT_IN_METRICS_VERSION_CONFIG) - ); - final ThreadCache cache = new ThreadCache(logContext, cacheSizeBytes, streamsMetrics); final AbstractTaskCreator activeTaskCreator = new TaskCreator( @@ -525,7 +518,7 @@ public static StreamThread create(final InternalTopologyBuilder builder, time, clientSupplier, threadProducer, - threadClientId, + threadId, log); final AbstractTaskCreator standbyTaskCreator = new StandbyTaskCreator( builder, @@ -534,6 +527,7 @@ public static StreamThread create(final InternalTopologyBuilder builder, stateDirectory, changelogReader, time, + threadId, log); final TaskManager taskManager = new TaskManager( changelogReader, @@ -549,7 +543,7 @@ public static StreamThread create(final InternalTopologyBuilder builder, log.info("Creating consumer client"); final String applicationId = config.getString(StreamsConfig.APPLICATION_ID_CONFIG); - final Map consumerConfigs = config.getMainConsumerConfigs(applicationId, getConsumerClientId(threadClientId), threadIdx); + final Map consumerConfigs = config.getMainConsumerConfigs(applicationId, getConsumerClientId(threadId), threadIdx); consumerConfigs.put(StreamsConfig.InternalConfig.TASK_MANAGER_FOR_PARTITION_ASSIGNOR, taskManager); final AtomicInteger assignmentErrorCode = new AtomicInteger(); consumerConfigs.put(StreamsConfig.InternalConfig.ASSIGNMENT_ERROR_CODE, assignmentErrorCode); @@ -572,7 +566,7 @@ public static StreamThread create(final InternalTopologyBuilder builder, taskManager, streamsMetrics, builder, - threadClientId, + threadId, logContext, assignmentErrorCode) .updateThreadMetadata(getSharedAdminClientId(clientId)); @@ -587,29 +581,29 @@ public StreamThread(final Time time, final TaskManager taskManager, final StreamsMetricsImpl streamsMetrics, final InternalTopologyBuilder builder, - final String threadClientId, + final String threadId, final LogContext logContext, final AtomicInteger assignmentErrorCode) { - super(threadClientId); + super(threadId); this.stateLock = new Object(); this.standbyRecords = new HashMap<>(); this.streamsMetrics = streamsMetrics; - this.commitSensor = ThreadMetrics.commitSensor(streamsMetrics); - this.pollSensor = ThreadMetrics.pollSensor(streamsMetrics); - this.processSensor = ThreadMetrics.processSensor(streamsMetrics); - this.punctuateSensor = ThreadMetrics.punctuateSensor(streamsMetrics); + this.commitSensor = ThreadMetrics.commitSensor(threadId, streamsMetrics); + this.pollSensor = ThreadMetrics.pollSensor(threadId, streamsMetrics); + this.processSensor = ThreadMetrics.processSensor(threadId, streamsMetrics); + this.punctuateSensor = ThreadMetrics.punctuateSensor(threadId, streamsMetrics); // 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 // its metrics initialised. Otherwise, those sensors would have been created during processing, which could // lead to missing metrics. For instance, if no task were created, the metrics for created and closed // tasks would never be added to the metrics. - ThreadMetrics.createTaskSensor(streamsMetrics); - ThreadMetrics.closeTaskSensor(streamsMetrics); - ThreadMetrics.skipRecordSensor(streamsMetrics); - ThreadMetrics.commitOverTasksSensor(streamsMetrics); + ThreadMetrics.createTaskSensor(threadId, streamsMetrics); + ThreadMetrics.closeTaskSensor(threadId, streamsMetrics); + ThreadMetrics.skipRecordSensor(threadId, streamsMetrics); + ThreadMetrics.commitOverTasksSensor(threadId, streamsMetrics); this.time = time; this.builder = builder; @@ -659,10 +653,6 @@ public static String getSharedAdminClientId(final String clientId) { return clientId + "-admin"; } - public void setRocksDBMetricsRecordingTrigger(final RocksDBMetricsRecordingTrigger rocksDBMetricsRecordingTrigger) { - streamsMetrics.setRocksDBMetricsRecordingTrigger(rocksDBMetricsRecordingTrigger); - } - /** * Execute the stream processors * @@ -1131,7 +1121,7 @@ private void completeShutdown(final boolean cleanRun) { } catch (final Throwable e) { log.error("Failed to close restore consumer due to the following error:", e); } - streamsMetrics.removeAllThreadLevelSensors(); + streamsMetrics.removeAllThreadLevelSensors(getName()); setState(State.DEAD); log.info("Shutdown complete"); diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/metrics/StreamsMetricsImpl.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/metrics/StreamsMetricsImpl.java index 0341ab768cdc8..aea1f03341517 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/metrics/StreamsMetricsImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/metrics/StreamsMetricsImpl.java @@ -18,6 +18,8 @@ import org.apache.kafka.common.Metric; import org.apache.kafka.common.MetricName; +import org.apache.kafka.common.metrics.Gauge; +import org.apache.kafka.common.metrics.MetricConfig; import org.apache.kafka.common.metrics.Metrics; import org.apache.kafka.common.metrics.Sensor; import org.apache.kafka.common.metrics.Sensor.RecordingLevel; @@ -51,12 +53,43 @@ public enum Version { FROM_100_TO_23 } + static class ImmutableMetricValue implements Gauge { + private final T value; + + public ImmutableMetricValue(final T value) { + this.value = value; + } + + @Override + public T value(final MetricConfig config, final long now) { + return value; + } + + @Override + public boolean equals(final Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + final ImmutableMetricValue that = (ImmutableMetricValue) o; + return Objects.equals(value, that.value); + } + + @Override + public int hashCode() { + return Objects.hash(value); + } + } + private final Metrics metrics; private final Map parentSensors; - private final String threadName; + private final String clientId; private final Version version; - private final Deque threadLevelSensors = new LinkedList<>(); + private final Deque clientLevelMetrics = new LinkedList<>(); + private final Map> threadLevelSensors = new HashMap<>(); private final Map> taskLevelSensors = new HashMap<>(); private final Map> nodeLevelSensors = new HashMap<>(); private final Map> cacheLevelSensors = new HashMap<>(); @@ -66,7 +99,15 @@ public enum Version { private static final String SENSOR_PREFIX_DELIMITER = "."; private static final String SENSOR_NAME_DELIMITER = ".s."; - + private static final String SENSOR_TASK_LABEL = "task"; + private static final String SENSOR_NODE_LABEL = "node"; + private static final String SENSOR_CACHE_LABEL = "cache"; + private static final String SENSOR_STORE_LABEL = "store"; + private static final String SENSOR_ENTITY_LABEL = "entity"; + private static final String SENSOR_EXTERNAL_LABEL = "external"; + private static final String SENSOR_INTERNAL_LABEL = "internal"; + + public static final String CLIENT_ID_TAG = "client-id"; public static final String THREAD_ID_TAG = "thread-id"; public static final String THREAD_ID_TAG_0100_TO_23 = "client-id"; public static final String TASK_ID_TAG = "task-id"; @@ -92,6 +133,7 @@ public enum Version { public static final String GROUP_PREFIX = GROUP_PREFIX_WO_DELIMITER + "-"; public static final String GROUP_SUFFIX = "-metrics"; public static final String STATE_LEVEL_GROUP_SUFFIX = "-state" + GROUP_SUFFIX; + public static final String CLIENT_LEVEL_GROUP = GROUP_PREFIX_WO_DELIMITER + GROUP_SUFFIX; public static final String THREAD_LEVEL_GROUP = GROUP_PREFIX_WO_DELIMITER + GROUP_SUFFIX; public static final String TASK_LEVEL_GROUP = GROUP_PREFIX + "task" + GROUP_SUFFIX; public static final String STATE_LEVEL_GROUP = GROUP_PREFIX + "state" + GROUP_SUFFIX; @@ -103,12 +145,12 @@ public enum Version { public static final String EXPIRED_WINDOW_RECORD_DROP = "expired-window-record-drop"; public static final String LATE_RECORD_DROP = "late-record-drop"; - public StreamsMetricsImpl(final Metrics metrics, final String threadName, final String builtInMetricsVersion) { + public StreamsMetricsImpl(final Metrics metrics, final String clientId, final String builtInMetricsVersion) { Objects.requireNonNull(metrics, "Metrics cannot be null"); Objects.requireNonNull(builtInMetricsVersion, "Built-in metrics version cannot be null"); this.metrics = metrics; - this.threadName = threadName; - this.version = parseBuiltInMetricsVersion(builtInMetricsVersion); + this.clientId = clientId; + version = parseBuiltInMetricsVersion(builtInMetricsVersion); this.parentSensors = new HashMap<>(); } @@ -133,29 +175,62 @@ public RocksDBMetricsRecordingTrigger rocksDBMetricsRecordingTrigger() { return rocksDBMetricsRecordingTrigger; } - public final Sensor threadLevelSensor(final String sensorName, + public void addClientLevelImmutableMetric(final String name, + final String description, + final RecordingLevel recordingLevel, + final T value) { + final MetricName metricName = metrics.metricName(name, CLIENT_LEVEL_GROUP, description, clientLevelTagMap()); + final MetricConfig metricConfig = new MetricConfig().recordLevel(recordingLevel); + synchronized (clientLevelMetrics) { + metrics.addMetric(metricName, metricConfig, new ImmutableMetricValue<>(value)); + clientLevelMetrics.push(metricName); + } + } + + public void addClientLevelMutableMetric(final String name, + final String description, + final RecordingLevel recordingLevel, + final Gauge valueProvider) { + final MetricName metricName = metrics.metricName(name, CLIENT_LEVEL_GROUP, description, clientLevelTagMap()); + final MetricConfig metricConfig = new MetricConfig().recordLevel(recordingLevel); + synchronized (clientLevelMetrics) { + metrics.addMetric(metricName, metricConfig, valueProvider); + clientLevelMetrics.push(metricName); + } + } + + public final Sensor threadLevelSensor(final String threadId, + final String sensorName, final RecordingLevel recordingLevel, final Sensor... parents) { + final String key = threadSensorPrefix(threadId); synchronized (threadLevelSensors) { - final String fullSensorName = threadSensorPrefix() + SENSOR_NAME_DELIMITER + sensorName; + threadLevelSensors.putIfAbsent(key, new LinkedList<>()); + final String fullSensorName = key + SENSOR_NAME_DELIMITER + sensorName; final Sensor sensor = metrics.sensor(fullSensorName, recordingLevel, parents); - threadLevelSensors.push(fullSensorName); + threadLevelSensors.get(key).push(fullSensorName); return sensor; } } - private String threadSensorPrefix() { - return "internal" + SENSOR_PREFIX_DELIMITER + threadName; + private String threadSensorPrefix(final String threadId) { + return SENSOR_INTERNAL_LABEL + SENSOR_PREFIX_DELIMITER + threadId; } - public Map threadLevelTagMap() { + public Map clientLevelTagMap() { final Map tagMap = new LinkedHashMap<>(); - tagMap.put(THREAD_ID_TAG_0100_TO_23, threadName); + tagMap.put(CLIENT_ID_TAG, clientId); return tagMap; } - public Map threadLevelTagMap(final String... tags) { - final Map tagMap = threadLevelTagMap(); + public Map threadLevelTagMap(final String threadId) { + final Map tagMap = new LinkedHashMap<>(); + tagMap.put(THREAD_ID_TAG_0100_TO_23, threadId); + return tagMap; + } + + public Map threadLevelTagMap(final String threadId, final String... tags) { + final Map tagMap = threadLevelTagMap(threadId); if (tags != null) { if ((tags.length % 2) != 0) { throw new IllegalArgumentException("Tags needs to be specified in key-value pairs"); @@ -168,48 +243,58 @@ public Map threadLevelTagMap(final String... tags) { return tagMap; } - public final void removeAllThreadLevelSensors() { + public final void removeAllClientLevelMetrics() { + synchronized (clientLevelMetrics) { + while (!clientLevelMetrics.isEmpty()) { + metrics.removeMetric(clientLevelMetrics.pop()); + } + } + } + + public final void removeAllThreadLevelSensors(final String threadId) { + final String key = threadSensorPrefix(threadId); synchronized (threadLevelSensors) { - while (!threadLevelSensors.isEmpty()) { - metrics.removeSensor(threadLevelSensors.pop()); + final Deque sensors = threadLevelSensors.remove(key); + while (sensors != null && !sensors.isEmpty()) { + metrics.removeSensor(sensors.pop()); } } } - public Map taskLevelTagMap(final String taskName) { - final Map tagMap = threadLevelTagMap(); - tagMap.put(TASK_ID_TAG, taskName); + public Map taskLevelTagMap(final String threadId, final String taskId) { + final Map tagMap = threadLevelTagMap(threadId); + tagMap.put(TASK_ID_TAG, taskId); return tagMap; } - public Map storeLevelTagMap(final String taskName, final String storeType, final String storeName) { - final Map tagMap = taskLevelTagMap(taskName); + public Map storeLevelTagMap(final String threadId, + final String taskName, + final String storeType, + final String storeName) { + final Map tagMap = taskLevelTagMap(threadId, taskName); tagMap.put(storeType + "-" + STORE_ID_TAG, storeName); return tagMap; } - public final Sensor taskLevelSensor(final String taskName, + public final Sensor taskLevelSensor(final String threadId, + final String taskId, final String sensorName, final RecordingLevel recordingLevel, final Sensor... parents) { - final String key = taskSensorPrefix(taskName); + final String key = taskSensorPrefix(threadId, taskId); synchronized (taskLevelSensors) { if (!taskLevelSensors.containsKey(key)) { taskLevelSensors.put(key, new LinkedList<>()); } - final String fullSensorName = key + SENSOR_NAME_DELIMITER + sensorName; - final Sensor sensor = metrics.sensor(fullSensorName, recordingLevel, parents); - taskLevelSensors.get(key).push(fullSensorName); - return sensor; } } - public final void removeAllTaskLevelSensors(final String taskName) { - final String key = taskSensorPrefix(taskName); + public final void removeAllTaskLevelSensors(final String threadId, final String taskId) { + final String key = taskSensorPrefix(threadId, taskId); synchronized (taskLevelSensors) { final Deque sensors = taskLevelSensors.remove(key); while (sensors != null && !sensors.isEmpty()) { @@ -218,16 +303,18 @@ public final void removeAllTaskLevelSensors(final String taskName) { } } - private String taskSensorPrefix(final String taskName) { - return threadSensorPrefix() + SENSOR_PREFIX_DELIMITER + "task" + SENSOR_PREFIX_DELIMITER + taskName; + private String taskSensorPrefix(final String threadId, final String taskId) { + return threadSensorPrefix(threadId) + SENSOR_PREFIX_DELIMITER + SENSOR_TASK_LABEL + SENSOR_PREFIX_DELIMITER + + taskId; } - public Sensor nodeLevelSensor(final String taskName, + public Sensor nodeLevelSensor(final String threadId, + final String taskId, final String processorNodeName, final String sensorName, final Sensor.RecordingLevel recordingLevel, final Sensor... parents) { - final String key = nodeSensorPrefix(taskName, processorNodeName); + final String key = nodeSensorPrefix(threadId, taskId, processorNodeName); synchronized (nodeLevelSensors) { if (!nodeLevelSensors.containsKey(key)) { nodeLevelSensors.put(key, new LinkedList<>()); @@ -243,8 +330,10 @@ public Sensor nodeLevelSensor(final String taskName, } } - public final void removeAllNodeLevelSensors(final String taskName, final String processorNodeName) { - final String key = nodeSensorPrefix(taskName, processorNodeName); + public final void removeAllNodeLevelSensors(final String threadId, + final String taskId, + final String processorNodeName) { + final String key = nodeSensorPrefix(threadId, taskId, processorNodeName); synchronized (nodeLevelSensors) { final Deque sensors = nodeLevelSensors.remove(key); while (sensors != null && !sensors.isEmpty()) { @@ -253,16 +342,18 @@ public final void removeAllNodeLevelSensors(final String taskName, final String } } - private String nodeSensorPrefix(final String taskName, final String processorNodeName) { - return taskSensorPrefix(taskName) + SENSOR_PREFIX_DELIMITER + "node" + SENSOR_PREFIX_DELIMITER + processorNodeName; + private String nodeSensorPrefix(final String threadId, final String taskId, final String processorNodeName) { + return taskSensorPrefix(threadId, taskId) + + SENSOR_PREFIX_DELIMITER + SENSOR_NODE_LABEL + SENSOR_PREFIX_DELIMITER + processorNodeName; } - public Sensor cacheLevelSensor(final String taskName, + public Sensor cacheLevelSensor(final String threadId, + final String taskName, final String storeName, final String sensorName, final Sensor.RecordingLevel recordingLevel, final Sensor... parents) { - final String key = cacheSensorPrefix(taskName, storeName); + final String key = cacheSensorPrefix(threadId, taskName, storeName); synchronized (cacheLevelSensors) { if (!cacheLevelSensors.containsKey(key)) { cacheLevelSensors.put(key, new LinkedList<>()); @@ -278,20 +369,22 @@ public Sensor cacheLevelSensor(final String taskName, } } - public Map cacheLevelTagMap(final String taskName, final String storeName) { + public Map cacheLevelTagMap(final String threadId, + final String taskId, + final String storeName) { final Map tagMap = new LinkedHashMap<>(); - tagMap.put(TASK_ID_TAG, taskName); - tagMap.put(RECORD_CACHE_ID_TAG, storeName); if (version == Version.FROM_100_TO_23) { - tagMap.put(THREAD_ID_TAG_0100_TO_23, Thread.currentThread().getName()); + tagMap.put(THREAD_ID_TAG_0100_TO_23, threadId); } else { - tagMap.put(THREAD_ID_TAG, Thread.currentThread().getName()); + tagMap.put(THREAD_ID_TAG, threadId); } + tagMap.put(TASK_ID_TAG, taskId); + tagMap.put(RECORD_CACHE_ID_TAG, storeName); return tagMap; } - public final void removeAllCacheLevelSensors(final String taskName, final String cacheName) { - final String key = cacheSensorPrefix(taskName, cacheName); + public final void removeAllCacheLevelSensors(final String threadId, final String taskId, final String cacheName) { + final String key = cacheSensorPrefix(threadId, taskId, cacheName); synchronized (cacheLevelSensors) { final Deque strings = cacheLevelSensors.remove(key); while (strings != null && !strings.isEmpty()) { @@ -300,16 +393,18 @@ public final void removeAllCacheLevelSensors(final String taskName, final String } } - private String cacheSensorPrefix(final String taskName, final String cacheName) { - return taskSensorPrefix(taskName) + SENSOR_PREFIX_DELIMITER + "cache" + SENSOR_PREFIX_DELIMITER + cacheName; + private String cacheSensorPrefix(final String threadId, final String taskId, final String cacheName) { + return taskSensorPrefix(threadId, taskId) + + SENSOR_PREFIX_DELIMITER + SENSOR_CACHE_LABEL + SENSOR_PREFIX_DELIMITER + cacheName; } - public final Sensor storeLevelSensor(final String taskName, + public final Sensor storeLevelSensor(final String threadId, + final String taskId, final String storeName, final String sensorName, final Sensor.RecordingLevel recordingLevel, final Sensor... parents) { - final String key = storeSensorPrefix(taskName, storeName); + final String key = storeSensorPrefix(threadId, taskId, storeName); synchronized (storeLevelSensors) { if (!storeLevelSensors.containsKey(key)) { storeLevelSensors.put(key, new LinkedList<>()); @@ -323,8 +418,10 @@ public final Sensor storeLevelSensor(final String taskName, } } - public final void removeAllStoreLevelSensors(final String taskName, final String storeName) { - final String key = storeSensorPrefix(taskName, storeName); + public final void removeAllStoreLevelSensors(final String threadId, + final String taskId, + final String storeName) { + final String key = storeSensorPrefix(threadId, taskId, storeName); synchronized (storeLevelSensors) { final Deque sensors = storeLevelSensors.remove(key); while (sensors != null && !sensors.isEmpty()) { @@ -333,8 +430,11 @@ public final void removeAllStoreLevelSensors(final String taskName, final String } } - private String storeSensorPrefix(final String taskName, final String storeName) { - return taskSensorPrefix(taskName) + SENSOR_PREFIX_DELIMITER + "store" + SENSOR_PREFIX_DELIMITER + storeName; + private String storeSensorPrefix(final String threadId, + final String taskId, + final String storeName) { + return taskSensorPrefix(threadId, taskId) + + SENSOR_PREFIX_DELIMITER + SENSOR_STORE_LABEL + SENSOR_PREFIX_DELIMITER + storeName; } @Override @@ -362,9 +462,9 @@ public void recordThroughput(final Sensor sensor, final long value) { sensor.record(value); } - public final Map tagMap(final String... tags) { + public final Map tagMap(final String threadId, final String... tags) { final Map tagMap = new LinkedHashMap<>(); - tagMap.put("client-id", threadName); + tagMap.put(THREAD_ID_TAG_0100_TO_23, threadId); if (tags != null) { if ((tags.length % 2) != 0) { throw new IllegalArgumentException("Tags needs to be specified in key-value pairs"); @@ -378,11 +478,14 @@ public final Map tagMap(final String... tags) { } - private Map constructTags(final String scopeName, final String entityName, final String... tags) { + private Map constructTags(final String threadId, + final String scopeName, + final String entityName, + final String... tags) { final String[] updatedTags = Arrays.copyOf(tags, tags.length + 2); updatedTags[tags.length] = scopeName + "-id"; updatedTags[tags.length + 1] = entityName; - return tagMap(updatedTags); + return tagMap(threadId, updatedTags); } @@ -397,16 +500,21 @@ public Sensor addLatencyAndThroughputSensor(final String scopeName, final String... tags) { final String group = groupNameFromScope(scopeName); - final Map tagMap = constructTags(scopeName, entityName, tags); - final Map allTagMap = constructTags(scopeName, "all", tags); + final String threadId = Thread.currentThread().getName(); + final Map tagMap = constructTags(threadId, scopeName, entityName, tags); + final Map allTagMap = constructTags(threadId, scopeName, "all", tags); // first add the global operation metrics if not yet, with the global tags only - final Sensor parent = metrics.sensor(externalParentSensorName(operationName), recordingLevel); + final Sensor parent = metrics.sensor(externalParentSensorName(threadId, operationName), recordingLevel); addAvgAndMaxLatencyToSensor(parent, group, allTagMap, operationName); addInvocationRateAndCountToSensor(parent, group, allTagMap, operationName); // add the operation metrics with additional tags - final Sensor sensor = metrics.sensor(externalChildSensorName(operationName, entityName), recordingLevel, parent); + final Sensor sensor = metrics.sensor( + externalChildSensorName(threadId, operationName, entityName), + recordingLevel, + parent + ); addAvgAndMaxLatencyToSensor(sensor, group, tagMap, operationName); addInvocationRateAndCountToSensor(sensor, group, tagMap, operationName); @@ -427,15 +535,23 @@ public Sensor addThroughputSensor(final String scopeName, final String... tags) { final String group = groupNameFromScope(scopeName); - final Map tagMap = constructTags(scopeName, entityName, tags); - final Map allTagMap = constructTags(scopeName, "all", tags); + final String threadId = Thread.currentThread().getName(); + final Map tagMap = constructTags(threadId, scopeName, entityName, tags); + final Map allTagMap = constructTags(threadId, scopeName, "all", tags); // first add the global operation metrics if not yet, with the global tags only - final Sensor parent = metrics.sensor(externalParentSensorName(operationName), recordingLevel); + final Sensor parent = metrics.sensor( + externalParentSensorName(threadId, operationName), + recordingLevel + ); addInvocationRateAndCountToSensor(parent, group, allTagMap, operationName); // add the operation metrics with additional tags - final Sensor sensor = metrics.sensor(externalChildSensorName(operationName, entityName), recordingLevel, parent); + final Sensor sensor = metrics.sensor( + externalChildSensorName(threadId, operationName, entityName), + recordingLevel, + parent + ); addInvocationRateAndCountToSensor(sensor, group, tagMap, operationName); parentSensors.put(sensor, parent); @@ -444,23 +560,22 @@ public Sensor addThroughputSensor(final String scopeName, } - private String externalChildSensorName(final String operationName, final String entityName) { - return "external" + SENSOR_PREFIX_DELIMITER + threadName - + SENSOR_PREFIX_DELIMITER + "entity" + SENSOR_PREFIX_DELIMITER + entityName + private String externalChildSensorName(final String threadId, final String operationName, final String entityName) { + return SENSOR_EXTERNAL_LABEL + SENSOR_PREFIX_DELIMITER + threadId + + SENSOR_PREFIX_DELIMITER + SENSOR_ENTITY_LABEL + SENSOR_PREFIX_DELIMITER + entityName + SENSOR_NAME_DELIMITER + operationName; } - private String externalParentSensorName(final String operationName) { - return "external" + SENSOR_PREFIX_DELIMITER + threadName + SENSOR_NAME_DELIMITER + operationName; + private String externalParentSensorName(final String threadId, final String operationName) { + return SENSOR_EXTERNAL_LABEL + SENSOR_PREFIX_DELIMITER + threadId + SENSOR_NAME_DELIMITER + operationName; } - private static void addAvgAndMaxToSensor(final Sensor sensor, - final String group, - final Map tags, - final String operation, - final String descriptionOfAvg, - final String descriptionOfMax) { + final String group, + final Map tags, + final String operation, + final String descriptionOfAvg, + final String descriptionOfMax) { sensor.add( new MetricName( operation + AVG_SUFFIX, diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/metrics/ThreadMetrics.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/metrics/ThreadMetrics.java index 8cf8a8b3772aa..9b83f71a73c0c 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/metrics/ThreadMetrics.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/metrics/ThreadMetrics.java @@ -72,108 +72,124 @@ private ThreadMetrics() {} private static final String PROCESS_LATENCY = PROCESS + LATENCY_SUFFIX; private static final String PUNCTUATE_LATENCY = PUNCTUATE + LATENCY_SUFFIX; - public static Sensor createTaskSensor(final StreamsMetricsImpl streamsMetrics) { - final Sensor createTaskSensor = streamsMetrics.threadLevelSensor(CREATE_TASK, RecordingLevel.INFO); - addInvocationRateAndCountToSensor(createTaskSensor, - THREAD_LEVEL_GROUP, - streamsMetrics.threadLevelTagMap(), - CREATE_TASK, - CREATE_TASK_TOTAL_DESCRIPTION, - CREATE_TASK_RATE_DESCRIPTION); + public static Sensor createTaskSensor(final String threadId, final StreamsMetricsImpl streamsMetrics) { + final Sensor createTaskSensor = streamsMetrics.threadLevelSensor(threadId, CREATE_TASK, RecordingLevel.INFO); + addInvocationRateAndCountToSensor( + createTaskSensor, + THREAD_LEVEL_GROUP, + streamsMetrics.threadLevelTagMap(threadId), + CREATE_TASK, + CREATE_TASK_TOTAL_DESCRIPTION, + CREATE_TASK_RATE_DESCRIPTION + ); return createTaskSensor; } - public static Sensor closeTaskSensor(final StreamsMetricsImpl streamsMetrics) { - final Sensor closeTaskSensor = streamsMetrics.threadLevelSensor(CLOSE_TASK, RecordingLevel.INFO); - addInvocationRateAndCountToSensor(closeTaskSensor, - THREAD_LEVEL_GROUP, - streamsMetrics.threadLevelTagMap(), - CLOSE_TASK, - CLOSE_TASK_TOTAL_DESCRIPTION, - CLOSE_TASK_RATE_DESCRIPTION); + public static Sensor closeTaskSensor(final String threadId, final StreamsMetricsImpl streamsMetrics) { + final Sensor closeTaskSensor = streamsMetrics.threadLevelSensor(threadId, CLOSE_TASK, RecordingLevel.INFO); + addInvocationRateAndCountToSensor( + closeTaskSensor, + THREAD_LEVEL_GROUP, + streamsMetrics.threadLevelTagMap(threadId), + CLOSE_TASK, + CLOSE_TASK_TOTAL_DESCRIPTION, + CLOSE_TASK_RATE_DESCRIPTION + ); return closeTaskSensor; } - public static Sensor commitSensor(final StreamsMetricsImpl streamsMetrics) { - final Sensor commitSensor = streamsMetrics.threadLevelSensor(COMMIT, Sensor.RecordingLevel.INFO); - final Map tagMap = streamsMetrics.threadLevelTagMap(); + public static Sensor commitSensor(final String threadId, final StreamsMetricsImpl streamsMetrics) { + final Sensor commitSensor = streamsMetrics.threadLevelSensor(threadId, COMMIT, Sensor.RecordingLevel.INFO); + final Map tagMap = streamsMetrics.threadLevelTagMap(threadId); addAvgAndMaxToSensor(commitSensor, THREAD_LEVEL_GROUP, tagMap, COMMIT_LATENCY); - addInvocationRateAndCountToSensor(commitSensor, - THREAD_LEVEL_GROUP, - tagMap, - COMMIT, - COMMIT_TOTAL_DESCRIPTION, - COMMIT_RATE_DESCRIPTION); + addInvocationRateAndCountToSensor( + commitSensor, + THREAD_LEVEL_GROUP, + tagMap, + COMMIT, + COMMIT_TOTAL_DESCRIPTION, + COMMIT_RATE_DESCRIPTION + ); return commitSensor; } - public static Sensor pollSensor(final StreamsMetricsImpl streamsMetrics) { - final Sensor pollSensor = streamsMetrics.threadLevelSensor(POLL, Sensor.RecordingLevel.INFO); - final Map tagMap = streamsMetrics.threadLevelTagMap(); + public static Sensor pollSensor(final String threadId, final StreamsMetricsImpl streamsMetrics) { + final Sensor pollSensor = streamsMetrics.threadLevelSensor(threadId, POLL, Sensor.RecordingLevel.INFO); + final Map tagMap = streamsMetrics.threadLevelTagMap(threadId); addAvgAndMaxToSensor(pollSensor, THREAD_LEVEL_GROUP, tagMap, POLL_LATENCY); - addInvocationRateAndCountToSensor(pollSensor, - THREAD_LEVEL_GROUP, - tagMap, - POLL, - POLL_TOTAL_DESCRIPTION, - POLL_RATE_DESCRIPTION); + addInvocationRateAndCountToSensor( + pollSensor, + THREAD_LEVEL_GROUP, + tagMap, + POLL, + POLL_TOTAL_DESCRIPTION, + POLL_RATE_DESCRIPTION + ); return pollSensor; } - public static Sensor processSensor(final StreamsMetricsImpl streamsMetrics) { - final Sensor processSensor = streamsMetrics.threadLevelSensor(PROCESS, Sensor.RecordingLevel.INFO); - final Map tagMap = streamsMetrics.threadLevelTagMap(); + public static Sensor processSensor(final String threadId, final StreamsMetricsImpl streamsMetrics) { + final Sensor processSensor = streamsMetrics.threadLevelSensor(threadId, PROCESS, Sensor.RecordingLevel.INFO); + final Map tagMap = streamsMetrics.threadLevelTagMap(threadId); addAvgAndMaxToSensor(processSensor, THREAD_LEVEL_GROUP, tagMap, PROCESS_LATENCY); - addInvocationRateAndCountToSensor(processSensor, - THREAD_LEVEL_GROUP, - tagMap, - PROCESS, - PROCESS_TOTAL_DESCRIPTION, - PROCESS_RATE_DESCRIPTION); - + addInvocationRateAndCountToSensor( + processSensor, + THREAD_LEVEL_GROUP, + tagMap, + PROCESS, + PROCESS_TOTAL_DESCRIPTION, + PROCESS_RATE_DESCRIPTION + ); return processSensor; } - public static Sensor punctuateSensor(final StreamsMetricsImpl streamsMetrics) { - final Sensor punctuateSensor = streamsMetrics.threadLevelSensor(PUNCTUATE, Sensor.RecordingLevel.INFO); - final Map tagMap = streamsMetrics.threadLevelTagMap(); + public static Sensor punctuateSensor(final String threadId, final StreamsMetricsImpl streamsMetrics) { + final Sensor punctuateSensor = streamsMetrics.threadLevelSensor(threadId, PUNCTUATE, Sensor.RecordingLevel.INFO); + final Map tagMap = streamsMetrics.threadLevelTagMap(threadId); addAvgAndMaxToSensor(punctuateSensor, THREAD_LEVEL_GROUP, tagMap, PUNCTUATE_LATENCY); - addInvocationRateAndCountToSensor(punctuateSensor, - THREAD_LEVEL_GROUP, - tagMap, - PUNCTUATE, - PUNCTUATE_TOTAL_DESCRIPTION, - PUNCTUATE_RATE_DESCRIPTION); - + addInvocationRateAndCountToSensor( + punctuateSensor, + THREAD_LEVEL_GROUP, + tagMap, + PUNCTUATE, + PUNCTUATE_TOTAL_DESCRIPTION, + PUNCTUATE_RATE_DESCRIPTION + ); return punctuateSensor; } - public static Sensor skipRecordSensor(final StreamsMetricsImpl streamsMetrics) { - final Sensor skippedRecordsSensor = streamsMetrics.threadLevelSensor(SKIP_RECORD, Sensor.RecordingLevel.INFO); - addInvocationRateAndCountToSensor(skippedRecordsSensor, - THREAD_LEVEL_GROUP, - streamsMetrics.threadLevelTagMap(), - SKIP_RECORD, - SKIP_RECORD_TOTAL_DESCRIPTION, - SKIP_RECORD_RATE_DESCRIPTION); - + public static Sensor skipRecordSensor(final String threadId, final StreamsMetricsImpl streamsMetrics) { + final Sensor skippedRecordsSensor = + streamsMetrics.threadLevelSensor(threadId, SKIP_RECORD, Sensor.RecordingLevel.INFO); + addInvocationRateAndCountToSensor( + skippedRecordsSensor, + THREAD_LEVEL_GROUP, + streamsMetrics.threadLevelTagMap(threadId), + SKIP_RECORD, + SKIP_RECORD_TOTAL_DESCRIPTION, + SKIP_RECORD_RATE_DESCRIPTION + ); return skippedRecordsSensor; } - public static Sensor commitOverTasksSensor(final StreamsMetricsImpl streamsMetrics) { - final Sensor commitOverTasksSensor = streamsMetrics.threadLevelSensor(COMMIT, Sensor.RecordingLevel.DEBUG); - final Map tagMap = streamsMetrics.threadLevelTagMap(TASK_ID_TAG, ROLLUP_VALUE); - addAvgAndMaxToSensor(commitOverTasksSensor, - TASK_LEVEL_GROUP, - tagMap, - COMMIT_LATENCY); - addInvocationRateAndCountToSensor(commitOverTasksSensor, - TASK_LEVEL_GROUP, - tagMap, - COMMIT, - COMMIT_OVER_TASKS_TOTAL_DESCRIPTION, - COMMIT_OVER_TASKS_RATE_DESCRIPTION); - + public static Sensor commitOverTasksSensor(final String threadId, final StreamsMetricsImpl streamsMetrics) { + final Sensor commitOverTasksSensor = + streamsMetrics.threadLevelSensor(threadId, COMMIT, Sensor.RecordingLevel.DEBUG); + final Map tagMap = streamsMetrics.threadLevelTagMap(threadId, TASK_ID_TAG, ROLLUP_VALUE); + addAvgAndMaxToSensor( + commitOverTasksSensor, + TASK_LEVEL_GROUP, + tagMap, + COMMIT_LATENCY + ); + addInvocationRateAndCountToSensor( + commitOverTasksSensor, + TASK_LEVEL_GROUP, + tagMap, + COMMIT, + COMMIT_OVER_TASKS_TOTAL_DESCRIPTION, + COMMIT_OVER_TASKS_RATE_DESCRIPTION + ); return commitOverTasksSensor; } } diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/AbstractRocksDBSegmentedBytesStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/AbstractRocksDBSegmentedBytesStore.java index 97dc8d5bc5f6b..65d2cecf1848c 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/AbstractRocksDBSegmentedBytesStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/AbstractRocksDBSegmentedBytesStore.java @@ -174,9 +174,11 @@ public void init(final ProcessorContext context, this.context = (InternalProcessorContext) context; final StreamsMetricsImpl metrics = this.context.metrics(); + final String threadId = Thread.currentThread().getName(); final String taskName = context.taskId().toString(); expiredRecordSensor = metrics.storeLevelSensor( + threadId, taskName, name(), EXPIRED_WINDOW_RECORD_DROP, @@ -185,7 +187,7 @@ public void init(final ProcessorContext context, addInvocationRateAndCountToSensor( expiredRecordSensor, "stream-" + metricScope + "-metrics", - metrics.tagMap("task-id", taskName, metricScope + "-id", name()), + metrics.tagMap(threadId, "task-id", taskName, metricScope + "-id", name()), EXPIRED_WINDOW_RECORD_DROP ); diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemorySessionStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemorySessionStore.java index 6c64b049ffb3c..85751c72902d6 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemorySessionStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemorySessionStore.java @@ -75,8 +75,10 @@ public String name() { @Override public void init(final ProcessorContext context, final StateStore root) { final StreamsMetricsImpl metrics = ((InternalProcessorContext) context).metrics(); + final String threadId = Thread.currentThread().getName(); final String taskName = context.taskId().toString(); expiredRecordSensor = metrics.storeLevelSensor( + threadId, taskName, name(), EXPIRED_WINDOW_RECORD_DROP, @@ -85,7 +87,7 @@ public void init(final ProcessorContext context, final StateStore root) { addInvocationRateAndCountToSensor( expiredRecordSensor, "stream-" + metricScope + "-metrics", - metrics.tagMap("task-id", taskName, metricScope + "-id", name()), + metrics.tagMap(threadId, "task-id", taskName, metricScope + "-id", name()), EXPIRED_WINDOW_RECORD_DROP ); diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryWindowStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryWindowStore.java index 1a3e26b7fd2a7..49dc566f0281d 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryWindowStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryWindowStore.java @@ -91,8 +91,10 @@ public void init(final ProcessorContext context, final StateStore root) { this.context = (InternalProcessorContext) context; final StreamsMetricsImpl metrics = this.context.metrics(); + final String threadId = Thread.currentThread().getName(); final String taskName = context.taskId().toString(); expiredRecordSensor = metrics.storeLevelSensor( + threadId, taskName, name(), EXPIRED_WINDOW_RECORD_DROP, @@ -101,7 +103,7 @@ public void init(final ProcessorContext context, final StateStore root) { addInvocationRateAndCountToSensor( expiredRecordSensor, "stream-" + metricScope + "-metrics", - metrics.tagMap("task-id", taskName, metricScope + "-id", name()), + metrics.tagMap(threadId, "task-id", taskName, metricScope + "-id", name()), EXPIRED_WINDOW_RECORD_DROP ); diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/KeyValueSegments.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/KeyValueSegments.java index 01510c14790a5..fc49c129b998b 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/KeyValueSegments.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/KeyValueSegments.java @@ -31,7 +31,7 @@ class KeyValueSegments extends AbstractSegments { final long retentionPeriod, final long segmentInterval) { super(name, retentionPeriod, segmentInterval); - metricsRecorder = new RocksDBMetricsRecorder(metricsScope, name); + metricsRecorder = new RocksDBMetricsRecorder(metricsScope, Thread.currentThread().getName(), name); } @Override diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredKeyValueStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredKeyValueStore.java index 07e171f9eb813..2f50ba7a9022f 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredKeyValueStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredKeyValueStore.java @@ -65,8 +65,10 @@ public class MeteredKeyValueStore private Sensor rangeTime; private Sensor flushTime; private StreamsMetricsImpl metrics; + private final String threadId; private String taskName; + MeteredKeyValueStore(final KeyValueStore inner, final String metricsScope, final Time time, @@ -74,6 +76,7 @@ public class MeteredKeyValueStore final Serde valueSerde) { super(inner); this.metricsScope = metricsScope; + threadId = Thread.currentThread().getName(); this.time = time != null ? time : Time.SYSTEM; this.keySerde = keySerde; this.valueSerde = valueSerde; @@ -86,20 +89,112 @@ public void init(final ProcessorContext context, taskName = context.taskId().toString(); final String metricsGroup = "stream-" + metricsScope + "-state-metrics"; - final Map taskTags = metrics.storeLevelTagMap(taskName, metricsScope, ROLLUP_VALUE); - final Map storeTags = metrics.storeLevelTagMap(taskName, metricsScope, name()); + final Map taskTags = + metrics.storeLevelTagMap(threadId, taskName, metricsScope, ROLLUP_VALUE); + final Map storeTags = + metrics.storeLevelTagMap(threadId, taskName, metricsScope, name()); initStoreSerde(context); - putTime = createTaskAndStoreLatencyAndThroughputSensors(DEBUG, "put", metrics, metricsGroup, taskName, name(), taskTags, storeTags); - putIfAbsentTime = createTaskAndStoreLatencyAndThroughputSensors(DEBUG, "put-if-absent", metrics, metricsGroup, taskName, name(), taskTags, storeTags); - putAllTime = createTaskAndStoreLatencyAndThroughputSensors(DEBUG, "put-all", metrics, metricsGroup, taskName, name(), taskTags, storeTags); - getTime = createTaskAndStoreLatencyAndThroughputSensors(DEBUG, "get", metrics, metricsGroup, taskName, name(), taskTags, storeTags); - allTime = createTaskAndStoreLatencyAndThroughputSensors(DEBUG, "all", metrics, metricsGroup, taskName, name(), taskTags, storeTags); - rangeTime = createTaskAndStoreLatencyAndThroughputSensors(DEBUG, "range", metrics, metricsGroup, taskName, name(), taskTags, storeTags); - flushTime = createTaskAndStoreLatencyAndThroughputSensors(DEBUG, "flush", metrics, metricsGroup, taskName, name(), taskTags, storeTags); - deleteTime = createTaskAndStoreLatencyAndThroughputSensors(DEBUG, "delete", metrics, metricsGroup, taskName, name(), taskTags, storeTags); - final Sensor restoreTime = createTaskAndStoreLatencyAndThroughputSensors(DEBUG, "restore", metrics, metricsGroup, taskName, name(), taskTags, storeTags); + putTime = createTaskAndStoreLatencyAndThroughputSensors( + DEBUG, + "put", + metrics, + metricsGroup, + threadId, + taskName, + name(), + taskTags, + storeTags + ); + putIfAbsentTime = createTaskAndStoreLatencyAndThroughputSensors( + DEBUG, + "put-if-absent", + metrics, + metricsGroup, + threadId, + taskName, + name(), + taskTags, + storeTags + ); + putAllTime = createTaskAndStoreLatencyAndThroughputSensors( + DEBUG, + "put-all", + metrics, + metricsGroup, + threadId, + taskName, + name(), + taskTags, + storeTags + ); + getTime = createTaskAndStoreLatencyAndThroughputSensors( + DEBUG, + "get", + metrics, + metricsGroup, + threadId, + taskName, + name(), + taskTags, + storeTags + ); + allTime = createTaskAndStoreLatencyAndThroughputSensors( + DEBUG, + "all", + metrics, + metricsGroup, + threadId, + taskName, + name(), + taskTags, + storeTags + ); + rangeTime = createTaskAndStoreLatencyAndThroughputSensors( + DEBUG, + "range", + metrics, + metricsGroup, + threadId, + taskName, + name(), + taskTags, + storeTags + ); + flushTime = createTaskAndStoreLatencyAndThroughputSensors( + DEBUG, + "flush", + metrics, + metricsGroup, + threadId, + taskName, + name(), + taskTags, + storeTags + ); + deleteTime = createTaskAndStoreLatencyAndThroughputSensors( + DEBUG, + "delete", + metrics, + metricsGroup, + threadId, + taskName, + name(), + taskTags, + storeTags + ); + final Sensor restoreTime = createTaskAndStoreLatencyAndThroughputSensors( + DEBUG, + "restore", + metrics, + metricsGroup, + threadId, + taskName, + name(), + taskTags, + storeTags + ); // register and possibly restore the state from the logs if (restoreTime.shouldRecord()) { @@ -247,7 +342,7 @@ public long approximateNumEntries() { @Override public void close() { super.close(); - metrics.removeAllStoreLevelSensors(taskName, name()); + metrics.removeAllStoreLevelSensors(threadId, taskName, name()); } private interface Action { diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredSessionStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredSessionStore.java index 09030c23ce460..ebd3bbe930219 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredSessionStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredSessionStore.java @@ -52,6 +52,7 @@ public class MeteredSessionStore private Sensor flushTime; private Sensor removeTime; private String taskName; + private final String threadId; MeteredSessionStore(final SessionStore inner, final String metricsScope, @@ -59,6 +60,7 @@ public class MeteredSessionStore final Serde valueSerde, final Time time) { super(inner); + threadId = Thread.currentThread().getName(); this.metricsScope = metricsScope; this.keySerde = keySerde; this.valueSerde = valueSerde; @@ -78,14 +80,66 @@ public void init(final ProcessorContext context, taskName = context.taskId().toString(); final String metricsGroup = "stream-" + metricsScope + "-state-metrics"; - final Map taskTags = metrics.storeLevelTagMap(taskName, metricsScope, ROLLUP_VALUE); - final Map storeTags = metrics.storeLevelTagMap(taskName, metricsScope, name()); + final Map taskTags = + metrics.storeLevelTagMap(threadId, taskName, metricsScope, ROLLUP_VALUE); + final Map storeTags = + metrics.storeLevelTagMap(threadId, taskName, metricsScope, name()); - putTime = createTaskAndStoreLatencyAndThroughputSensors(DEBUG, "put", metrics, metricsGroup, taskName, name(), taskTags, storeTags); - fetchTime = createTaskAndStoreLatencyAndThroughputSensors(DEBUG, "fetch", metrics, metricsGroup, taskName, name(), taskTags, storeTags); - flushTime = createTaskAndStoreLatencyAndThroughputSensors(DEBUG, "flush", metrics, metricsGroup, taskName, name(), taskTags, storeTags); - removeTime = createTaskAndStoreLatencyAndThroughputSensors(DEBUG, "remove", metrics, metricsGroup, taskName, name(), taskTags, storeTags); - final Sensor restoreTime = createTaskAndStoreLatencyAndThroughputSensors(DEBUG, "restore", metrics, metricsGroup, taskName, name(), taskTags, storeTags); + putTime = createTaskAndStoreLatencyAndThroughputSensors( + DEBUG, + "put", + metrics, + metricsGroup, + threadId, + taskName, + name(), + taskTags, + storeTags + ); + fetchTime = createTaskAndStoreLatencyAndThroughputSensors( + DEBUG, + "fetch", + metrics, + metricsGroup, + threadId, + taskName, + name(), + taskTags, + storeTags + ); + flushTime = createTaskAndStoreLatencyAndThroughputSensors( + DEBUG, + "flush", + metrics, + metricsGroup, + threadId, + taskName, + name(), + taskTags, + storeTags + ); + removeTime = createTaskAndStoreLatencyAndThroughputSensors( + DEBUG, + "remove", + metrics, + metricsGroup, + threadId, + taskName, + name(), + taskTags, + storeTags + ); + final Sensor restoreTime = createTaskAndStoreLatencyAndThroughputSensors( + DEBUG, + "restore", + metrics, + metricsGroup, + threadId, + taskName, + name(), + taskTags, + storeTags + ); // register and possibly restore the state from the logs final long startNs = time.nanoseconds(); @@ -240,7 +294,7 @@ public void flush() { @Override public void close() { super.close(); - metrics.removeAllStoreLevelSensors(taskName, name()); + metrics.removeAllStoreLevelSensors(threadId, taskName, name()); } private Bytes keyBytes(final K key) { diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredWindowStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredWindowStore.java index ac86679169af1..149f9f807819b 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredWindowStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredWindowStore.java @@ -54,6 +54,7 @@ public class MeteredWindowStore private Sensor fetchTime; private Sensor flushTime; private ProcessorContext context; + private final String threadId; private String taskName; MeteredWindowStore(final WindowStore inner, @@ -64,6 +65,7 @@ public class MeteredWindowStore final Serde valueSerde) { super(inner); this.windowSizeMs = windowSizeMs; + threadId = Thread.currentThread().getName(); this.metricsScope = metricsScope; this.time = time; this.keySerde = keySerde; @@ -79,13 +81,55 @@ public void init(final ProcessorContext context, taskName = context.taskId().toString(); final String metricsGroup = GROUP_PREFIX + metricsScope + STATE_LEVEL_GROUP_SUFFIX; - final Map taskTags = metrics.storeLevelTagMap(taskName, metricsScope, ROLLUP_VALUE); - final Map storeTags = metrics.storeLevelTagMap(taskName, metricsScope, name()); + final Map taskTags = + metrics.storeLevelTagMap(threadId, taskName, metricsScope, ROLLUP_VALUE); + final Map storeTags = + metrics.storeLevelTagMap(threadId, taskName, metricsScope, name()); - putTime = createTaskAndStoreLatencyAndThroughputSensors(DEBUG, "put", metrics, metricsGroup, taskName, name(), taskTags, storeTags); - fetchTime = createTaskAndStoreLatencyAndThroughputSensors(DEBUG, "fetch", metrics, metricsGroup, taskName, name(), taskTags, storeTags); - flushTime = createTaskAndStoreLatencyAndThroughputSensors(DEBUG, "flush", metrics, metricsGroup, taskName, name(), taskTags, storeTags); - final Sensor restoreTime = createTaskAndStoreLatencyAndThroughputSensors(DEBUG, "restore", metrics, metricsGroup, taskName, name(), taskTags, storeTags); + putTime = createTaskAndStoreLatencyAndThroughputSensors( + DEBUG, + "put", + metrics, + metricsGroup, + threadId, + taskName, + name(), + taskTags, + storeTags + ); + fetchTime = createTaskAndStoreLatencyAndThroughputSensors( + DEBUG, + "fetch", + metrics, + metricsGroup, + threadId, + taskName, + name(), + taskTags, + storeTags + ); + flushTime = createTaskAndStoreLatencyAndThroughputSensors( + DEBUG, + "flush", + metrics, + metricsGroup, + threadId, + taskName, + name(), + taskTags, + storeTags + ); + final Sensor restoreTime = createTaskAndStoreLatencyAndThroughputSensors( + DEBUG, + "restore", + metrics, + metricsGroup, + threadId, + taskName, + name(), + taskTags, + storeTags + ); // register and possibly restore the state from the logs final long startNs = time.nanoseconds(); @@ -218,7 +262,7 @@ public void flush() { @Override public void close() { super.close(); - metrics.removeAllStoreLevelSensors(taskName, name()); + metrics.removeAllStoreLevelSensors(threadId, taskName, name()); } private Bytes keyBytes(final K key) { diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/NamedCache.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/NamedCache.java index 2d53440ef4724..4693fbc7b6540 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/NamedCache.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/NamedCache.java @@ -60,7 +60,12 @@ class NamedCache { this.streamsMetrics = streamsMetrics; storeName = ThreadCache.underlyingStoreNamefromCacheName(name); taskName = ThreadCache.taskIDfromCacheName(name); - hitRatioSensor = NamedCacheMetrics.hitRatioSensor(streamsMetrics, taskName, storeName); + hitRatioSensor = NamedCacheMetrics.hitRatioSensor( + streamsMetrics, + Thread.currentThread().getName(), + taskName, + storeName + ); } synchronized final String name() { @@ -315,7 +320,7 @@ synchronized void close() { currentSizeBytes = 0; dirtyKeys.clear(); cache.clear(); - streamsMetrics.removeAllCacheLevelSensors(taskName, storeName); + streamsMetrics.removeAllCacheLevelSensors(Thread.currentThread().getName(), taskName, storeName); } /** diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBStore.java index 74ccd04becfb3..d04b7c25eb543 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBStore.java @@ -115,7 +115,7 @@ public class RocksDBStore implements KeyValueStore, BulkLoadingSt RocksDBStore(final String name, final String metricsScope) { - this(name, DB_FILE_DIR, new RocksDBMetricsRecorder(metricsScope, name)); + this(name, DB_FILE_DIR, new RocksDBMetricsRecorder(metricsScope, Thread.currentThread().getName(), name)); } RocksDBStore(final String name, diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/TimestampedSegments.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/TimestampedSegments.java index 6545de0f691fc..400511f8ff4e0 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/TimestampedSegments.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/TimestampedSegments.java @@ -31,7 +31,7 @@ class TimestampedSegments extends AbstractSegments { final long retentionPeriod, final long segmentInterval) { super(name, retentionPeriod, segmentInterval); - metricsRecorder = new RocksDBMetricsRecorder(metricsScope, name); + metricsRecorder = new RocksDBMetricsRecorder(metricsScope, Thread.currentThread().getName(), name); } @Override diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/metrics/NamedCacheMetrics.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/metrics/NamedCacheMetrics.java index 3744428ba5828..e7e0e62e6e2cb 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/metrics/NamedCacheMetrics.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/metrics/NamedCacheMetrics.java @@ -35,6 +35,7 @@ private NamedCacheMetrics() {} public static Sensor hitRatioSensor(final StreamsMetricsImpl streamsMetrics, + final String threadId, final String taskName, final String storeName) { @@ -43,6 +44,7 @@ public static Sensor hitRatioSensor(final StreamsMetricsImpl streamsMetrics, if (streamsMetrics.version() == FROM_100_TO_23) { hitRatioName = HIT_RATIO_0100_TO_23; final Sensor taskLevelHitRatioSensor = streamsMetrics.taskLevelSensor( + threadId, taskName, hitRatioName, Sensor.RecordingLevel.DEBUG @@ -50,13 +52,14 @@ public static Sensor hitRatioSensor(final StreamsMetricsImpl streamsMetrics, addAvgAndMinAndMaxToSensor( taskLevelHitRatioSensor, CACHE_LEVEL_GROUP, - streamsMetrics.cacheLevelTagMap(taskName, ROLLUP_VALUE), + streamsMetrics.cacheLevelTagMap(threadId, taskName, ROLLUP_VALUE), hitRatioName, HIT_RATIO_AVG_DESCRIPTION, HIT_RATIO_MIN_DESCRIPTION, HIT_RATIO_MAX_DESCRIPTION ); hitRatioSensor = streamsMetrics.cacheLevelSensor( + threadId, taskName, storeName, hitRatioName, @@ -66,6 +69,7 @@ public static Sensor hitRatioSensor(final StreamsMetricsImpl streamsMetrics, } else { hitRatioName = HIT_RATIO; hitRatioSensor = streamsMetrics.cacheLevelSensor( + threadId, taskName, storeName, hitRatioName, @@ -75,7 +79,7 @@ public static Sensor hitRatioSensor(final StreamsMetricsImpl streamsMetrics, addAvgAndMinAndMaxToSensor( hitRatioSensor, CACHE_LEVEL_GROUP, - streamsMetrics.cacheLevelTagMap(taskName, storeName), + streamsMetrics.cacheLevelTagMap(threadId, taskName, storeName), hitRatioName, HIT_RATIO_AVG_DESCRIPTION, HIT_RATIO_MIN_DESCRIPTION, diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/metrics/RocksDBMetrics.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/metrics/RocksDBMetrics.java index 04da4bc53ab7b..e354cb82ea963 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/metrics/RocksDBMetrics.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/metrics/RocksDBMetrics.java @@ -96,16 +96,24 @@ private RocksDBMetrics() {} private static final String NUMBER_OF_FILE_ERRORS_DESCRIPTION = "Total number of file errors occurred"; public static class RocksDBMetricContext { + private final String threadId; private final String taskName; private final String metricsScope; private final String storeName; - public RocksDBMetricContext(final String taskName, final String metricsScope, final String storeName) { + public RocksDBMetricContext(final String threadId, + final String taskName, + final String metricsScope, + final String storeName) { + this.threadId = threadId; this.taskName = taskName; this.metricsScope = metricsScope; this.storeName = storeName; } + public String threadId() { + return threadId; + } public String taskName() { return taskName; } @@ -125,14 +133,15 @@ public boolean equals(final Object o) { return false; } final RocksDBMetricContext that = (RocksDBMetricContext) o; - return Objects.equals(taskName, that.taskName) && + return Objects.equals(threadId, that.threadId) && + Objects.equals(taskName, that.taskName) && Objects.equals(metricsScope, that.metricsScope) && Objects.equals(storeName, that.storeName); } @Override public int hashCode() { - return Objects.hash(taskName, metricsScope, storeName); + return Objects.hash(threadId, taskName, metricsScope, storeName); } } @@ -142,8 +151,12 @@ public static Sensor bytesWrittenToDatabaseSensor(final StreamsMetricsImpl strea addRateOfSumAndSumMetricsToSensor( sensor, STATE_LEVEL_GROUP, - streamsMetrics - .storeLevelTagMap(metricContext.taskName(), metricContext.metricsScope(), metricContext.storeName()), + streamsMetrics.storeLevelTagMap( + metricContext.threadId(), + metricContext.taskName(), + metricContext.metricsScope(), + metricContext.storeName() + ), BYTES_WRITTEN_TO_DB, BYTES_WRITTEN_TO_DB_RATE_DESCRIPTION, BYTES_WRITTEN_TO_DB_TOTAL_DESCRIPTION @@ -157,8 +170,12 @@ public static Sensor bytesReadFromDatabaseSensor(final StreamsMetricsImpl stream addRateOfSumAndSumMetricsToSensor( sensor, STATE_LEVEL_GROUP, - streamsMetrics - .storeLevelTagMap(metricContext.taskName(), metricContext.metricsScope(), metricContext.storeName()), + streamsMetrics.storeLevelTagMap( + metricContext.threadId(), + metricContext.taskName(), + metricContext.metricsScope(), + metricContext.storeName() + ), BYTES_READ_FROM_DB, BYTES_READ_FROM_DB_RATE_DESCRIPTION, BYTES_READ_FROM_DB_TOTAL_DESCRIPTION @@ -172,8 +189,12 @@ public static Sensor memtableBytesFlushedSensor(final StreamsMetricsImpl streams addRateOfSumAndSumMetricsToSensor( sensor, STATE_LEVEL_GROUP, - streamsMetrics - .storeLevelTagMap(metricContext.taskName(), metricContext.metricsScope(), metricContext.storeName()), + streamsMetrics.storeLevelTagMap( + metricContext.threadId(), + metricContext.taskName(), + metricContext.metricsScope(), + metricContext.storeName() + ), MEMTABLE_BYTES_FLUSHED, MEMTABLE_BYTES_FLUSHED_RATE_DESCRIPTION, MEMTABLE_BYTES_FLUSHED_TOTAL_DESCRIPTION @@ -187,8 +208,12 @@ public static Sensor memtableHitRatioSensor(final StreamsMetricsImpl streamsMetr addValueMetricToSensor( sensor, STATE_LEVEL_GROUP, - streamsMetrics - .storeLevelTagMap(metricContext.taskName(), metricContext.metricsScope(), metricContext.storeName()), + streamsMetrics.storeLevelTagMap( + metricContext.threadId(), + metricContext.taskName(), + metricContext.metricsScope(), + metricContext.storeName() + ), MEMTABLE_HIT_RATIO, MEMTABLE_HIT_RATIO_DESCRIPTION ); @@ -201,8 +226,12 @@ public static Sensor memtableAvgFlushTimeSensor(final StreamsMetricsImpl streams addValueMetricToSensor( sensor, STATE_LEVEL_GROUP, - streamsMetrics - .storeLevelTagMap(metricContext.taskName(), metricContext.metricsScope(), metricContext.storeName()), + streamsMetrics.storeLevelTagMap( + metricContext.threadId(), + metricContext.taskName(), + metricContext.metricsScope(), + metricContext.storeName() + ), MEMTABLE_FLUSH_TIME_AVG, MEMTABLE_FLUSH_TIME_AVG_DESCRIPTION ); @@ -215,8 +244,12 @@ public static Sensor memtableMinFlushTimeSensor(final StreamsMetricsImpl streams addValueMetricToSensor( sensor, STATE_LEVEL_GROUP, - streamsMetrics - .storeLevelTagMap(metricContext.taskName(), metricContext.metricsScope(), metricContext.storeName()), + streamsMetrics.storeLevelTagMap( + metricContext.threadId(), + metricContext.taskName(), + metricContext.metricsScope(), + metricContext.storeName() + ), MEMTABLE_FLUSH_TIME_MIN, MEMTABLE_FLUSH_TIME_MIN_DESCRIPTION ); @@ -229,8 +262,12 @@ public static Sensor memtableMaxFlushTimeSensor(final StreamsMetricsImpl streams addValueMetricToSensor( sensor, STATE_LEVEL_GROUP, - streamsMetrics - .storeLevelTagMap(metricContext.taskName(), metricContext.metricsScope(), metricContext.storeName()), + streamsMetrics.storeLevelTagMap( + metricContext.threadId(), + metricContext.taskName(), + metricContext.metricsScope(), + metricContext.storeName() + ), MEMTABLE_FLUSH_TIME_MAX, MEMTABLE_FLUSH_TIME_MAX_DESCRIPTION ); @@ -243,8 +280,12 @@ public static Sensor writeStallDurationSensor(final StreamsMetricsImpl streamsMe addAvgAndSumMetricsToSensor( sensor, STATE_LEVEL_GROUP, - streamsMetrics - .storeLevelTagMap(metricContext.taskName(), metricContext.metricsScope(), metricContext.storeName()), + streamsMetrics.storeLevelTagMap( + metricContext.threadId(), + metricContext.taskName(), + metricContext.metricsScope(), + metricContext.storeName() + ), WRITE_STALL_DURATION, WRITE_STALL_DURATION_AVG_DESCRIPTION, WRITE_STALL_DURATION_TOTAL_DESCRIPTION @@ -258,8 +299,12 @@ public static Sensor blockCacheDataHitRatioSensor(final StreamsMetricsImpl strea addValueMetricToSensor( sensor, STATE_LEVEL_GROUP, - streamsMetrics - .storeLevelTagMap(metricContext.taskName(), metricContext.metricsScope(), metricContext.storeName()), + streamsMetrics.storeLevelTagMap( + metricContext.threadId(), + metricContext.taskName(), + metricContext.metricsScope(), + metricContext.storeName() + ), BLOCK_CACHE_DATA_HIT_RATIO, BLOCK_CACHE_DATA_HIT_RATIO_DESCRIPTION ); @@ -272,7 +317,12 @@ public static Sensor blockCacheIndexHitRatioSensor(final StreamsMetricsImpl stre addValueMetricToSensor( sensor, STATE_LEVEL_GROUP, - streamsMetrics.storeLevelTagMap(metricContext.taskName(), metricContext.metricsScope(), metricContext.storeName()), + streamsMetrics.storeLevelTagMap( + metricContext.threadId(), + metricContext.taskName(), + metricContext.metricsScope(), + metricContext.storeName() + ), BLOCK_CACHE_INDEX_HIT_RATIO, BLOCK_CACHE_INDEX_HIT_RATIO_DESCRIPTION ); @@ -285,8 +335,12 @@ public static Sensor blockCacheFilterHitRatioSensor(final StreamsMetricsImpl str addValueMetricToSensor( sensor, STATE_LEVEL_GROUP, - streamsMetrics - .storeLevelTagMap(metricContext.taskName(), metricContext.metricsScope(), metricContext.storeName()), + streamsMetrics.storeLevelTagMap( + metricContext.threadId(), + metricContext.taskName(), + metricContext.metricsScope(), + metricContext.storeName() + ), BLOCK_CACHE_FILTER_HIT_RATIO, BLOCK_CACHE_FILTER_HIT_RATIO_DESCRIPTION ); @@ -299,8 +353,12 @@ public static Sensor bytesReadDuringCompactionSensor(final StreamsMetricsImpl st addRateOfSumMetricToSensor( sensor, STATE_LEVEL_GROUP, - streamsMetrics - .storeLevelTagMap(metricContext.taskName(), metricContext.metricsScope(), metricContext.storeName()), + streamsMetrics.storeLevelTagMap( + metricContext.threadId(), + metricContext.taskName(), + metricContext.metricsScope(), + metricContext.storeName() + ), BYTES_READ_DURING_COMPACTION, BYTES_READ_DURING_COMPACTION_DESCRIPTION ); @@ -313,8 +371,12 @@ public static Sensor bytesWrittenDuringCompactionSensor(final StreamsMetricsImpl addRateOfSumMetricToSensor( sensor, STATE_LEVEL_GROUP, - streamsMetrics - .storeLevelTagMap(metricContext.taskName(), metricContext.metricsScope(), metricContext.storeName()), + streamsMetrics.storeLevelTagMap( + metricContext.threadId(), + metricContext.taskName(), + metricContext.metricsScope(), + metricContext.storeName() + ), BYTES_WRITTEN_DURING_COMPACTION, BYTES_WRITTEN_DURING_COMPACTION_DESCRIPTION ); @@ -327,8 +389,12 @@ public static Sensor compactionTimeAvgSensor(final StreamsMetricsImpl streamsMet addValueMetricToSensor( sensor, STATE_LEVEL_GROUP, - streamsMetrics - .storeLevelTagMap(metricContext.taskName(), metricContext.metricsScope(), metricContext.storeName()), + streamsMetrics.storeLevelTagMap( + metricContext.threadId(), + metricContext.taskName(), + metricContext.metricsScope(), + metricContext.storeName() + ), COMPACTION_TIME_AVG, COMPACTION_TIME_AVG_DESCRIPTION ); @@ -341,8 +407,12 @@ public static Sensor compactionTimeMinSensor(final StreamsMetricsImpl streamsMet addValueMetricToSensor( sensor, STATE_LEVEL_GROUP, - streamsMetrics - .storeLevelTagMap(metricContext.taskName(), metricContext.metricsScope(), metricContext.storeName()), + streamsMetrics.storeLevelTagMap( + metricContext.threadId(), + metricContext.taskName(), + metricContext.metricsScope(), + metricContext.storeName() + ), COMPACTION_TIME_MIN, COMPACTION_TIME_MIN_DESCRIPTION ); @@ -355,8 +425,12 @@ public static Sensor compactionTimeMaxSensor(final StreamsMetricsImpl streamsMet addValueMetricToSensor( sensor, STATE_LEVEL_GROUP, - streamsMetrics - .storeLevelTagMap(metricContext.taskName(), metricContext.metricsScope(), metricContext.storeName()), + streamsMetrics.storeLevelTagMap( + metricContext.threadId(), + metricContext.taskName(), + metricContext.metricsScope(), + metricContext.storeName() + ), COMPACTION_TIME_MAX, COMPACTION_TIME_MAX_DESCRIPTION ); @@ -369,8 +443,12 @@ public static Sensor numberOfOpenFilesSensor(final StreamsMetricsImpl streamsMet addSumMetricToSensor( sensor, STATE_LEVEL_GROUP, - streamsMetrics - .storeLevelTagMap(metricContext.taskName(), metricContext.metricsScope(), metricContext.storeName()), + streamsMetrics.storeLevelTagMap( + metricContext.threadId(), + metricContext.taskName(), + metricContext.metricsScope(), + metricContext.storeName() + ), NUMBER_OF_OPEN_FILES, false, NUMBER_OF_OPEN_FILES_DESCRIPTION @@ -384,8 +462,12 @@ public static Sensor numberOfFileErrorsSensor(final StreamsMetricsImpl streamsMe addSumMetricToSensor( sensor, STATE_LEVEL_GROUP, - streamsMetrics - .storeLevelTagMap(metricContext.taskName(), metricContext.metricsScope(), metricContext.storeName()), + streamsMetrics.storeLevelTagMap( + metricContext.threadId(), + metricContext.taskName(), + metricContext.metricsScope(), + metricContext.storeName() + ), NUMBER_OF_FILE_ERRORS, NUMBER_OF_FILE_ERRORS_DESCRIPTION ); @@ -396,6 +478,7 @@ private static Sensor createSensor(final StreamsMetricsImpl streamsMetrics, final RocksDBMetricContext metricContext, final String sensorName) { return streamsMetrics.storeLevelSensor( + metricContext.threadId(), metricContext.taskName(), metricContext.storeName(), sensorName, diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/metrics/RocksDBMetricsRecorder.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/metrics/RocksDBMetricsRecorder.java index de2a37c07bec1..59ade54132ab1 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/metrics/RocksDBMetricsRecorder.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/metrics/RocksDBMetricsRecorder.java @@ -48,12 +48,16 @@ public class RocksDBMetricsRecorder { private final Map statisticsToRecord = new ConcurrentHashMap<>(); private final String metricsScope; private final String storeName; + private final String threadId; private TaskId taskId; private StreamsMetricsImpl streamsMetrics; private boolean isInitialized = false; - public RocksDBMetricsRecorder(final String metricsScope, final String storeName) { + public RocksDBMetricsRecorder(final String metricsScope, + final String threadId, + final String storeName) { this.metricsScope = metricsScope; + this.threadId = threadId; this.storeName = storeName; final LogContext logContext = new LogContext(String.format("[RocksDB Metrics Recorder for %s] ", storeName)); logger = logContext.logger(RocksDBMetricsRecorder.class); @@ -97,7 +101,8 @@ public void addStatistics(final String segmentName, } private void initSensors(final StreamsMetricsImpl streamsMetrics, final TaskId taskId) { - final RocksDBMetricContext metricContext = new RocksDBMetricContext(taskId.toString(), metricsScope, storeName); + final RocksDBMetricContext metricContext = + new RocksDBMetricContext(threadId, taskId.toString(), metricsScope, storeName); bytesWrittenToDatabaseSensor = RocksDBMetrics.bytesWrittenToDatabaseSensor(streamsMetrics, metricContext); bytesReadFromDatabaseSensor = RocksDBMetrics.bytesReadFromDatabaseSensor(streamsMetrics, metricContext); memtableBytesFlushedSensor = RocksDBMetrics.memtableBytesFlushedSensor(streamsMetrics, metricContext); diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/metrics/Sensors.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/metrics/Sensors.java index 8ed4d47f24d53..ae967f38bc21a 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/metrics/Sensors.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/metrics/Sensors.java @@ -18,6 +18,7 @@ import org.apache.kafka.common.MetricName; import org.apache.kafka.common.metrics.Sensor; +import org.apache.kafka.common.metrics.Sensor.RecordingLevel; import org.apache.kafka.common.metrics.stats.Avg; import org.apache.kafka.common.metrics.stats.Max; import org.apache.kafka.common.metrics.stats.Value; @@ -33,18 +34,20 @@ public final class Sensors { private Sensors() {} - public static Sensor createTaskAndStoreLatencyAndThroughputSensors(final Sensor.RecordingLevel level, + public static Sensor createTaskAndStoreLatencyAndThroughputSensors(final RecordingLevel level, final String operation, final StreamsMetricsImpl metrics, final String metricsGroup, + final String threadId, final String taskName, final String storeName, final Map taskTags, final Map storeTags) { - final Sensor taskSensor = metrics.taskLevelSensor(taskName, operation, level); + final Sensor taskSensor = metrics.taskLevelSensor(threadId, taskName, operation, level); addAvgAndMaxLatencyToSensor(taskSensor, metricsGroup, taskTags, operation); addInvocationRateAndCountToSensor(taskSensor, metricsGroup, taskTags, operation); - final Sensor sensor = metrics.storeLevelSensor(taskName, storeName, operation, level, taskSensor); + final Sensor sensor = metrics + .storeLevelSensor(threadId, taskName, storeName, operation, level, taskSensor); addAvgAndMaxLatencyToSensor(sensor, metricsGroup, storeTags, operation); addInvocationRateAndCountToSensor(sensor, metricsGroup, storeTags, operation); return sensor; @@ -64,10 +67,12 @@ private static Sensor getBufferSizeOrCountSensor(final StateStore store, final InternalProcessorContext context, final String property) { final StreamsMetricsImpl metrics = context.metrics(); + final String threadId = Thread.currentThread().getName(); final String sensorName = "suppression-buffer-" + property; final Sensor sensor = metrics.storeLevelSensor( + threadId, context.taskId().toString(), store.name(), sensorName, @@ -77,8 +82,11 @@ private static Sensor getBufferSizeOrCountSensor(final StateStore store, final String metricsGroup = "stream-buffer-metrics"; final Map tags = metrics.tagMap( - "task-id", context.taskId().toString(), - "buffer-id", store.name() + threadId, + "task-id", + context.taskId().toString(), + "buffer-id", + store.name() ); sensor.add( diff --git a/streams/src/test/java/org/apache/kafka/streams/KafkaStreamsTest.java b/streams/src/test/java/org/apache/kafka/streams/KafkaStreamsTest.java index fae9020ecac50..d4850634a9129 100644 --- a/streams/src/test/java/org/apache/kafka/streams/KafkaStreamsTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/KafkaStreamsTest.java @@ -28,6 +28,7 @@ import org.apache.kafka.common.serialization.StringSerializer; import org.apache.kafka.common.utils.MockTime; import org.apache.kafka.common.utils.Time; +import org.apache.kafka.streams.internals.metrics.ClientMetrics; import org.apache.kafka.streams.kstream.Materialized; import org.apache.kafka.streams.processor.AbstractProcessor; import org.apache.kafka.streams.processor.StateRestoreListener; @@ -37,6 +38,7 @@ import org.apache.kafka.streams.processor.internals.StateDirectory; import org.apache.kafka.streams.processor.internals.StreamThread; import org.apache.kafka.streams.processor.internals.StreamsMetadataState; +import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl; import org.apache.kafka.streams.state.KeyValueStore; import org.apache.kafka.streams.state.StoreBuilder; import org.apache.kafka.streams.state.Stores; @@ -84,10 +86,11 @@ import static org.junit.Assert.fail; @RunWith(PowerMockRunner.class) -@PrepareForTest({KafkaStreams.class, StreamThread.class}) +@PrepareForTest({KafkaStreams.class, StreamThread.class, ClientMetrics.class}) public class KafkaStreamsTest { private static final int NUM_THREADS = 2; + private final static String APPLICATION_ID = "appId"; @Rule public TestName testName = new TestName(); @@ -139,7 +142,7 @@ public void before() throws Exception { metricsReportersCapture = EasyMock.newCapture(); props = new Properties(); - props.put(StreamsConfig.APPLICATION_ID_CONFIG, "appId"); + props.put(StreamsConfig.APPLICATION_ID_CONFIG, APPLICATION_ID); props.put(StreamsConfig.CLIENT_ID_CONFIG, "clientId"); props.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:2018"); props.put(StreamsConfig.METRIC_REPORTER_CLASSES_CONFIG, MockMetricsReporter.class.getName()); @@ -169,6 +172,15 @@ private void prepareStreams() throws Exception { return null; }).anyTimes(); + PowerMock.mockStatic(ClientMetrics.class); + EasyMock.expect(ClientMetrics.version()).andReturn("1.56"); + EasyMock.expect(ClientMetrics.commitId()).andReturn("1a2b3c4d5e"); + ClientMetrics.addVersionMetric(anyObject(StreamsMetricsImpl.class)); + ClientMetrics.addCommitIdMetric(anyObject(StreamsMetricsImpl.class)); + ClientMetrics.addApplicationIdMetric(anyObject(StreamsMetricsImpl.class), EasyMock.eq(APPLICATION_ID)); + ClientMetrics.addTopologyDescriptionMetric(anyObject(StreamsMetricsImpl.class), anyString()); + ClientMetrics.addStateMetric(anyObject(StreamsMetricsImpl.class), anyObject()); + // setup stream threads PowerMock.mockStatic(StreamThread.class); EasyMock.expect(StreamThread.create( @@ -178,7 +190,7 @@ private void prepareStreams() throws Exception { anyObject(Admin.class), anyObject(UUID.class), anyObject(String.class), - anyObject(Metrics.class), + anyObject(StreamsMetricsImpl.class), anyObject(Time.class), anyObject(StreamsMetadataState.class), anyLong(), @@ -203,11 +215,10 @@ private void prepareStreams() throws Exception { anyObject(Consumer.class), anyObject(StateDirectory.class), anyLong(), - anyObject(Metrics.class), + anyObject(StreamsMetricsImpl.class), anyObject(Time.class), anyString(), - anyObject(StateRestoreListener.class), - anyObject(RocksDBMetricsRecordingTrigger.class) + anyObject(StateRestoreListener.class) ).andReturn(globalStreamThread).anyTimes(); EasyMock.expect(globalStreamThread.state()).andAnswer(globalThreadState::get).anyTimes(); globalStreamThread.setStateListener(EasyMock.capture(threadStatelistenerCapture)); @@ -240,7 +251,16 @@ private void prepareStreams() throws Exception { globalStreamThread.join(); EasyMock.expectLastCall().anyTimes(); - PowerMock.replay(StreamThread.class, Metrics.class, metrics, streamThreadOne, streamThreadTwo, GlobalStreamThread.class, globalStreamThread); + PowerMock.replay( + StreamThread.class, + Metrics.class, + metrics, + ClientMetrics.class, + streamThreadOne, + streamThreadTwo, + GlobalStreamThread.class, + globalStreamThread + ); } private void prepareStreamThread(final StreamThread thread, final boolean terminable) throws Exception { @@ -249,8 +269,6 @@ private void prepareStreamThread(final StreamThread thread, final boolean termin thread.setStateListener(EasyMock.capture(threadStatelistenerCapture)); EasyMock.expectLastCall().anyTimes(); - thread.setRocksDBMetricsRecordingTrigger(EasyMock.anyObject(RocksDBMetricsRecordingTrigger.class)); - EasyMock.expectLastCall().anyTimes(); thread.start(); EasyMock.expectLastCall().andAnswer(() -> { diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/MetricsIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/MetricsIntegrationTest.java index 88659c3559886..6030ac67f6778 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/MetricsIntegrationTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/MetricsIntegrationTest.java @@ -29,6 +29,7 @@ import org.apache.kafka.streams.KeyValue; import org.apache.kafka.streams.StreamsBuilder; import org.apache.kafka.streams.StreamsConfig; +import org.apache.kafka.streams.Topology; import org.apache.kafka.streams.integration.utils.EmbeddedKafkaCluster; import org.apache.kafka.streams.integration.utils.IntegrationTestUtils; import org.apache.kafka.streams.kstream.Consumed; @@ -69,8 +70,10 @@ public class MetricsIntegrationTest { public static final EmbeddedKafkaCluster CLUSTER = new EmbeddedKafkaCluster(NUM_BROKERS); private final long timeout = 60000; + private final static String APPLICATION_ID_VALUE = "stream-metrics-test"; // Metric group + private static final String STREAM_CLIENT_NODE_METRICS = "stream-metrics"; private static final String STREAM_THREAD_NODE_METRICS = "stream-metrics"; private static final String STREAM_TASK_NODE_METRICS = "stream-task-metrics"; private static final String STREAM_PROCESSOR_NODE_METRICS = "stream-processor-node-metrics"; @@ -82,6 +85,11 @@ public class MetricsIntegrationTest { private static final String STREAM_STORE_SESSION_ROCKSDB_STATE_METRICS = "stream-rocksdb-session-state-metrics"; // Metrics name + private static final String VERSION = "version"; + private static final String COMMIT_ID = "commit-id"; + private static final String APPLICATION_ID = "application-id"; + private static final String TOPOLOGY_DESCRIPTION = "topology-description"; + private static final String STATE = "state"; private static final String PUT_LATENCY_AVG = "put-latency-avg"; private static final String PUT_LATENCY_MAX = "put-latency-max"; private static final String PUT_IF_ABSENT_LATENCY_AVG = "put-if-absent-latency-avg"; @@ -205,7 +213,7 @@ public void before() throws InterruptedException { builder = new StreamsBuilder(); CLUSTER.createTopics(STREAM_INPUT, STREAM_OUTPUT_1, STREAM_OUTPUT_2, STREAM_OUTPUT_3, STREAM_OUTPUT_4); streamsConfiguration = new Properties(); - streamsConfiguration.put(StreamsConfig.APPLICATION_ID_CONFIG, "stream-metrics-test"); + streamsConfiguration.put(StreamsConfig.APPLICATION_ID_CONFIG, APPLICATION_ID_VALUE); streamsConfiguration.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, CLUSTER.bootstrapServers()); streamsConfiguration.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.Integer().getClass()); streamsConfiguration.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.String().getClass()); @@ -220,7 +228,13 @@ public void after() throws InterruptedException { } private void startApplication() throws InterruptedException { - kafkaStreams = new KafkaStreams(builder.build(), streamsConfiguration); + final Topology topology = builder.build(); + kafkaStreams = new KafkaStreams(topology, streamsConfiguration); + + verifyStateMetric(State.CREATED); + verifyTopologyDescriptionMetric(topology.describe().toString()); + verifyApplicationIdMetric(APPLICATION_ID_VALUE); + kafkaStreams.start(); TestUtils.waitForCondition( () -> kafkaStreams.state() == State.RUNNING, @@ -306,6 +320,7 @@ private void shouldAddMetricsOnAllLevels(final String builtInMetricsVersion) thr .to(STREAM_OUTPUT_4); startApplication(); + verifyStateMetric(State.RUNNING); checkThreadLevelMetrics(); checkTaskLevelMetrics(); checkProcessorLevelMetrics(); @@ -339,6 +354,8 @@ public void shouldAddMetricsForWindowStore() throws Exception { startApplication(); + verifyStateMetric(State.RUNNING); + waitUntilAllRecordsAreConsumed(); checkWindowStoreMetrics(); @@ -369,6 +386,8 @@ public void shouldAddMetricsForSessionStore() throws Exception { startApplication(); + verifyStateMetric(State.RUNNING); + waitUntilAllRecordsAreConsumed(); checkSessionStoreMetrics(); @@ -401,10 +420,44 @@ public void shouldNotAddRocksDBMetricsIfRecordingLevelIsInfo() throws Exception closeApplication(); } + private void verifyStateMetric(final State state) { + final List metricsList = new ArrayList(kafkaStreams.metrics().values()).stream() + .filter(m -> m.metricName().name().equals(STATE) && + m.metricName().group().equals(STREAM_CLIENT_NODE_METRICS)) + .collect(Collectors.toList()); + assertThat(metricsList.size(), is(1)); + assertThat(metricsList.get(0).metricValue(), is(state)); + } + + private void verifyTopologyDescriptionMetric(final String topologyDescription) { + final List metricsList = new ArrayList(kafkaStreams.metrics().values()).stream() + .filter(m -> m.metricName().name().equals(TOPOLOGY_DESCRIPTION) && + m.metricName().group().equals(STREAM_CLIENT_NODE_METRICS)) + .collect(Collectors.toList()); + assertThat(metricsList.size(), is(1)); + assertThat(metricsList.get(0).metricValue(), is(topologyDescription)); + } + + private void verifyApplicationIdMetric(final String applicationId) { + final List metricsList = new ArrayList(kafkaStreams.metrics().values()).stream() + .filter(m -> m.metricName().name().equals(APPLICATION_ID) && + m.metricName().group().equals(STREAM_CLIENT_NODE_METRICS)) + .collect(Collectors.toList()); + assertThat(metricsList.size(), is(1)); + assertThat(metricsList.get(0).metricValue(), is(applicationId)); + } + private void checkThreadLevelMetrics() { final List listMetricThread = new ArrayList(kafkaStreams.metrics().values()).stream() .filter(m -> m.metricName().group().equals(STREAM_THREAD_NODE_METRICS)) .collect(Collectors.toList()); + // instance-level metrics start + checkMetricByName(listMetricThread, VERSION, 1); + checkMetricByName(listMetricThread, COMMIT_ID, 1); + checkMetricByName(listMetricThread, APPLICATION_ID, 1); + checkMetricByName(listMetricThread, TOPOLOGY_DESCRIPTION, 1); + checkMetricByName(listMetricThread, STATE, 1); + // instance-level metrics end checkMetricByName(listMetricThread, COMMIT_LATENCY_AVG, 1); checkMetricByName(listMetricThread, COMMIT_LATENCY_MAX, 1); checkMetricByName(listMetricThread, POLL_LATENCY_AVG, 1); diff --git a/streams/src/test/java/org/apache/kafka/streams/internals/metrics/ClientMetricsTest.java b/streams/src/test/java/org/apache/kafka/streams/internals/metrics/ClientMetricsTest.java new file mode 100644 index 0000000000000..20f5ac85e4d8b --- /dev/null +++ b/streams/src/test/java/org/apache/kafka/streams/internals/metrics/ClientMetricsTest.java @@ -0,0 +1,137 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.streams.internals.metrics; + + +import org.apache.kafka.common.metrics.Gauge; +import org.apache.kafka.common.metrics.Sensor.RecordingLevel; +import org.apache.kafka.streams.KafkaStreams.State; +import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl; +import org.junit.Test; + +import static org.easymock.EasyMock.and; +import static org.easymock.EasyMock.eq; +import static org.easymock.EasyMock.mock; +import static org.easymock.EasyMock.not; +import static org.easymock.EasyMock.notNull; +import static org.powermock.api.easymock.PowerMock.replay; +import static org.powermock.api.easymock.PowerMock.verify; + +public class ClientMetricsTest { + + private final StreamsMetricsImpl streamsMetrics = mock(StreamsMetricsImpl.class); + + private interface OneParamMetricAdder { + void addMetric(final StreamsMetricsImpl streamsMetrics); + } + + private interface TwoParamMetricAdder { + void addMetric(final StreamsMetricsImpl streamsMetrics, final String value); + } + + /* + * This test may fail when executed from a IDE since it expects the /kafka/kafka-streams-version.properties on the + * class path. + */ + @Test + public void shouldAddVersionMetric() { + final String name = "version"; + final String description = "The version of the Kafka Streams client"; + setUpAndVerifyMetricOneParam(name, description, ClientMetrics::addVersionMetric); + } + + /* + * This test may fail when executed from a IDE since it expects the /kafka/kafka-streams-version.properties on the + * class path. + */ + @Test + public void shouldAddCommitIdMetric() { + final String name = "commit-id"; + final String description = "The version control commit ID of the Kafka Streams client"; + setUpAndVerifyMetricOneParam(name, description, ClientMetrics::addCommitIdMetric); + } + + @Test + public void shouldAddApplicationIdMetric() { + final String name = "application-id"; + final String description = "The application ID of the Kafka Streams client"; + setUpAndVerifyMetricTwoParam(name, description, "thisIsAnID", ClientMetrics::addApplicationIdMetric); + } + + @Test + public void shouldAddTopologyDescriptionMetric() { + final String name = "topology-description"; + final String description = "The description of the topology executed in the Kafka Streams client"; + setUpAndVerifyMetricTwoParam( + name, + description, + "thisIsATopologyDescription", + ClientMetrics::addTopologyDescriptionMetric + ); + } + + @Test + public void shouldAddStateMetric() { + final String name = "state"; + final String description = "The state of the Kafka Streams client"; + final Gauge stateProvider = (config, now) -> State.RUNNING; + streamsMetrics.addClientLevelMutableMetric( + eq(name), + eq(description), + eq(RecordingLevel.INFO), + eq(stateProvider) + ); + replay(streamsMetrics); + + ClientMetrics.addStateMetric(streamsMetrics, stateProvider); + + verify(streamsMetrics); + } + + private void setUpAndVerifyMetricOneParam(final String name, + final String description, + final OneParamMetricAdder metricAdder) { + streamsMetrics.addClientLevelImmutableMetric( + eq(name), + eq(description), + eq(RecordingLevel.INFO), + and(not(eq("unknown")), notNull()) + ); + replay(streamsMetrics); + + metricAdder.addMetric(streamsMetrics); + + verify(streamsMetrics); + } + + private void setUpAndVerifyMetricTwoParam(final String name, + final String description, + final String value, + final TwoParamMetricAdder metricAdder) { + streamsMetrics.addClientLevelImmutableMetric( + eq(name), + eq(description), + eq(RecordingLevel.INFO), + eq(value) + ); + replay(streamsMetrics); + + metricAdder.addMetric(streamsMetrics, value); + + verify(streamsMetrics); + } +} \ No newline at end of file diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamSessionWindowAggregateProcessorTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamSessionWindowAggregateProcessorTest.java index 5b207372964fc..4230958fc4a6d 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamSessionWindowAggregateProcessorTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamSessionWindowAggregateProcessorTest.java @@ -70,6 +70,7 @@ public class KStreamSessionWindowAggregateProcessorTest { private static final long GAP_MS = 5 * 60 * 1000L; private static final String STORE_NAME = "session-store"; + private final String threadId = Thread.currentThread().getName(); private final ToInternal toInternal = new ToInternal(); private final Initializer initializer = () -> 0L; private final Aggregator aggregator = (aggKey, value, aggregate) -> aggregate + 1; @@ -93,7 +94,7 @@ public void initializeStore() { final File stateDir = TestUtils.tempDirectory(); metrics = new Metrics(); final MockStreamsMetrics metrics = new MockStreamsMetrics(KStreamSessionWindowAggregateProcessorTest.this.metrics); - ThreadMetrics.skipRecordSensor(metrics); + ThreadMetrics.skipRecordSensor(threadId, metrics); context = new InternalMockProcessorContext( stateDir, @@ -420,7 +421,7 @@ public void shouldLogAndMeterWhenSkippingLateRecordWithZeroGrace() { "stream-processor-node-metrics", "The total number of occurrence of late-record-drop operations.", mkMap( - mkEntry("client-id", "test"), + mkEntry("client-id", threadId), mkEntry("task-id", "0_0"), mkEntry("processor-node-id", "TESTING_NODE") ) @@ -433,7 +434,7 @@ public void shouldLogAndMeterWhenSkippingLateRecordWithZeroGrace() { "stream-processor-node-metrics", "The average number of occurrence of late-record-drop operations.", mkMap( - mkEntry("client-id", "test"), + mkEntry("client-id", threadId), mkEntry("task-id", "0_0"), mkEntry("processor-node-id", "TESTING_NODE") ) @@ -494,7 +495,7 @@ public void shouldLogAndMeterWhenSkippingLateRecordWithNonzeroGrace() { "stream-processor-node-metrics", "The total number of occurrence of late-record-drop operations.", mkMap( - mkEntry("client-id", "test"), + mkEntry("client-id", threadId), mkEntry("task-id", "0_0"), mkEntry("processor-node-id", "TESTING_NODE") ) @@ -507,7 +508,7 @@ public void shouldLogAndMeterWhenSkippingLateRecordWithNonzeroGrace() { "stream-processor-node-metrics", "The average number of occurrence of late-record-drop operations.", mkMap( - mkEntry("client-id", "test"), + mkEntry("client-id", threadId), mkEntry("task-id", "0_0"), mkEntry("processor-node-id", "TESTING_NODE") ) diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamWindowAggregateTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamWindowAggregateTest.java index 61398201e5f56..9051ea69438a0 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamWindowAggregateTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamWindowAggregateTest.java @@ -65,6 +65,7 @@ public class KStreamWindowAggregateTest { private final ConsumerRecordFactory recordFactory = new ConsumerRecordFactory<>(new StringSerializer(), new StringSerializer()); private final Properties props = StreamsTestUtils.getStreamsConfig(Serdes.String(), Serdes.String()); + private final String threadId = Thread.currentThread().getName(); @Test public void testAggBasic() { @@ -394,7 +395,7 @@ private void assertLatenessMetrics(final TopologyTestDriver driver, "stream-processor-node-metrics", "The total number of occurrence of late-record-drop operations.", mkMap( - mkEntry("client-id", "topology-test-driver-virtual-thread"), + mkEntry("client-id", threadId), mkEntry("task-id", "0_0"), mkEntry("processor-node-id", "KSTREAM-AGGREGATE-0000000001") ) @@ -406,7 +407,7 @@ private void assertLatenessMetrics(final TopologyTestDriver driver, "stream-processor-node-metrics", "The average number of occurrence of late-record-drop operations.", mkMap( - mkEntry("client-id", "topology-test-driver-virtual-thread"), + mkEntry("client-id", threadId), mkEntry("task-id", "0_0"), mkEntry("processor-node-id", "KSTREAM-AGGREGATE-0000000001") ) @@ -418,7 +419,7 @@ private void assertLatenessMetrics(final TopologyTestDriver driver, "stream-task-metrics", "The max observed lateness of records.", mkMap( - mkEntry("client-id", "topology-test-driver-virtual-thread"), + mkEntry("client-id", threadId), mkEntry("task-id", "0_0") ) ); @@ -429,7 +430,7 @@ private void assertLatenessMetrics(final TopologyTestDriver driver, "stream-task-metrics", "The average observed lateness of records.", mkMap( - mkEntry("client-id", "topology-test-driver-virtual-thread"), + mkEntry("client-id", threadId), mkEntry("task-id", "0_0") ) ); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/suppress/KTableSuppressProcessorMetricsTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/suppress/KTableSuppressProcessorMetricsTest.java index ef46663309099..3fd75fffc2cc8 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/suppress/KTableSuppressProcessorMetricsTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/suppress/KTableSuppressProcessorMetricsTest.java @@ -43,90 +43,91 @@ public class KTableSuppressProcessorMetricsTest { private static final long ARBITRARY_LONG = 5L; + private final String threadId = Thread.currentThread().getName(); - private static final MetricName EVICTION_TOTAL_METRIC = new MetricName( + private final MetricName evictionTotalMetric = new MetricName( "suppression-emit-total", "stream-processor-node-metrics", "The total number of occurrence of suppression-emit operations.", mkMap( - mkEntry("client-id", "mock-processor-context-virtual-thread"), + mkEntry("client-id", threadId), mkEntry("task-id", "0_0"), mkEntry("processor-node-id", "testNode") ) ); - private static final MetricName EVICTION_RATE_METRIC = new MetricName( + private final MetricName evictionRateMetric = new MetricName( "suppression-emit-rate", "stream-processor-node-metrics", "The average number of occurrence of suppression-emit operation per second.", mkMap( - mkEntry("client-id", "mock-processor-context-virtual-thread"), + mkEntry("client-id", threadId), mkEntry("task-id", "0_0"), mkEntry("processor-node-id", "testNode") ) ); - private static final MetricName BUFFER_SIZE_AVG_METRIC = new MetricName( + private final MetricName bufferSizeAvgMetric = new MetricName( "suppression-buffer-size-avg", "stream-buffer-metrics", "The average size of buffered records.", mkMap( - mkEntry("client-id", "mock-processor-context-virtual-thread"), + mkEntry("client-id", threadId), mkEntry("task-id", "0_0"), mkEntry("buffer-id", "test-store") ) ); - private static final MetricName BUFFER_SIZE_CURRENT_METRIC = new MetricName( + private final MetricName bufferSizeCurrentMetric = new MetricName( "suppression-buffer-size-current", "stream-buffer-metrics", "The current size of buffered records.", mkMap( - mkEntry("client-id", "mock-processor-context-virtual-thread"), + mkEntry("client-id", threadId), mkEntry("task-id", "0_0"), mkEntry("buffer-id", "test-store") ) ); - private static final MetricName BUFFER_SIZE_MAX_METRIC = new MetricName( + private final MetricName bufferSizeMaxMetric = new MetricName( "suppression-buffer-size-max", "stream-buffer-metrics", "The max size of buffered records.", mkMap( - mkEntry("client-id", "mock-processor-context-virtual-thread"), + mkEntry("client-id", threadId), mkEntry("task-id", "0_0"), mkEntry("buffer-id", "test-store") ) ); - private static final MetricName BUFFER_COUNT_AVG_METRIC = new MetricName( + private final MetricName bufferCountAvgMetric = new MetricName( "suppression-buffer-count-avg", "stream-buffer-metrics", "The average count of buffered records.", mkMap( - mkEntry("client-id", "mock-processor-context-virtual-thread"), + mkEntry("client-id", threadId), mkEntry("task-id", "0_0"), mkEntry("buffer-id", "test-store") ) ); - private static final MetricName BUFFER_COUNT_CURRENT_METRIC = new MetricName( + private final MetricName bufferCountCurrentMetric = new MetricName( "suppression-buffer-count-current", "stream-buffer-metrics", "The current count of buffered records.", mkMap( - mkEntry("client-id", "mock-processor-context-virtual-thread"), + mkEntry("client-id", threadId), mkEntry("task-id", "0_0"), mkEntry("buffer-id", "test-store") ) ); - private static final MetricName BUFFER_COUNT_MAX_METRIC = new MetricName( + private final MetricName bufferCountMaxMetric = new MetricName( "suppression-buffer-count-max", "stream-buffer-metrics", "The max count of buffered records.", mkMap( - mkEntry("client-id", "mock-processor-context-virtual-thread"), + mkEntry("client-id", threadId), mkEntry("task-id", "0_0"), mkEntry("buffer-id", "test-store") ) @@ -166,14 +167,14 @@ public void shouldRecordMetrics() { { final Map metrics = context.metrics().metrics(); - verifyMetric(metrics, EVICTION_RATE_METRIC, is(0.0)); - verifyMetric(metrics, EVICTION_TOTAL_METRIC, is(0.0)); - verifyMetric(metrics, BUFFER_SIZE_AVG_METRIC, is(21.5)); - verifyMetric(metrics, BUFFER_SIZE_CURRENT_METRIC, is(43.0)); - verifyMetric(metrics, BUFFER_SIZE_MAX_METRIC, is(43.0)); - verifyMetric(metrics, BUFFER_COUNT_AVG_METRIC, is(0.5)); - verifyMetric(metrics, BUFFER_COUNT_CURRENT_METRIC, is(1.0)); - verifyMetric(metrics, BUFFER_COUNT_MAX_METRIC, is(1.0)); + verifyMetric(metrics, evictionRateMetric, is(0.0)); + verifyMetric(metrics, evictionTotalMetric, is(0.0)); + verifyMetric(metrics, bufferSizeAvgMetric, is(21.5)); + verifyMetric(metrics, bufferSizeCurrentMetric, is(43.0)); + verifyMetric(metrics, bufferSizeMaxMetric, is(43.0)); + verifyMetric(metrics, bufferCountAvgMetric, is(0.5)); + verifyMetric(metrics, bufferCountCurrentMetric, is(1.0)); + verifyMetric(metrics, bufferCountMaxMetric, is(1.0)); } context.setRecordMetadata("", 0, 1L, null, timestamp + 1); @@ -182,14 +183,14 @@ public void shouldRecordMetrics() { { final Map metrics = context.metrics().metrics(); - verifyMetric(metrics, EVICTION_RATE_METRIC, greaterThan(0.0)); - verifyMetric(metrics, EVICTION_TOTAL_METRIC, is(1.0)); - verifyMetric(metrics, BUFFER_SIZE_AVG_METRIC, is(41.0)); - verifyMetric(metrics, BUFFER_SIZE_CURRENT_METRIC, is(39.0)); - verifyMetric(metrics, BUFFER_SIZE_MAX_METRIC, is(82.0)); - verifyMetric(metrics, BUFFER_COUNT_AVG_METRIC, is(1.0)); - verifyMetric(metrics, BUFFER_COUNT_CURRENT_METRIC, is(1.0)); - verifyMetric(metrics, BUFFER_COUNT_MAX_METRIC, is(2.0)); + verifyMetric(metrics, evictionRateMetric, greaterThan(0.0)); + verifyMetric(metrics, evictionTotalMetric, is(1.0)); + verifyMetric(metrics, bufferSizeAvgMetric, is(41.0)); + verifyMetric(metrics, bufferSizeCurrentMetric, is(39.0)); + verifyMetric(metrics, bufferSizeMaxMetric, is(82.0)); + verifyMetric(metrics, bufferCountAvgMetric, is(1.0)); + verifyMetric(metrics, bufferCountCurrentMetric, is(1.0)); + verifyMetric(metrics, bufferCountMaxMetric, is(2.0)); } } diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/GlobalStreamThreadTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/GlobalStreamThreadTest.java index ce08a6ac83e21..ddb20c951ca47 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/GlobalStreamThreadTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/GlobalStreamThreadTest.java @@ -34,6 +34,7 @@ import org.apache.kafka.streams.kstream.internals.TimestampedKeyValueStoreMaterializer; import org.apache.kafka.streams.kstream.internals.MaterializedInternal; import org.apache.kafka.streams.processor.StateStore; +import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl; import org.apache.kafka.streams.state.KeyValueStore; import org.apache.kafka.test.MockStateRestoreListener; import org.apache.kafka.test.TestUtils; @@ -107,11 +108,10 @@ public String newStoreName(final String prefix) { mockConsumer, new StateDirectory(config, time, true), 0, - new Metrics(), + new StreamsMetricsImpl(new Metrics(), "test-client", StreamsConfig.METRICS_LATEST), new MockTime(), "clientId", - stateRestoreListener, - null + stateRestoreListener ); } @@ -143,11 +143,10 @@ public List partitionsFor(final String topic) { mockConsumer, new StateDirectory(config, time, true), 0, - new Metrics(), + new StreamsMetricsImpl(new Metrics(), "test-client", StreamsConfig.METRICS_LATEST), new MockTime(), "clientId", - stateRestoreListener, - null + stateRestoreListener ); try { diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorNodeTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorNodeTest.java index c3d85832d5b18..42ad94e23cb0b 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorNodeTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorNodeTest.java @@ -104,13 +104,14 @@ public void testMetrics() { final ProcessorNode node = new ProcessorNode<>("name", new NoOpProcessor(), Collections.emptySet()); node.init(context); + final String threadId = Thread.currentThread().getName(); final String[] latencyOperations = {"process", "punctuate", "create", "destroy"}; final String throughputOperation = "forward"; final String groupName = "stream-processor-node-metrics"; final Map metricTags = new LinkedHashMap<>(); metricTags.put("processor-node-id", node.name()); metricTags.put("task-id", context.taskId().toString()); - metricTags.put("client-id", "mock"); + metricTags.put("client-id", threadId); for (final String opName : latencyOperations) { StreamsTestUtils.getMetricByNameFilterByTags(metrics.metrics(), opName + "-latency-avg", groupName, metricTags); @@ -137,8 +138,8 @@ public void testMetrics() { final JmxReporter reporter = new JmxReporter("kafka.streams"); metrics.addReporter(reporter); - assertTrue(reporter.containsMbean(String.format("kafka.streams:type=%s,client-id=mock,task-id=%s,processor-node-id=%s", - groupName, context.taskId().toString(), node.name()))); + assertTrue(reporter.containsMbean(String.format("kafka.streams:type=%s,client-id=%s,task-id=%s,processor-node-id=%s", + groupName, threadId, context.taskId().toString(), node.name()))); } } diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StandbyTaskTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StandbyTaskTest.java index 6f42fb26526e6..88d4d6f46cf7c 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StandbyTaskTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StandbyTaskTest.java @@ -97,6 +97,7 @@ public class StandbyTaskTest { + private final String threadId = Thread.currentThread().getName(); private final TaskId taskId = new TaskId(0, 1); private StandbyTask task; private final Serializer intSerializer = new IntegerSerializer(); @@ -782,7 +783,7 @@ void closeStateManager(final boolean clean) throws ProcessorStateException { private MetricName setupCloseTaskMetric() { final MetricName metricName = new MetricName("name", "group", "description", Collections.emptyMap()); - final Sensor sensor = streamsMetrics.threadLevelSensor("task-closed", Sensor.RecordingLevel.INFO); + final Sensor sensor = streamsMetrics.threadLevelSensor(threadId, "task-closed", Sensor.RecordingLevel.INFO); sensor.add(metricName, new CumulativeSum()); return metricName; } diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamTaskTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamTaskTest.java index a4f529d8795f1..bda86c4423eaf 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamTaskTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamTaskTest.java @@ -414,10 +414,18 @@ public void testMetrics() { assertNotNull(getMetric("%s-latency-max", "The max latency of %s operation.", "all")); assertNotNull(getMetric("%s-rate", "The average number of occurrence of %s operation per second.", "all")); + final String threadId = Thread.currentThread().getName(); final JmxReporter reporter = new JmxReporter("kafka.streams"); metrics.addReporter(reporter); - assertTrue(reporter.containsMbean(String.format("kafka.streams:type=stream-task-metrics,client-id=test,task-id=%s", task.id.toString()))); - assertTrue(reporter.containsMbean("kafka.streams:type=stream-task-metrics,client-id=test,task-id=all")); + assertTrue(reporter.containsMbean(String.format( + "kafka.streams:type=stream-task-metrics,client-id=%s,task-id=%s", + threadId, + task.id.toString() + ))); + assertTrue(reporter.containsMbean(String.format( + "kafka.streams:type=stream-task-metrics,client-id=%s,task-id=all", + threadId + ))); } private KafkaMetric getMetric(final String nameFormat, final String descriptionFormat, final String taskId) { @@ -425,7 +433,7 @@ private KafkaMetric getMetric(final String nameFormat, final String descriptionF String.format(nameFormat, "commit"), "stream-task-metrics", String.format(descriptionFormat, "commit"), - mkMap(mkEntry("task-id", taskId), mkEntry("client-id", "test")) + mkMap(mkEntry("task-id", taskId), mkEntry("client-id", Thread.currentThread().getName())) )); } @@ -760,7 +768,11 @@ public void shouldBeProcessableIfWaitedForTooLong() { task.initializeStateStores(); task.initializeTopology(); - final MetricName enforcedProcessMetric = metrics.metricName("enforced-processing-total", "stream-task-metrics", mkMap(mkEntry("client-id", "test"), mkEntry("task-id", taskId00.toString()))); + final MetricName enforcedProcessMetric = metrics.metricName( + "enforced-processing-total", + "stream-task-metrics", + mkMap(mkEntry("client-id", Thread.currentThread().getName()), mkEntry("task-id", taskId00.toString())) + ); assertFalse(task.isProcessable(0L)); assertEquals(0.0, metrics.metric(enforcedProcessMetric).metricValue()); diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java index ef5b95b979afe..8d10e4cf2c3aa 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java @@ -101,6 +101,7 @@ import static org.hamcrest.CoreMatchers.equalTo; import static org.hamcrest.CoreMatchers.not; import static org.hamcrest.CoreMatchers.nullValue; +import static org.hamcrest.CoreMatchers.startsWith; import static org.hamcrest.MatcherAssert.assertThat; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; @@ -112,11 +113,15 @@ public class StreamThreadTest { - private final String clientId = "clientId"; - private final String applicationId = "stream-thread-test"; + private final static String APPLICATION_ID = "stream-thread-test"; + private final static UUID PROCESS_ID = UUID.fromString("87bf53a8-54f2-485f-a4b6-acdbec0a8b3d"); + private final static String CLIENT_ID = APPLICATION_ID + "-" + PROCESS_ID; + private final int threadIdx = 1; private final MockTime mockTime = new MockTime(); private final Metrics metrics = new Metrics(); + private final StreamsMetricsImpl streamsMetrics = + new StreamsMetricsImpl(metrics, APPLICATION_ID, StreamsConfig.METRICS_LATEST); private final MockClientSupplier clientSupplier = new MockClientSupplier(); private final InternalStreamsBuilder internalStreamsBuilder = new InternalStreamsBuilder(new InternalTopologyBuilder()); private final StreamsConfig config = new StreamsConfig(configProps(false)); @@ -124,16 +129,14 @@ public class StreamThreadTest { private final StateDirectory stateDirectory = new StateDirectory(config, mockTime, true); private final ConsumedInternal consumed = new ConsumedInternal<>(); - private UUID processId = UUID.randomUUID(); private InternalTopologyBuilder internalTopologyBuilder; private StreamsMetadataState streamsMetadataState; @Before public void setUp() { - processId = UUID.randomUUID(); - + Thread.currentThread().setName(CLIENT_ID + "-StreamThread-" + threadIdx); internalTopologyBuilder = InternalStreamsBuilderTest.internalTopologyBuilder(internalStreamsBuilder); - internalTopologyBuilder.setApplicationId(applicationId); + internalTopologyBuilder.setApplicationId(APPLICATION_ID); streamsMetadataState = new StreamsMetadataState(internalTopologyBuilder, StreamsMetadataState.UNKNOWN_HOST); } @@ -151,7 +154,7 @@ public void setUp() { private Properties configProps(final boolean enableEoS) { return mkProperties(mkMap( - mkEntry(StreamsConfig.APPLICATION_ID_CONFIG, applicationId), + mkEntry(StreamsConfig.APPLICATION_ID_CONFIG, APPLICATION_ID), mkEntry(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:2171"), mkEntry(StreamsConfig.BUFFERED_RECORDS_PER_PARTITION_CONFIG, "3"), mkEntry(StreamsConfig.DEFAULT_TIMESTAMP_EXTRACTOR_CLASS_CONFIG, MockTimestampExtractor.class.getName()), @@ -164,7 +167,7 @@ private Properties configProps(final boolean enableEoS) { public void testPartitionAssignmentChangeForSingleGroup() { internalTopologyBuilder.addSource(null, "source1", null, null, null, topic1); - final StreamThread thread = createStreamThread(clientId, config, false); + final StreamThread thread = createStreamThread(CLIENT_ID, config, false); final StateListenerStub stateListener = new StateListenerStub(); thread.setStateListener(stateListener); @@ -201,7 +204,7 @@ public void testPartitionAssignmentChangeForSingleGroup() { @Test public void testStateChangeStartClose() throws Exception { - final StreamThread thread = createStreamThread(clientId, config, false); + final StreamThread thread = createStreamThread(CLIENT_ID, config, false); final StateListenerStub stateListener = new StateListenerStub(); thread.setStateListener(stateListener); @@ -238,7 +241,7 @@ private StreamThread createStreamThread(@SuppressWarnings("SameParameterValue") final StreamsConfig config, final boolean eosEnabled) { if (eosEnabled) { - clientSupplier.setApplicationIdForProducer(applicationId); + clientSupplier.setApplicationIdForProducer(APPLICATION_ID); } clientSupplier.setClusterForAdminClient(createCluster()); @@ -248,9 +251,9 @@ private StreamThread createStreamThread(@SuppressWarnings("SameParameterValue") config, clientSupplier, clientSupplier.getAdmin(config.getAdminConfigs(clientId)), - processId, + PROCESS_ID, clientId, - metrics, + streamsMetrics, mockTime, streamsMetadataState, 0, @@ -262,7 +265,7 @@ private StreamThread createStreamThread(@SuppressWarnings("SameParameterValue") @Test public void testMetricsCreatedAtStartup() { - final StreamThread thread = createStreamThread(clientId, config, false); + final StreamThread thread = createStreamThread(CLIENT_ID, config, false); final String defaultGroupName = "stream-metrics"; final Map defaultTags = Collections.singletonMap("client-id", thread.getName()); final String descriptionIsNotVerified = ""; @@ -324,7 +327,7 @@ public void testMetricsCreatedAtStartup() { final JmxReporter reporter = new JmxReporter("kafka.streams"); metrics.addReporter(reporter); - assertEquals(clientId + "-StreamThread-1", thread.getName()); + assertEquals(CLIENT_ID + "-StreamThread-1", thread.getName()); assertTrue(reporter.containsMbean(String.format("kafka.streams:type=%s,client-id=%s", defaultGroupName, thread.getName()))); @@ -342,7 +345,7 @@ public void shouldNotCommitBeforeTheCommitInterval() { final Consumer consumer = EasyMock.createNiceMock(Consumer.class); final TaskManager taskManager = mockTaskManagerCommit(consumer, 1, 1); - final StreamsMetricsImpl streamsMetrics = new StreamsMetricsImpl(metrics, clientId, StreamsConfig.METRICS_LATEST); + final StreamsMetricsImpl streamsMetrics = new StreamsMetricsImpl(metrics, CLIENT_ID, StreamsConfig.METRICS_LATEST); final StreamThread thread = new StreamThread( mockTime, config, @@ -353,7 +356,7 @@ public void shouldNotCommitBeforeTheCommitInterval() { taskManager, streamsMetrics, internalTopologyBuilder, - clientId, + CLIENT_ID, new LogContext(""), new AtomicInteger() ); @@ -375,12 +378,12 @@ public void shouldRespectNumIterationsInMainLoop() { final Properties properties = new Properties(); properties.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 100L); - final StreamsConfig config = new StreamsConfig(StreamsTestUtils.getStreamsConfig(applicationId, + final StreamsConfig config = new StreamsConfig(StreamsTestUtils.getStreamsConfig(APPLICATION_ID, "localhost:2171", Serdes.ByteArraySerde.class.getName(), Serdes.ByteArraySerde.class.getName(), properties)); - final StreamThread thread = createStreamThread(clientId, config, false); + final StreamThread thread = createStreamThread(CLIENT_ID, config, false); thread.setState(StreamThread.State.STARTING); thread.setState(StreamThread.State.PARTITIONS_REVOKED); @@ -470,7 +473,7 @@ public void shouldNotCauseExceptionIfNothingCommitted() { final Consumer consumer = EasyMock.createNiceMock(Consumer.class); final TaskManager taskManager = mockTaskManagerCommit(consumer, 1, 0); - final StreamsMetricsImpl streamsMetrics = new StreamsMetricsImpl(metrics, clientId, StreamsConfig.METRICS_LATEST); + final StreamsMetricsImpl streamsMetrics = new StreamsMetricsImpl(metrics, CLIENT_ID, StreamsConfig.METRICS_LATEST); final StreamThread thread = new StreamThread( mockTime, config, @@ -481,7 +484,7 @@ public void shouldNotCauseExceptionIfNothingCommitted() { taskManager, streamsMetrics, internalTopologyBuilder, - clientId, + CLIENT_ID, new LogContext(""), new AtomicInteger() ); @@ -505,7 +508,7 @@ public void shouldCommitAfterTheCommitInterval() { final Consumer consumer = EasyMock.createNiceMock(Consumer.class); final TaskManager taskManager = mockTaskManagerCommit(consumer, 2, 1); - final StreamsMetricsImpl streamsMetrics = new StreamsMetricsImpl(metrics, clientId, StreamsConfig.METRICS_LATEST); + final StreamsMetricsImpl streamsMetrics = new StreamsMetricsImpl(metrics, CLIENT_ID, StreamsConfig.METRICS_LATEST); final StreamThread thread = new StreamThread( mockTime, config, @@ -516,7 +519,7 @@ public void shouldCommitAfterTheCommitInterval() { taskManager, streamsMetrics, internalTopologyBuilder, - clientId, + CLIENT_ID, new LogContext(""), new AtomicInteger() ); @@ -543,7 +546,7 @@ public void shouldInjectSharedProducerForAllTasksUsingClientSupplierOnCreateIfEo internalTopologyBuilder.addSource(null, "source1", null, null, null, topic1); internalStreamsBuilder.buildAndOptimizeTopology(); - final StreamThread thread = createStreamThread(clientId, config, false); + final StreamThread thread = createStreamThread(CLIENT_ID, config, false); thread.setState(StreamThread.State.STARTING); thread.rebalanceListener.onPartitionsRevoked(Collections.emptyList()); @@ -580,7 +583,7 @@ public void shouldInjectSharedProducerForAllTasksUsingClientSupplierOnCreateIfEo public void shouldInjectProducerPerTaskUsingClientSupplierOnCreateIfEosEnable() { internalTopologyBuilder.addSource(null, "source1", null, null, null, topic1); - final StreamThread thread = createStreamThread(clientId, new StreamsConfig(configProps(true)), true); + final StreamThread thread = createStreamThread(CLIENT_ID, new StreamsConfig(configProps(true)), true); thread.setState(StreamThread.State.STARTING); thread.rebalanceListener.onPartitionsRevoked(Collections.emptyList()); @@ -619,7 +622,7 @@ public void shouldInjectProducerPerTaskUsingClientSupplierOnCreateIfEosEnable() public void shouldCloseAllTaskProducersOnCloseIfEosEnabled() { internalTopologyBuilder.addSource(null, "source1", null, null, null, topic1); - final StreamThread thread = createStreamThread(clientId, new StreamsConfig(configProps(true)), true); + final StreamThread thread = createStreamThread(CLIENT_ID, new StreamsConfig(configProps(true)), true); thread.setState(StreamThread.State.STARTING); thread.rebalanceListener.onPartitionsRevoked(Collections.emptyList()); @@ -660,7 +663,7 @@ public void shouldShutdownTaskManagerOnClose() { EasyMock.replay(taskManager, consumer); final StreamsMetricsImpl streamsMetrics = - new StreamsMetricsImpl(metrics, clientId, StreamsConfig.METRICS_LATEST); + new StreamsMetricsImpl(metrics, CLIENT_ID, StreamsConfig.METRICS_LATEST); final StreamThread thread = new StreamThread( mockTime, config, @@ -671,10 +674,10 @@ public void shouldShutdownTaskManagerOnClose() { taskManager, streamsMetrics, internalTopologyBuilder, - clientId, + CLIENT_ID, new LogContext(""), new AtomicInteger() - ).updateThreadMetadata(getSharedAdminClientId(clientId)); + ).updateThreadMetadata(getSharedAdminClientId(CLIENT_ID)); thread.setStateListener( (t, newState, oldState) -> { if (oldState == StreamThread.State.CREATED && newState == StreamThread.State.STARTING) { @@ -694,7 +697,7 @@ public void shouldShutdownTaskManagerOnCloseWithoutStart() { EasyMock.replay(taskManager, consumer); final StreamsMetricsImpl streamsMetrics = - new StreamsMetricsImpl(metrics, clientId, StreamsConfig.METRICS_LATEST); + new StreamsMetricsImpl(metrics, CLIENT_ID, StreamsConfig.METRICS_LATEST); final StreamThread thread = new StreamThread( mockTime, config, @@ -705,10 +708,10 @@ public void shouldShutdownTaskManagerOnCloseWithoutStart() { taskManager, streamsMetrics, internalTopologyBuilder, - clientId, + CLIENT_ID, new LogContext(""), new AtomicInteger() - ).updateThreadMetadata(getSharedAdminClientId(clientId)); + ).updateThreadMetadata(getSharedAdminClientId(CLIENT_ID)); thread.shutdown(); EasyMock.verify(taskManager); } @@ -755,22 +758,24 @@ private void setStreamThread(final StreamThread streamThread) { final MockStreamThreadConsumer mockStreamThreadConsumer = new MockStreamThreadConsumer<>(OffsetResetStrategy.EARLIEST); - final TaskManager taskManager = new TaskManager(new MockChangelogReader(), - processId, - "log-prefix", - mockStreamThreadConsumer, - streamsMetadataState, - null, - null, - null, - new AssignedStreamsTasks(new LogContext()), - new AssignedStandbyTasks(new LogContext())); + final TaskManager taskManager = new TaskManager( + new MockChangelogReader(), + PROCESS_ID, + "log-prefix", + mockStreamThreadConsumer, + streamsMetadataState, + null, + null, + null, + new AssignedStreamsTasks(new LogContext()), + new AssignedStandbyTasks(new LogContext()) + ); taskManager.setConsumer(mockStreamThreadConsumer); taskManager.setAssignmentMetadata(Collections.emptyMap(), Collections.emptyMap()); taskManager.setPartitionsToTaskId(Collections.emptyMap()); final StreamsMetricsImpl streamsMetrics = - new StreamsMetricsImpl(metrics, clientId, StreamsConfig.METRICS_LATEST); + new StreamsMetricsImpl(metrics, CLIENT_ID, StreamsConfig.METRICS_LATEST); final StreamThread thread = new StreamThread( mockTime, config, @@ -781,10 +786,10 @@ private void setStreamThread(final StreamThread streamThread) { taskManager, streamsMetrics, internalTopologyBuilder, - clientId, + CLIENT_ID, new LogContext(""), new AtomicInteger() - ).updateThreadMetadata(getSharedAdminClientId(clientId)); + ).updateThreadMetadata(getSharedAdminClientId(CLIENT_ID)); mockStreamThreadConsumer.setStreamThread(thread); mockStreamThreadConsumer.assign(assignedPartitions); @@ -805,7 +810,7 @@ public void shouldOnlyShutdownOnce() { EasyMock.replay(taskManager, consumer); final StreamsMetricsImpl streamsMetrics = - new StreamsMetricsImpl(metrics, clientId, StreamsConfig.METRICS_LATEST); + new StreamsMetricsImpl(metrics, CLIENT_ID, StreamsConfig.METRICS_LATEST); final StreamThread thread = new StreamThread( mockTime, config, @@ -816,10 +821,10 @@ public void shouldOnlyShutdownOnce() { taskManager, streamsMetrics, internalTopologyBuilder, - clientId, + CLIENT_ID, new LogContext(""), new AtomicInteger() - ).updateThreadMetadata(getSharedAdminClientId(clientId)); + ).updateThreadMetadata(getSharedAdminClientId(CLIENT_ID)); thread.shutdown(); // Execute the run method. Verification of the mock will check that shutdown was only done once thread.run(); @@ -831,7 +836,7 @@ public void shouldNotNullPointerWhenStandbyTasksAssignedAndNoStateStoresForTopol internalTopologyBuilder.addSource(null, "name", null, null, null, "topic"); internalTopologyBuilder.addSink("out", "output", null, null, null, "name"); - final StreamThread thread = createStreamThread(clientId, config, false); + final StreamThread thread = createStreamThread(CLIENT_ID, config, false); thread.setState(StreamThread.State.STARTING); thread.rebalanceListener.onPartitionsRevoked(Collections.emptyList()); @@ -852,7 +857,7 @@ public void shouldCloseTaskAsZombieAndRemoveFromActiveTasksIfProducerWasFencedWh internalTopologyBuilder.addSource(null, "source", null, null, null, topic1); internalTopologyBuilder.addSink("sink", "dummyTopic", null, null, null, "source"); - final StreamThread thread = createStreamThread(clientId, new StreamsConfig(configProps(true)), true); + final StreamThread thread = createStreamThread(CLIENT_ID, new StreamsConfig(configProps(true)), true); final MockConsumer consumer = clientSupplier.consumer; @@ -914,7 +919,7 @@ public void shouldCloseTaskAsZombieAndRemoveFromActiveTasksIfProducerWasFencedWh @Test public void shouldCloseTaskAsZombieAndRemoveFromActiveTasksIfProducerGotFencedInCommitTransactionWhenSuspendingTaks() { - final StreamThread thread = createStreamThread(clientId, new StreamsConfig(configProps(true)), true); + final StreamThread thread = createStreamThread(CLIENT_ID, new StreamsConfig(configProps(true)), true); internalTopologyBuilder.addSource(null, "name", null, null, null, topic1); internalTopologyBuilder.addSink("out", "output", null, null, null, "name"); @@ -953,7 +958,7 @@ public void shouldCloseTaskAsZombieAndRemoveFromActiveTasksIfProducerGotFencedIn @Test public void shouldCloseTaskAsZombieAndRemoveFromActiveTasksIfProducerGotFencedInCloseTransactionWhenSuspendingTasks() { - final StreamThread thread = createStreamThread(clientId, new StreamsConfig(configProps(true)), true); + final StreamThread thread = createStreamThread(CLIENT_ID, new StreamsConfig(configProps(true)), true); internalTopologyBuilder.addSource(null, "name", null, null, null, topic1); internalTopologyBuilder.addSink("out", "output", null, null, null, "name"); @@ -1015,7 +1020,7 @@ public void onChange(final Thread thread, public void shouldReturnActiveTaskMetadataWhileRunningState() { internalTopologyBuilder.addSource(null, "source", null, null, null, topic1); - final StreamThread thread = createStreamThread(clientId, config, false); + final StreamThread thread = createStreamThread(CLIENT_ID, config, false); thread.setState(StreamThread.State.STARTING); thread.rebalanceListener.onPartitionsRevoked(Collections.emptySet()); @@ -1047,11 +1052,11 @@ public void shouldReturnActiveTaskMetadataWhileRunningState() { assertTrue("#threadState() was: " + metadata.threadState() + "; expected either RUNNING, STARTING, PARTITIONS_REVOKED, PARTITIONS_ASSIGNED, or CREATED", Arrays.asList("RUNNING", "STARTING", "PARTITIONS_REVOKED", "PARTITIONS_ASSIGNED", "CREATED").contains(metadata.threadState())); final String threadName = metadata.threadName(); - assertTrue(threadName.startsWith("clientId-StreamThread-")); + assertThat(threadName, startsWith(CLIENT_ID + "-StreamThread-" + threadIdx)); assertEquals(threadName + "-consumer", metadata.consumerClientId()); assertEquals(threadName + "-restore-consumer", metadata.restoreConsumerClientId()); assertEquals(Collections.singleton(threadName + "-producer"), metadata.producerClientIds()); - assertEquals("clientId-admin", metadata.adminClientId()); + assertEquals(CLIENT_ID + "-admin", metadata.adminClientId()); } @Test @@ -1060,7 +1065,7 @@ public void shouldReturnStandbyTaskMetadataWhileRunningState() { .groupByKey().count(Materialized.as("count-one")); internalStreamsBuilder.buildAndOptimizeTopology(); - final StreamThread thread = createStreamThread(clientId, config, false); + final StreamThread thread = createStreamThread(CLIENT_ID, config, false); final MockConsumer restoreConsumer = clientSupplier.restoreConsumer; restoreConsumer.updatePartitions( "stream-thread-test-count-one-changelog", @@ -1106,8 +1111,8 @@ public void shouldReturnStandbyTaskMetadataWhileRunningState() { public void shouldUpdateStandbyTask() throws Exception { final String storeName1 = "count-one"; final String storeName2 = "table-two"; - final String changelogName1 = applicationId + "-" + storeName1 + "-changelog"; - final String changelogName2 = applicationId + "-" + storeName2 + "-changelog"; + final String changelogName1 = APPLICATION_ID + "-" + storeName1 + "-changelog"; + final String changelogName2 = APPLICATION_ID + "-" + storeName2 + "-changelog"; final TopicPartition partition1 = new TopicPartition(changelogName1, 1); final TopicPartition partition2 = new TopicPartition(changelogName2, 1); internalStreamsBuilder @@ -1119,7 +1124,7 @@ public void shouldUpdateStandbyTask() throws Exception { internalStreamsBuilder.table(topic2, new ConsumedInternal<>(), materialized); internalStreamsBuilder.buildAndOptimizeTopology(); - final StreamThread thread = createStreamThread(clientId, config, false); + final StreamThread thread = createStreamThread(CLIENT_ID, config, false); final MockConsumer restoreConsumer = clientSupplier.restoreConsumer; restoreConsumer.updatePartitions(changelogName1, singletonList( @@ -1227,7 +1232,7 @@ private StandbyTask createStandbyTask() { final LogContext logContext = new LogContext("test"); final Logger log = logContext.logger(StreamThreadTest.class); final StreamsMetricsImpl streamsMetrics = - new StreamsMetricsImpl(metrics, clientId, StreamsConfig.METRICS_LATEST); + new StreamsMetricsImpl(metrics, CLIENT_ID, StreamsConfig.METRICS_LATEST); final StreamThread.StandbyTaskCreator standbyTaskCreator = new StreamThread.StandbyTaskCreator( internalTopologyBuilder, config, @@ -1235,6 +1240,7 @@ private StandbyTask createStandbyTask() { stateDirectory, new MockChangelogReader(), mockTime, + CLIENT_ID, log); return standbyTaskCreator.createTask( new MockConsumer<>(OffsetResetStrategy.EARLIEST), @@ -1264,7 +1270,7 @@ public void close() {} internalStreamsBuilder.stream(Collections.singleton(topic1), consumed).process(punctuateProcessor); internalStreamsBuilder.buildAndOptimizeTopology(); - final StreamThread thread = createStreamThread(clientId, config, false); + final StreamThread thread = createStreamThread(CLIENT_ID, config, false); thread.setState(StreamThread.State.STARTING); thread.rebalanceListener.onPartitionsRevoked(Collections.emptySet()); @@ -1321,7 +1327,7 @@ public void close() {} @Test public void shouldAlwaysUpdateTasksMetadataAfterChangingState() { - final StreamThread thread = createStreamThread(clientId, config, false); + final StreamThread thread = createStreamThread(CLIENT_ID, config, false); ThreadMetadata metadata = thread.threadMetadata(); assertEquals(StreamThread.State.CREATED.name(), metadata.threadState()); @@ -1339,7 +1345,7 @@ public void shouldAlwaysReturnEmptyTasksMetadataWhileRebalancingStateAndTasksNot .groupByKey().count(Materialized.as("count-one")); internalStreamsBuilder.buildAndOptimizeTopology(); - final StreamThread thread = createStreamThread(clientId, config, false); + final StreamThread thread = createStreamThread(CLIENT_ID, config, false); final MockConsumer restoreConsumer = clientSupplier.restoreConsumer; restoreConsumer.updatePartitions("stream-thread-test-count-one-changelog", asList( @@ -1500,7 +1506,7 @@ public void shouldRecordSkippedMetricForDeserializationException() { StreamsConfig.DEFAULT_DESERIALIZATION_EXCEPTION_HANDLER_CLASS_CONFIG, LogAndContinueExceptionHandler.class.getName()); config.setProperty(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.Integer().getClass().getName()); - final StreamThread thread = createStreamThread(clientId, new StreamsConfig(config), false); + final StreamThread thread = createStreamThread(CLIENT_ID, new StreamsConfig(config), false); thread.setState(StreamThread.State.STARTING); thread.setState(StreamThread.State.PARTITIONS_REVOKED); @@ -1571,7 +1577,7 @@ public void shouldReportSkippedRecordsForInvalidTimestamps() { config.setProperty( StreamsConfig.DEFAULT_TIMESTAMP_EXTRACTOR_CLASS_CONFIG, LogAndSkipOnInvalidTimestamp.class.getName()); - final StreamThread thread = createStreamThread(clientId, new StreamsConfig(config), false); + final StreamThread thread = createStreamThread(CLIENT_ID, new StreamsConfig(config), false); thread.setState(StreamThread.State.STARTING); thread.setState(StreamThread.State.PARTITIONS_REVOKED); @@ -1672,21 +1678,21 @@ public void producerMetricsVerificationWithoutEOS() { final TaskManager taskManager = mockTaskManagerCommit(consumer, 1, 0); final StreamsMetricsImpl streamsMetrics = - new StreamsMetricsImpl(metrics, clientId, StreamsConfig.METRICS_LATEST); + new StreamsMetricsImpl(metrics, CLIENT_ID, StreamsConfig.METRICS_LATEST); final StreamThread thread = new StreamThread( - mockTime, - config, - producer, - consumer, - consumer, - null, - taskManager, - streamsMetrics, - internalTopologyBuilder, - clientId, - new LogContext(""), - new AtomicInteger() - ); + mockTime, + config, + producer, + consumer, + consumer, + null, + taskManager, + streamsMetrics, + internalTopologyBuilder, + CLIENT_ID, + new LogContext(""), + new AtomicInteger() + ); final MetricName testMetricName = new MetricName("test_metric", "", "", new HashMap<>()); final Metric testMetric = new KafkaMetric( new Object(), @@ -1711,21 +1717,21 @@ public void adminClientMetricsVerification() { final Consumer consumer = EasyMock.createNiceMock(Consumer.class); final TaskManager taskManager = EasyMock.createNiceMock(TaskManager.class); - final StreamsMetricsImpl streamsMetrics = new StreamsMetricsImpl(metrics, clientId, StreamsConfig.METRICS_LATEST); + final StreamsMetricsImpl streamsMetrics = new StreamsMetricsImpl(metrics, CLIENT_ID, StreamsConfig.METRICS_LATEST); final StreamThread thread = new StreamThread( - mockTime, - config, - producer, - consumer, - consumer, - null, - taskManager, - streamsMetrics, - internalTopologyBuilder, - clientId, - new LogContext(""), - new AtomicInteger() - ); + mockTime, + config, + producer, + consumer, + consumer, + null, + taskManager, + streamsMetrics, + internalTopologyBuilder, + CLIENT_ID, + new LogContext(""), + new AtomicInteger() + ); final MetricName testMetricName = new MetricName("test_metric", "", "", new HashMap<>()); final Metric testMetric = new KafkaMetric( new Object(), diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/metrics/StreamsMetricsImplTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/metrics/StreamsMetricsImplTest.java index b589119766166..69f7546d0e89e 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/metrics/StreamsMetricsImplTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/metrics/StreamsMetricsImplTest.java @@ -17,6 +17,7 @@ package org.apache.kafka.streams.processor.internals.metrics; 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.MetricConfig; import org.apache.kafka.common.metrics.Metrics; @@ -24,9 +25,11 @@ import org.apache.kafka.common.metrics.Sensor.RecordingLevel; import org.apache.kafka.common.utils.MockTime; import org.apache.kafka.streams.StreamsConfig; +import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl.ImmutableMetricValue; import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl.Version; import org.easymock.EasyMock; import org.easymock.EasyMockSupport; +import org.easymock.IArgumentMatcher; import org.junit.Test; import java.time.Duration; @@ -36,54 +39,222 @@ import static org.apache.kafka.common.utils.Utils.mkEntry; import static org.apache.kafka.common.utils.Utils.mkMap; +import static org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl.CLIENT_LEVEL_GROUP; import static org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl.PROCESSOR_NODE_METRICS_GROUP; import static org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl.addAvgAndMaxLatencyToSensor; import static org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl.addInvocationRateAndCountToSensor; +import static org.easymock.EasyMock.anyObject; +import static org.easymock.EasyMock.anyString; +import static org.easymock.EasyMock.eq; +import static org.easymock.EasyMock.expect; +import static org.easymock.EasyMock.replay; +import static org.easymock.EasyMock.resetToDefault; +import static org.easymock.EasyMock.verify; +import static org.hamcrest.CoreMatchers.equalToObject; import static org.hamcrest.CoreMatchers.is; import static org.hamcrest.CoreMatchers.notNullValue; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.greaterThan; import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNull; public class StreamsMetricsImplTest extends EasyMockSupport { private final static String SENSOR_PREFIX_DELIMITER = "."; private final static String SENSOR_NAME_DELIMITER = ".s."; private final static String INTERNAL_PREFIX = "internal"; - private final static String THREAD_NAME = "test-thread"; private final static String VERSION = StreamsConfig.METRICS_LATEST; + private final static String CLIENT_ID = "test-client"; + private final static String THREAD_ID = "test-thread"; + private final static String TASK_ID = "test-task"; + private final static String METRIC_NAME1 = "test-metric1"; + private final static String METRIC_NAME2 = "test-metric2"; private final Metrics metrics = new Metrics(); private final Sensor sensor = metrics.sensor("dummy"); + private final String storeName = "store"; + private final String sensorName1 = "sensor1"; + private final String sensorName2 = "sensor2"; private final String metricNamePrefix = "metric"; private final String group = "group"; private final Map tags = mkMap(mkEntry("tag", "value")); private final String description1 = "description number one"; private final String description2 = "description number two"; private final String description3 = "description number three"; + private final Map clientLevelTags = mkMap(mkEntry("client-id", CLIENT_ID)); + private final MetricName metricName1 = + new MetricName(METRIC_NAME1, CLIENT_LEVEL_GROUP, description1, clientLevelTags); + private final MetricName metricName2 = + new MetricName(METRIC_NAME1, CLIENT_LEVEL_GROUP, description2, clientLevelTags); private final MockTime time = new MockTime(0); - private final StreamsMetricsImpl streamsMetrics = new StreamsMetricsImpl(metrics, THREAD_NAME, VERSION); + private final StreamsMetricsImpl streamsMetrics = new StreamsMetricsImpl(metrics, CLIENT_ID, VERSION); + + private static MetricConfig eqMetricConfig(final MetricConfig metricConfig) { + EasyMock.reportMatcher(new IArgumentMatcher() { + private final StringBuffer message = new StringBuffer(); + + @Override + public boolean matches(final Object argument) { + if (argument instanceof MetricConfig) { + final MetricConfig otherMetricConfig = (MetricConfig) argument; + final boolean equalsComparisons = + (otherMetricConfig.quota() == metricConfig.quota() || + otherMetricConfig.quota().equals(metricConfig.quota())) && + otherMetricConfig.tags().equals(metricConfig.tags()); + if (otherMetricConfig.eventWindow() == metricConfig.eventWindow() && + otherMetricConfig.recordLevel() == metricConfig.recordLevel() && + equalsComparisons && + otherMetricConfig.samples() == metricConfig.samples() && + otherMetricConfig.timeWindowMs() == metricConfig.timeWindowMs()) { + + return true; + } else { + message.append("{ "); + message.append("eventWindow="); + message.append(otherMetricConfig.eventWindow()); + message.append(", "); + message.append("recordLevel="); + message.append(otherMetricConfig.recordLevel()); + message.append(", "); + message.append("quota="); + message.append(otherMetricConfig.quota().toString()); + message.append(", "); + message.append("samples="); + message.append(otherMetricConfig.samples()); + message.append(", "); + message.append("tags="); + message.append(otherMetricConfig.tags().toString()); + message.append(", "); + message.append("timeWindowMs="); + message.append(otherMetricConfig.timeWindowMs()); + message.append(" }"); + } + } + message.append("not a MetricConfig object"); + return false; + } + + @Override + public void appendTo(final StringBuffer buffer) { + buffer.append(message); + } + }); + return null; + } + + + private void addSensorsOnAllLevels(final Metrics metrics, final StreamsMetricsImpl streamsMetrics) { + expect(metrics.sensor(anyString(), anyObject(RecordingLevel.class), anyObject(Sensor[].class))) + .andStubReturn(sensor); + expect(metrics.metricName(METRIC_NAME1, CLIENT_LEVEL_GROUP, description1, clientLevelTags)) + .andReturn(metricName1); + expect(metrics.metricName(METRIC_NAME2, CLIENT_LEVEL_GROUP, description2, clientLevelTags)) + .andReturn(metricName2); + replay(metrics); + streamsMetrics.addClientLevelImmutableMetric(METRIC_NAME1, description1, RecordingLevel.INFO, "value"); + streamsMetrics.addClientLevelImmutableMetric(METRIC_NAME2, description2, RecordingLevel.INFO, "value"); + streamsMetrics.threadLevelSensor(THREAD_ID, sensorName1, RecordingLevel.INFO); + streamsMetrics.threadLevelSensor(THREAD_ID, sensorName2, RecordingLevel.INFO); + streamsMetrics.taskLevelSensor(THREAD_ID, TASK_ID, sensorName1, RecordingLevel.INFO); + streamsMetrics.taskLevelSensor(THREAD_ID, TASK_ID, sensorName2, RecordingLevel.INFO); + streamsMetrics.storeLevelSensor(THREAD_ID, TASK_ID, storeName, sensorName1, RecordingLevel.INFO); + streamsMetrics.storeLevelSensor(THREAD_ID, TASK_ID, storeName, sensorName2, RecordingLevel.INFO); + } + + private void setupGetSensorTest(final Metrics metrics, + final String level, + final RecordingLevel recordingLevel) { + final String fullSensorName = + INTERNAL_PREFIX + SENSOR_PREFIX_DELIMITER + level + SENSOR_NAME_DELIMITER + sensorName1; + final Sensor[] parents = {}; + expect(metrics.sensor(fullSensorName, recordingLevel, parents)).andReturn(sensor); + replay(metrics); + } + + @Test + public void shouldAddClientLevelImmutableMetric() { + final Metrics metrics = mock(Metrics.class); + final RecordingLevel recordingLevel = RecordingLevel.INFO; + final MetricConfig metricConfig = new MetricConfig().recordLevel(recordingLevel); + final String value = "immutable-value"; + final ImmutableMetricValue immutableValue = new ImmutableMetricValue<>(value); + expect(metrics.metricName(METRIC_NAME1, CLIENT_LEVEL_GROUP, description1, clientLevelTags)) + .andReturn(metricName1); + metrics.addMetric(eq(metricName1), eqMetricConfig(metricConfig), eq(immutableValue)); + replay(metrics); + final StreamsMetricsImpl streamsMetrics = new StreamsMetricsImpl(metrics, CLIENT_ID, VERSION); + + streamsMetrics.addClientLevelImmutableMetric(METRIC_NAME1, description1, recordingLevel, value); + + verify(metrics); + } + + @Test + public void shouldAddClientLevelMutableMetric() { + final Metrics metrics = mock(Metrics.class); + final RecordingLevel recordingLevel = RecordingLevel.INFO; + final MetricConfig metricConfig = new MetricConfig().recordLevel(recordingLevel); + final Gauge valueProvider = (config, now) -> "mutable-value"; + expect(metrics.metricName(METRIC_NAME1, CLIENT_LEVEL_GROUP, description1, clientLevelTags)) + .andReturn(metricName1); + metrics.addMetric(eq(metricName1), eqMetricConfig(metricConfig), eq(valueProvider)); + replay(metrics); + final StreamsMetricsImpl streamsMetrics = new StreamsMetricsImpl(metrics, CLIENT_ID, VERSION); + + streamsMetrics.addClientLevelMutableMetric(METRIC_NAME1, description1, recordingLevel, valueProvider); + + verify(metrics); + } @Test public void shouldGetThreadLevelSensor() { final Metrics metrics = mock(Metrics.class); - final StreamsMetricsImpl streamsMetrics = new StreamsMetricsImpl(metrics, THREAD_NAME, VERSION); - final String sensorName = "sensor1"; - final String expectedFullSensorName = - INTERNAL_PREFIX + SENSOR_PREFIX_DELIMITER + THREAD_NAME + SENSOR_NAME_DELIMITER + sensorName; - final RecordingLevel recordingLevel = RecordingLevel.DEBUG; + final RecordingLevel recordingLevel = RecordingLevel.INFO; + setupGetSensorTest(metrics, THREAD_ID, recordingLevel); + final StreamsMetricsImpl streamsMetrics = new StreamsMetricsImpl(metrics, CLIENT_ID, VERSION); + + final Sensor actualSensor = streamsMetrics.threadLevelSensor(THREAD_ID, sensorName1, recordingLevel); + + verify(metrics); + assertThat(actualSensor, is(equalToObject(sensor))); + } + + private void setupRemoveSensorsTest(final Metrics metrics, + final String level, + final RecordingLevel recordingLevel) { + final String fullSensorNamePrefix = INTERNAL_PREFIX + SENSOR_PREFIX_DELIMITER + level + SENSOR_NAME_DELIMITER; final Sensor[] parents = {}; - EasyMock.expect(metrics.sensor(expectedFullSensorName, recordingLevel, parents)).andReturn(null); + resetToDefault(metrics); + metrics.removeSensor(fullSensorNamePrefix + sensorName1); + metrics.removeSensor(fullSensorNamePrefix + sensorName2); + replay(metrics); + } - replayAll(); + @Test + public void shouldRemoveClientLevelMetrics() { + final Metrics metrics = niceMock(Metrics.class); + final StreamsMetricsImpl streamsMetrics = new StreamsMetricsImpl(metrics, CLIENT_ID, VERSION); + addSensorsOnAllLevels(metrics, streamsMetrics); + resetToDefault(metrics); + expect(metrics.removeMetric(metricName1)).andStubReturn(null); + expect(metrics.removeMetric(metricName2)).andStubReturn(null); + replay(metrics); + + streamsMetrics.removeAllClientLevelMetrics(); + + verify(metrics); + } - final Sensor sensor = streamsMetrics.threadLevelSensor(sensorName, recordingLevel); + @Test + public void shouldRemoveThreadLevelSensors() { + final Metrics metrics = niceMock(Metrics.class); + final StreamsMetricsImpl streamsMetrics = new StreamsMetricsImpl(metrics, CLIENT_ID, VERSION); + addSensorsOnAllLevels(metrics, streamsMetrics); + setupRemoveSensorsTest(metrics, THREAD_ID, RecordingLevel.INFO); - verifyAll(); + streamsMetrics.removeAllThreadLevelSensors(THREAD_ID); - assertNull(sensor); + verify(metrics); } @Test(expected = NullPointerException.class) @@ -121,7 +292,7 @@ public void testRemoveSensor() { @Test public void testMultiLevelSensorRemoval() { final Metrics registry = new Metrics(); - final StreamsMetricsImpl metrics = new StreamsMetricsImpl(registry, THREAD_NAME, VERSION); + final StreamsMetricsImpl metrics = new StreamsMetricsImpl(registry, THREAD_ID, VERSION); for (final MetricName defaultMetric : registry.metrics().keySet()) { registry.removeMetric(defaultMetric); } @@ -133,39 +304,39 @@ public void testMultiLevelSensorRemoval() { final String processorNodeName = "processorNodeName"; final Map nodeTags = mkMap(mkEntry("nkey", "value")); - final Sensor parent1 = metrics.taskLevelSensor(taskName, operation, Sensor.RecordingLevel.DEBUG); + final Sensor parent1 = metrics.taskLevelSensor(THREAD_ID, taskName, operation, Sensor.RecordingLevel.DEBUG); addAvgAndMaxLatencyToSensor(parent1, PROCESSOR_NODE_METRICS_GROUP, taskTags, operation); addInvocationRateAndCountToSensor(parent1, PROCESSOR_NODE_METRICS_GROUP, taskTags, operation, "", ""); final int numberOfTaskMetrics = registry.metrics().size(); - final Sensor sensor1 = metrics.nodeLevelSensor(taskName, processorNodeName, operation, Sensor.RecordingLevel.DEBUG, parent1); + final Sensor sensor1 = metrics.nodeLevelSensor(THREAD_ID, taskName, processorNodeName, operation, Sensor.RecordingLevel.DEBUG, parent1); addAvgAndMaxLatencyToSensor(sensor1, PROCESSOR_NODE_METRICS_GROUP, nodeTags, operation); addInvocationRateAndCountToSensor(sensor1, PROCESSOR_NODE_METRICS_GROUP, nodeTags, operation, "", ""); assertThat(registry.metrics().size(), greaterThan(numberOfTaskMetrics)); - metrics.removeAllNodeLevelSensors(taskName, processorNodeName); + metrics.removeAllNodeLevelSensors(THREAD_ID, taskName, processorNodeName); assertThat(registry.metrics().size(), equalTo(numberOfTaskMetrics)); - final Sensor parent2 = metrics.taskLevelSensor(taskName, operation, Sensor.RecordingLevel.DEBUG); + final Sensor parent2 = metrics.taskLevelSensor(THREAD_ID, taskName, operation, Sensor.RecordingLevel.DEBUG); addAvgAndMaxLatencyToSensor(parent2, PROCESSOR_NODE_METRICS_GROUP, taskTags, operation); addInvocationRateAndCountToSensor(parent2, PROCESSOR_NODE_METRICS_GROUP, taskTags, operation, "", ""); assertThat(registry.metrics().size(), equalTo(numberOfTaskMetrics)); - final Sensor sensor2 = metrics.nodeLevelSensor(taskName, processorNodeName, operation, Sensor.RecordingLevel.DEBUG, parent2); + final Sensor sensor2 = metrics.nodeLevelSensor(THREAD_ID, taskName, processorNodeName, operation, Sensor.RecordingLevel.DEBUG, parent2); addAvgAndMaxLatencyToSensor(sensor2, PROCESSOR_NODE_METRICS_GROUP, nodeTags, operation); addInvocationRateAndCountToSensor(sensor2, PROCESSOR_NODE_METRICS_GROUP, nodeTags, operation, "", ""); assertThat(registry.metrics().size(), greaterThan(numberOfTaskMetrics)); - metrics.removeAllNodeLevelSensors(taskName, processorNodeName); + metrics.removeAllNodeLevelSensors(THREAD_ID, taskName, processorNodeName); assertThat(registry.metrics().size(), equalTo(numberOfTaskMetrics)); - metrics.removeAllTaskLevelSensors(taskName); + metrics.removeAllTaskLevelSensors(THREAD_ID, taskName); assertThat(registry.metrics().size(), equalTo(0)); } @@ -231,7 +402,7 @@ public void testTotalMetricDoesntDecrease() { "stream-scope-metrics", "", "client-id", - "", + Thread.currentThread().getName(), "scope-id", "entity" ); @@ -244,16 +415,24 @@ public void testTotalMetricDoesntDecrease() { } } + @Test + public void shouldGetClientLevelTagMap() { + final Map tagMap = streamsMetrics.clientLevelTagMap(); + + assertThat(tagMap.size(), equalTo(1)); + assertThat(tagMap.get(StreamsMetricsImpl.CLIENT_ID_TAG), equalTo(CLIENT_ID)); + } + @Test public void shouldGetStoreLevelTagMap() { final String taskName = "test-task"; final String storeType = "remote-window"; final String storeName = "window-keeper"; - final Map tagMap = streamsMetrics.storeLevelTagMap(taskName, storeType, storeName); + final Map tagMap = streamsMetrics.storeLevelTagMap(THREAD_ID, taskName, storeType, storeName); assertThat(tagMap.size(), equalTo(3)); - assertThat(tagMap.get(StreamsMetricsImpl.THREAD_ID_TAG_0100_TO_23), equalTo(THREAD_NAME)); + assertThat(tagMap.get(StreamsMetricsImpl.THREAD_ID_TAG_0100_TO_23), equalTo(THREAD_ID)); assertThat(tagMap.get(StreamsMetricsImpl.TASK_ID_TAG), equalTo(taskName)); assertThat(tagMap.get(storeType + "-" + StreamsMetricsImpl.STORE_ID_TAG), equalTo(storeName)); } @@ -270,18 +449,18 @@ public void shouldGetCacheLevelTagMapForBuiltInMetricsVersion0100To23() { private void shouldGetCacheLevelTagMap(final String builtInMetricsVersion) { final StreamsMetricsImpl streamsMetrics = - new StreamsMetricsImpl(metrics, THREAD_NAME, builtInMetricsVersion); + new StreamsMetricsImpl(metrics, THREAD_ID, builtInMetricsVersion); final String taskName = "taskName"; final String storeName = "storeName"; - final Map tagMap = streamsMetrics.cacheLevelTagMap(taskName, storeName); + final Map tagMap = streamsMetrics.cacheLevelTagMap(THREAD_ID, taskName, storeName); assertThat(tagMap.size(), equalTo(3)); assertThat( tagMap.get( builtInMetricsVersion.equals(StreamsConfig.METRICS_LATEST) ? StreamsMetricsImpl.THREAD_ID_TAG : StreamsMetricsImpl.THREAD_ID_TAG_0100_TO_23), - equalTo(Thread.currentThread().getName()) + equalTo(THREAD_ID) ); assertThat(tagMap.get(StreamsMetricsImpl.TASK_ID_TAG), equalTo(taskName)); assertThat(tagMap.get(StreamsMetricsImpl.RECORD_CACHE_ID_TAG), equalTo(storeName)); @@ -372,7 +551,7 @@ public void shouldAddAvgAndMinAndMaxMetricsToSensor() { @Test public void shouldReturnMetricsVersionCurrent() { assertThat( - new StreamsMetricsImpl(metrics, THREAD_NAME, StreamsConfig.METRICS_LATEST).version(), + new StreamsMetricsImpl(metrics, THREAD_ID, StreamsConfig.METRICS_LATEST).version(), equalTo(Version.LATEST) ); } @@ -380,7 +559,7 @@ public void shouldReturnMetricsVersionCurrent() { @Test public void shouldReturnMetricsVersionFrom100To23() { assertThat( - new StreamsMetricsImpl(metrics, THREAD_NAME, StreamsConfig.METRICS_0100_TO_23).version(), + new StreamsMetricsImpl(metrics, THREAD_ID, StreamsConfig.METRICS_0100_TO_23).version(), equalTo(Version.FROM_100_TO_23) ); } diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/metrics/ThreadMetricsTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/metrics/ThreadMetricsTest.java index b586258982510..67d5c226afc21 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/metrics/ThreadMetricsTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/metrics/ThreadMetricsTest.java @@ -43,6 +43,7 @@ @PrepareForTest(StreamsMetricsImpl.class) public class ThreadMetricsTest { + private static final String THREAD_ID = "thread-id"; private static final String THREAD_LEVEL_GROUP = "stream-metrics"; private static final String TASK_LEVEL_GROUP = "stream-task-metrics"; @@ -57,15 +58,15 @@ public void shouldGetCreateTaskSensor() { final String totalDescription = "The total number of newly created tasks"; final String rateDescription = "The average per-second number of newly created tasks"; mockStatic(StreamsMetricsImpl.class); - expect(streamsMetrics.threadLevelSensor(operation, RecordingLevel.INFO)).andReturn(dummySensor); - expect(streamsMetrics.threadLevelTagMap()).andReturn(dummyTagMap); + expect(streamsMetrics.threadLevelSensor(THREAD_ID, operation, RecordingLevel.INFO)).andReturn(dummySensor); + expect(streamsMetrics.threadLevelTagMap(THREAD_ID)).andReturn(dummyTagMap); StreamsMetricsImpl.addInvocationRateAndCountToSensor( dummySensor, THREAD_LEVEL_GROUP, dummyTagMap, operation, totalDescription, rateDescription); replayAll(); replay(StreamsMetricsImpl.class); - final Sensor sensor = ThreadMetrics.createTaskSensor(streamsMetrics); + final Sensor sensor = ThreadMetrics.createTaskSensor(THREAD_ID, streamsMetrics); verifyAll(); verify(StreamsMetricsImpl.class); @@ -79,15 +80,15 @@ public void shouldGetCloseTaskSensor() { final String totalDescription = "The total number of closed tasks"; final String rateDescription = "The average per-second number of closed tasks"; mockStatic(StreamsMetricsImpl.class); - expect(streamsMetrics.threadLevelSensor(operation, RecordingLevel.INFO)).andReturn(dummySensor); - expect(streamsMetrics.threadLevelTagMap()).andReturn(dummyTagMap); + expect(streamsMetrics.threadLevelSensor(THREAD_ID, operation, RecordingLevel.INFO)).andReturn(dummySensor); + expect(streamsMetrics.threadLevelTagMap(THREAD_ID)).andReturn(dummyTagMap); StreamsMetricsImpl.addInvocationRateAndCountToSensor( dummySensor, THREAD_LEVEL_GROUP, dummyTagMap, operation, totalDescription, rateDescription); replayAll(); replay(StreamsMetricsImpl.class); - final Sensor sensor = ThreadMetrics.closeTaskSensor(streamsMetrics); + final Sensor sensor = ThreadMetrics.closeTaskSensor(THREAD_ID, streamsMetrics); verifyAll(); verify(StreamsMetricsImpl.class); @@ -102,8 +103,8 @@ public void shouldGetCommitSensor() { final String totalDescription = "The total number of commit calls"; final String rateDescription = "The average per-second number of commit calls"; mockStatic(StreamsMetricsImpl.class); - expect(streamsMetrics.threadLevelSensor(operation, RecordingLevel.INFO)).andReturn(dummySensor); - expect(streamsMetrics.threadLevelTagMap()).andReturn(dummyTagMap); + expect(streamsMetrics.threadLevelSensor(THREAD_ID, operation, RecordingLevel.INFO)).andReturn(dummySensor); + expect(streamsMetrics.threadLevelTagMap(THREAD_ID)).andReturn(dummyTagMap); StreamsMetricsImpl.addInvocationRateAndCountToSensor( dummySensor, THREAD_LEVEL_GROUP, dummyTagMap, operation, totalDescription, rateDescription); StreamsMetricsImpl.addAvgAndMaxToSensor( @@ -112,7 +113,7 @@ public void shouldGetCommitSensor() { replayAll(); replay(StreamsMetricsImpl.class); - final Sensor sensor = ThreadMetrics.commitSensor(streamsMetrics); + final Sensor sensor = ThreadMetrics.commitSensor(THREAD_ID, streamsMetrics); verifyAll(); verify(StreamsMetricsImpl.class); @@ -127,8 +128,8 @@ public void shouldGetPollSensor() { final String totalDescription = "The total number of poll calls"; final String rateDescription = "The average per-second number of poll calls"; mockStatic(StreamsMetricsImpl.class); - expect(streamsMetrics.threadLevelSensor(operation, RecordingLevel.INFO)).andReturn(dummySensor); - expect(streamsMetrics.threadLevelTagMap()).andReturn(dummyTagMap); + expect(streamsMetrics.threadLevelSensor(THREAD_ID, operation, RecordingLevel.INFO)).andReturn(dummySensor); + expect(streamsMetrics.threadLevelTagMap(THREAD_ID)).andReturn(dummyTagMap); StreamsMetricsImpl.addInvocationRateAndCountToSensor( dummySensor, THREAD_LEVEL_GROUP, dummyTagMap, operation, totalDescription, rateDescription); StreamsMetricsImpl.addAvgAndMaxToSensor( @@ -137,7 +138,7 @@ public void shouldGetPollSensor() { replayAll(); replay(StreamsMetricsImpl.class); - final Sensor sensor = ThreadMetrics.pollSensor(streamsMetrics); + final Sensor sensor = ThreadMetrics.pollSensor(THREAD_ID, streamsMetrics); verifyAll(); verify(StreamsMetricsImpl.class); @@ -152,8 +153,8 @@ public void shouldGetProcessSensor() { final String totalDescription = "The total number of process calls"; final String rateDescription = "The average per-second number of process calls"; mockStatic(StreamsMetricsImpl.class); - expect(streamsMetrics.threadLevelSensor(operation, RecordingLevel.INFO)).andReturn(dummySensor); - expect(streamsMetrics.threadLevelTagMap()).andReturn(dummyTagMap); + expect(streamsMetrics.threadLevelSensor(THREAD_ID, operation, RecordingLevel.INFO)).andReturn(dummySensor); + expect(streamsMetrics.threadLevelTagMap(THREAD_ID)).andReturn(dummyTagMap); StreamsMetricsImpl.addInvocationRateAndCountToSensor( dummySensor, THREAD_LEVEL_GROUP, dummyTagMap, operation, totalDescription, rateDescription); StreamsMetricsImpl.addAvgAndMaxToSensor( @@ -162,7 +163,7 @@ public void shouldGetProcessSensor() { replayAll(); replay(StreamsMetricsImpl.class); - final Sensor sensor = ThreadMetrics.processSensor(streamsMetrics); + final Sensor sensor = ThreadMetrics.processSensor(THREAD_ID, streamsMetrics); verifyAll(); verify(StreamsMetricsImpl.class); @@ -177,8 +178,8 @@ public void shouldGetPunctuateSensor() { final String totalDescription = "The total number of punctuate calls"; final String rateDescription = "The average per-second number of punctuate calls"; mockStatic(StreamsMetricsImpl.class); - expect(streamsMetrics.threadLevelSensor(operation, RecordingLevel.INFO)).andReturn(dummySensor); - expect(streamsMetrics.threadLevelTagMap()).andReturn(dummyTagMap); + expect(streamsMetrics.threadLevelSensor(THREAD_ID, operation, RecordingLevel.INFO)).andReturn(dummySensor); + expect(streamsMetrics.threadLevelTagMap(THREAD_ID)).andReturn(dummyTagMap); StreamsMetricsImpl.addInvocationRateAndCountToSensor( dummySensor, THREAD_LEVEL_GROUP, dummyTagMap, operation, totalDescription, rateDescription); StreamsMetricsImpl.addAvgAndMaxToSensor( @@ -187,7 +188,7 @@ public void shouldGetPunctuateSensor() { replayAll(); replay(StreamsMetricsImpl.class); - final Sensor sensor = ThreadMetrics.punctuateSensor(streamsMetrics); + final Sensor sensor = ThreadMetrics.punctuateSensor(THREAD_ID, streamsMetrics); verifyAll(); verify(StreamsMetricsImpl.class); @@ -201,15 +202,15 @@ public void shouldGetSkipRecordSensor() { final String totalDescription = "The total number of skipped records"; final String rateDescription = "The average per-second number of skipped records"; mockStatic(StreamsMetricsImpl.class); - expect(streamsMetrics.threadLevelSensor(operation, RecordingLevel.INFO)).andReturn(dummySensor); - expect(streamsMetrics.threadLevelTagMap()).andReturn(dummyTagMap); + expect(streamsMetrics.threadLevelSensor(THREAD_ID, operation, RecordingLevel.INFO)).andReturn(dummySensor); + expect(streamsMetrics.threadLevelTagMap(THREAD_ID)).andReturn(dummyTagMap); StreamsMetricsImpl.addInvocationRateAndCountToSensor( dummySensor, THREAD_LEVEL_GROUP, dummyTagMap, operation, totalDescription, rateDescription); replayAll(); replay(StreamsMetricsImpl.class); - final Sensor sensor = ThreadMetrics.skipRecordSensor(streamsMetrics); + final Sensor sensor = ThreadMetrics.skipRecordSensor(THREAD_ID, streamsMetrics); verifyAll(); verify(StreamsMetricsImpl.class); @@ -224,8 +225,8 @@ public void shouldGetCommitOverTasksSensor() { final String totalDescription = "The total number of commit calls over all tasks"; final String rateDescription = "The average per-second number of commit calls over all tasks"; mockStatic(StreamsMetricsImpl.class); - expect(streamsMetrics.threadLevelSensor(operation, RecordingLevel.DEBUG)).andReturn(dummySensor); - expect(streamsMetrics.threadLevelTagMap(TASK_ID_TAG, ROLLUP_VALUE)).andReturn(dummyTagMap); + expect(streamsMetrics.threadLevelSensor(THREAD_ID, operation, RecordingLevel.DEBUG)).andReturn(dummySensor); + expect(streamsMetrics.threadLevelTagMap(THREAD_ID, TASK_ID_TAG, ROLLUP_VALUE)).andReturn(dummyTagMap); StreamsMetricsImpl.addInvocationRateAndCountToSensor( dummySensor, TASK_LEVEL_GROUP, dummyTagMap, operation, totalDescription, rateDescription); StreamsMetricsImpl.addAvgAndMaxToSensor( @@ -234,7 +235,7 @@ public void shouldGetCommitOverTasksSensor() { replayAll(); replay(StreamsMetricsImpl.class); - final Sensor sensor = ThreadMetrics.commitOverTasksSensor(streamsMetrics); + final Sensor sensor = ThreadMetrics.commitOverTasksSensor(THREAD_ID, streamsMetrics); verifyAll(); verify(StreamsMetricsImpl.class); diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/AbstractRocksDBSegmentedBytesStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/AbstractRocksDBSegmentedBytesStoreTest.java index 004f181c5f613..19ea36d3e7c16 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/AbstractRocksDBSegmentedBytesStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/AbstractRocksDBSegmentedBytesStoreTest.java @@ -415,13 +415,14 @@ public void shouldLogAndMeasureExpiredRecords() { LogCaptureAppender.unregister(appender); final Map metrics = context.metrics().metrics(); + final String threadId = Thread.currentThread().getName(); final Metric dropTotal = metrics.get(new MetricName( "expired-window-record-drop-total", "stream-metrics-scope-metrics", "The total number of occurrence of expired-window-record-drop operations.", mkMap( - mkEntry("client-id", "mock"), + mkEntry("client-id", threadId), mkEntry("task-id", "0_0"), mkEntry("metrics-scope-id", "bytes-store") ) @@ -432,7 +433,7 @@ public void shouldLogAndMeasureExpiredRecords() { "stream-metrics-scope-metrics", "The average number of occurrence of expired-window-record-drop operation per second.", mkMap( - mkEntry("client-id", "mock"), + mkEntry("client-id", threadId), mkEntry("task-id", "0_0"), mkEntry("metrics-scope-id", "bytes-store") ) diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/KeyValueSegmentTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/KeyValueSegmentTest.java index f8248220c9870..621fdd3afb72e 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/KeyValueSegmentTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/KeyValueSegmentTest.java @@ -39,7 +39,8 @@ public class KeyValueSegmentTest { - private final RocksDBMetricsRecorder metricsRecorder = new RocksDBMetricsRecorder("metrics-scope", "store-name"); + private final RocksDBMetricsRecorder metricsRecorder = + new RocksDBMetricsRecorder("metrics-scope", "thread-id", "store-name"); @Test public void shouldDeleteStateDirectoryOnDestroy() throws Exception { diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/MeteredKeyValueStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/MeteredKeyValueStoreTest.java index e494532cfe8cf..533d58aea396b 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/MeteredKeyValueStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/MeteredKeyValueStoreTest.java @@ -63,7 +63,7 @@ public class MeteredKeyValueStoreTest { private final TaskId taskId = new TaskId(0, 0); private final Map tags = mkMap( - mkEntry("client-id", "test"), + mkEntry("client-id", Thread.currentThread().getName()), mkEntry("task-id", taskId.toString()), mkEntry("scope-state-id", "metered") ); @@ -106,9 +106,9 @@ public void testMetrics() { final JmxReporter reporter = new JmxReporter("kafka.streams"); metrics.addReporter(reporter); assertTrue(reporter.containsMbean(String.format("kafka.streams:type=stream-%s-state-metrics,client-id=%s,task-id=%s,%s-state-id=%s", - "scope", "test", taskId.toString(), "scope", "metered"))); + "scope", Thread.currentThread().getName(), taskId.toString(), "scope", "metered"))); assertTrue(reporter.containsMbean(String.format("kafka.streams:type=stream-%s-state-metrics,client-id=%s,task-id=%s,%s-state-id=%s", - "scope", "test", taskId.toString(), "scope", "all"))); + "scope", Thread.currentThread().getName(), taskId.toString(), "scope", "all"))); } @Test diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/MeteredSessionStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/MeteredSessionStoreTest.java index e21fbc2052629..25f842c888ec4 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/MeteredSessionStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/MeteredSessionStoreTest.java @@ -63,9 +63,10 @@ @RunWith(EasyMockRunner.class) public class MeteredSessionStoreTest { + private final String threadId = Thread.currentThread().getName(); private final TaskId taskId = new TaskId(0, 0); private final Map tags = mkMap( - mkEntry("client-id", "test"), + mkEntry("client-id", threadId), mkEntry("task-id", taskId.toString()), mkEntry("scope-state-id", "metered") ); @@ -105,9 +106,9 @@ public void testMetrics() { final JmxReporter reporter = new JmxReporter("kafka.streams"); metrics.addReporter(reporter); assertTrue(reporter.containsMbean(String.format("kafka.streams:type=stream-%s-state-metrics,client-id=%s,task-id=%s,%s-state-id=%s", - "scope", "test", taskId.toString(), "scope", "metered"))); + "scope", threadId, taskId.toString(), "scope", "metered"))); assertTrue(reporter.containsMbean(String.format("kafka.streams:type=stream-%s-state-metrics,client-id=%s,task-id=%s,%s-state-id=%s", - "scope", "test", taskId.toString(), "scope", "all"))); + "scope", threadId, taskId.toString(), "scope", "all"))); } @Test diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/MeteredTimestampedKeyValueStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/MeteredTimestampedKeyValueStoreTest.java index 47a49455cb3c0..837cc21009ad2 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/MeteredTimestampedKeyValueStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/MeteredTimestampedKeyValueStoreTest.java @@ -64,9 +64,10 @@ @RunWith(EasyMockRunner.class) public class MeteredTimestampedKeyValueStoreTest { + private final String threadId = Thread.currentThread().getName(); private final TaskId taskId = new TaskId(0, 0); private final Map tags = mkMap( - mkEntry("client-id", "test"), + mkEntry("client-id", threadId), mkEntry("task-id", taskId.toString()), mkEntry("scope-state-id", "metered") ); @@ -111,9 +112,9 @@ public void testMetrics() { final JmxReporter reporter = new JmxReporter("kafka.streams"); metrics.addReporter(reporter); assertTrue(reporter.containsMbean(String.format("kafka.streams:type=stream-%s-state-metrics,client-id=%s,task-id=%s,%s-state-id=%s", - "scope", "test", taskId.toString(), "scope", "metered"))); + "scope", threadId, taskId.toString(), "scope", "metered"))); assertTrue(reporter.containsMbean(String.format("kafka.streams:type=stream-%s-state-metrics,client-id=%s,task-id=%s,%s-state-id=%s", - "scope", "test", taskId.toString(), "scope", "all"))); + "scope", threadId, taskId.toString(), "scope", "all"))); } @Test public void shouldWriteBytesToInnerStoreAndRecordPutMetric() { diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/MeteredWindowStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/MeteredWindowStoreTest.java index e62ab3a8a424a..588307ff37b23 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/MeteredWindowStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/MeteredWindowStoreTest.java @@ -95,10 +95,11 @@ public void testMetrics() { store.init(context, store); final JmxReporter reporter = new JmxReporter("kafka.streams"); metrics.addReporter(reporter); + final String threadId = Thread.currentThread().getName(); assertTrue(reporter.containsMbean(String.format("kafka.streams:type=stream-%s-state-metrics,client-id=%s,task-id=%s,%s-state-id=%s", - "scope", "test", context.taskId().toString(), "scope", "mocked-store"))); + "scope", threadId, context.taskId().toString(), "scope", "mocked-store"))); assertTrue(reporter.containsMbean(String.format("kafka.streams:type=stream-%s-state-metrics,client-id=%s,task-id=%s,%s-state-id=%s", - "scope", "test", context.taskId().toString(), "scope", "all"))); + "scope", threadId, context.taskId().toString(), "scope", "all"))); } @Test diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/SegmentIteratorTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/SegmentIteratorTest.java index 2e305dd406b25..7944e5f0f5188 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/SegmentIteratorTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/SegmentIteratorTest.java @@ -41,7 +41,8 @@ public class SegmentIteratorTest { - private final RocksDBMetricsRecorder rocksDBMetricsRecorder = new RocksDBMetricsRecorder("metrics-scope", "store-name"); + private final RocksDBMetricsRecorder rocksDBMetricsRecorder = + new RocksDBMetricsRecorder("metrics-scope", "thread-id", "store-name"); private final KeyValueSegment segmentOne = new KeyValueSegment("one", "one", 0, rocksDBMetricsRecorder); private final KeyValueSegment segmentTwo = diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/SessionBytesStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/SessionBytesStoreTest.java index 9b29e8b9ff607..fe2bd6e52743d 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/SessionBytesStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/SessionBytesStoreTest.java @@ -443,13 +443,14 @@ public void shouldLogAndMeasureExpiredRecords() { final Map metrics = context.metrics().metrics(); final String metricScope = getMetricsScope(); + final String threadId = Thread.currentThread().getName(); final Metric dropTotal = metrics.get(new MetricName( "expired-window-record-drop-total", "stream-" + metricScope + "-metrics", "The total number of occurrence of expired-window-record-drop operations.", mkMap( - mkEntry("client-id", "mock"), + mkEntry("client-id", threadId), mkEntry("task-id", "0_0"), mkEntry(metricScope + "-id", sessionStore.name()) ) @@ -460,7 +461,7 @@ public void shouldLogAndMeasureExpiredRecords() { "stream-" + metricScope + "-metrics", "The average number of occurrence of expired-window-record-drop operation per second.", mkMap( - mkEntry("client-id", "mock"), + mkEntry("client-id", threadId), mkEntry("task-id", "0_0"), mkEntry(metricScope + "-id", sessionStore.name()) ) diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/TimestampedSegmentTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/TimestampedSegmentTest.java index 7bd3e7ebe46eb..a0069c861fe10 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/TimestampedSegmentTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/TimestampedSegmentTest.java @@ -39,7 +39,8 @@ public class TimestampedSegmentTest { - private final RocksDBMetricsRecorder metricsRecorder = new RocksDBMetricsRecorder("metrics-scope", "store-name"); + private final RocksDBMetricsRecorder metricsRecorder = + new RocksDBMetricsRecorder("metrics-scope", "thread-id", "store-name"); @Test public void shouldDeleteStateDirectoryOnDestroy() throws Exception { diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/WindowBytesStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/WindowBytesStoreTest.java index 5177079dd50ba..489c8254b3c70 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/WindowBytesStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/WindowBytesStoreTest.java @@ -892,13 +892,14 @@ public void shouldLogAndMeasureExpiredRecords() { final Map metrics = context.metrics().metrics(); final String metricScope = getMetricsScope(); + final String threadId = Thread.currentThread().getName(); final Metric dropTotal = metrics.get(new MetricName( "expired-window-record-drop-total", "stream-" + metricScope + "-metrics", "The total number of occurrence of expired-window-record-drop operations.", mkMap( - mkEntry("client-id", "mock"), + mkEntry("client-id", threadId), mkEntry("task-id", "0_0"), mkEntry(metricScope + "-id", windowStore.name()) ) @@ -909,7 +910,7 @@ public void shouldLogAndMeasureExpiredRecords() { "stream-" + metricScope + "-metrics", "The average number of occurrence of expired-window-record-drop operation per second.", mkMap( - mkEntry("client-id", "mock"), + mkEntry("client-id", threadId), mkEntry("task-id", "0_0"), mkEntry(metricScope + "-id", windowStore.name()) ) diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/metrics/NamedCacheMetricsTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/metrics/NamedCacheMetricsTest.java index 46ea302738bea..263947050978d 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/metrics/NamedCacheMetricsTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/metrics/NamedCacheMetricsTest.java @@ -43,7 +43,8 @@ @PrepareForTest({StreamsMetricsImpl.class, Sensor.class}) public class NamedCacheMetricsTest { - private static final String TASK_NAME = "taskName"; + private static final String THREAD_ID = "test-thread"; + private static final String TASK_ID = "test-task"; private static final String STORE_NAME = "storeName"; private static final String HIT_RATIO_AVG_DESCRIPTION = "The average cache hit ratio"; private static final String HIT_RATIO_MIN_DESCRIPTION = "The minimum cache hit ratio"; @@ -61,7 +62,7 @@ public void shouldGetHitRatioSensorWithBuiltInMetricsVersionCurrent() { replay(streamsMetrics); replay(StreamsMetricsImpl.class); - final Sensor sensor = NamedCacheMetrics.hitRatioSensor(streamsMetrics, TASK_NAME, STORE_NAME); + final Sensor sensor = NamedCacheMetrics.hitRatioSensor(streamsMetrics, THREAD_ID, TASK_ID, STORE_NAME); verifyResult(sensor); } @@ -73,8 +74,9 @@ public void shouldGetHitRatioSensorWithBuiltInMetricsVersionBefore24() { final RecordingLevel recordingLevel = RecordingLevel.DEBUG; mockStatic(StreamsMetricsImpl.class); final Sensor parentSensor = mock(Sensor.class); - expect(streamsMetrics.taskLevelSensor(TASK_NAME, hitRatio, recordingLevel)).andReturn(parentSensor); - expect(streamsMetrics.cacheLevelTagMap(TASK_NAME, StreamsMetricsImpl.ROLLUP_VALUE)).andReturn(parentTagMap); + expect(streamsMetrics.taskLevelSensor(THREAD_ID, TASK_ID, hitRatio, recordingLevel)).andReturn(parentSensor); + expect(streamsMetrics.cacheLevelTagMap(THREAD_ID, TASK_ID, StreamsMetricsImpl.ROLLUP_VALUE)) + .andReturn(parentTagMap); StreamsMetricsImpl.addAvgAndMinAndMaxToSensor( parentSensor, StreamsMetricsImpl.CACHE_LEVEL_GROUP, @@ -87,7 +89,7 @@ public void shouldGetHitRatioSensorWithBuiltInMetricsVersionBefore24() { replay(streamsMetrics); replay(StreamsMetricsImpl.class); - final Sensor sensor = NamedCacheMetrics.hitRatioSensor(streamsMetrics, TASK_NAME, STORE_NAME); + final Sensor sensor = NamedCacheMetrics.hitRatioSensor(streamsMetrics, THREAD_ID, TASK_ID, STORE_NAME); verifyResult(sensor); } @@ -96,9 +98,9 @@ private void setUpStreamsMetrics(final Version builtInMetricsVersion, final String hitRatio, final Sensor... parents) { expect(streamsMetrics.version()).andReturn(builtInMetricsVersion); - expect(streamsMetrics.cacheLevelSensor(TASK_NAME, STORE_NAME, hitRatio, RecordingLevel.DEBUG, parents)) + expect(streamsMetrics.cacheLevelSensor(THREAD_ID, TASK_ID, STORE_NAME, hitRatio, RecordingLevel.DEBUG, parents)) .andReturn(expectedSensor); - expect(streamsMetrics.cacheLevelTagMap(TASK_NAME, STORE_NAME)).andReturn(tagMap); + expect(streamsMetrics.cacheLevelTagMap(THREAD_ID, TASK_ID, STORE_NAME)).andReturn(tagMap); StreamsMetricsImpl.addAvgAndMinAndMaxToSensor( expectedSensor, StreamsMetricsImpl.CACHE_LEVEL_GROUP, diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/metrics/RocksDBMetricsRecorderTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/metrics/RocksDBMetricsRecorderTest.java index 9bef2cc3b6f6b..f22c02ed4d4c2 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/metrics/RocksDBMetricsRecorderTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/metrics/RocksDBMetricsRecorderTest.java @@ -47,6 +47,7 @@ @PrepareForTest({RocksDBMetrics.class, Sensor.class}) public class RocksDBMetricsRecorderTest { private final static String METRICS_SCOPE = "metrics-scope"; + private final static String THREAD_ID = "thread-id"; private final static String STORE_NAME = "store-name"; private final static String SEGMENT_STORE_NAME_1 = "segment-store-name-1"; private final static String SEGMENT_STORE_NAME_2 = "segment-store-name-2"; @@ -72,7 +73,7 @@ public class RocksDBMetricsRecorderTest { private final TaskId taskId1 = new TaskId(0, 0); private final TaskId taskId2 = new TaskId(0, 2); - private final RocksDBMetricsRecorder recorder = new RocksDBMetricsRecorder(METRICS_SCOPE, STORE_NAME); + private final RocksDBMetricsRecorder recorder = new RocksDBMetricsRecorder(METRICS_SCOPE, THREAD_ID, STORE_NAME); @Before public void setUp() { @@ -327,7 +328,7 @@ public void shouldCorrectlyHandleHitRatioRecordingsWithZeroHitsAndMisses() { private void setUpMetricsMock() { mockStatic(RocksDBMetrics.class); final RocksDBMetricContext metricsContext = - new RocksDBMetricContext(taskId1.toString(), METRICS_SCOPE, STORE_NAME); + new RocksDBMetricContext(THREAD_ID, taskId1.toString(), METRICS_SCOPE, STORE_NAME); expect(RocksDBMetrics.bytesWrittenToDatabaseSensor(eq(streamsMetrics), eq(metricsContext))) .andReturn(bytesWrittenToDatabaseSensor); expect(RocksDBMetrics.bytesReadFromDatabaseSensor(eq(streamsMetrics), eq(metricsContext))) diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/metrics/RocksDBMetricsTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/metrics/RocksDBMetricsTest.java index aae7afeeb7e21..d90a11624d1bc 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/metrics/RocksDBMetricsTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/metrics/RocksDBMetricsTest.java @@ -44,9 +44,10 @@ public class RocksDBMetricsTest { private static final String STATE_LEVEL_GROUP = "stream-state-metrics"; - private final String taskName = "task"; - private final String storeType = "test-state-id"; - private final String storeName = "store"; + private static final String THREAD_ID = "test-thread"; + private static final String TASK_ID = "test-task"; + private static final String STORE_TYPE = "test-store-type"; + private static final String STORE_NAME = "store"; private final Metrics metrics = new Metrics(); private final Sensor sensor = metrics.sensor("dummy"); private final StreamsMetricsImpl streamsMetrics = createStrictMock(StreamsMetricsImpl.class); @@ -267,15 +268,17 @@ private void verifySumSensor(final String metricNamePrefix, private void setupStreamsMetricsMock(final String metricNamePrefix) { mockStatic(StreamsMetricsImpl.class); expect(streamsMetrics.storeLevelSensor( - taskName, - storeName, + THREAD_ID, + TASK_ID, + STORE_NAME, metricNamePrefix, RecordingLevel.DEBUG )).andReturn(sensor); expect(streamsMetrics.storeLevelTagMap( - taskName, - storeType, - storeName + THREAD_ID, + TASK_ID, + STORE_TYPE, + STORE_NAME )).andReturn(tags); } @@ -284,7 +287,7 @@ private void replayCallAndVerify(final SensorCreator sensorCreator) { replay(StreamsMetricsImpl.class); final Sensor sensor = - sensorCreator.sensor(streamsMetrics, new RocksDBMetricContext(taskName, storeType, storeName)); + sensorCreator.sensor(streamsMetrics, new RocksDBMetricContext(THREAD_ID, TASK_ID, STORE_TYPE, STORE_NAME)); verifyAll(); verify(StreamsMetricsImpl.class); diff --git a/streams/test-utils/src/main/java/org/apache/kafka/streams/TopologyTestDriver.java b/streams/test-utils/src/main/java/org/apache/kafka/streams/TopologyTestDriver.java index a0ca6b2cad2d4..9b3d0fdd7faf2 100644 --- a/streams/test-utils/src/main/java/org/apache/kafka/streams/TopologyTestDriver.java +++ b/streams/test-utils/src/main/java/org/apache/kafka/streams/TopologyTestDriver.java @@ -280,24 +280,25 @@ public List partitionsFor(final String topic) { metrics = new Metrics(metricConfig, mockWallClockTime); - final String threadName = "topology-test-driver-virtual-thread"; + final String threadId = Thread.currentThread().getName(); final StreamsMetricsImpl streamsMetrics = new StreamsMetricsImpl( metrics, - threadName, + "test-client", StreamsConfig.METRICS_LATEST ); streamsMetrics.setRocksDBMetricsRecordingTrigger(new RocksDBMetricsRecordingTrigger()); - final Sensor skippedRecordsSensor = streamsMetrics.threadLevelSensor("skipped-records", Sensor.RecordingLevel.INFO); + final Sensor skippedRecordsSensor = + streamsMetrics.threadLevelSensor(threadId, "skipped-records", Sensor.RecordingLevel.INFO); final String threadLevelGroup = "stream-metrics"; skippedRecordsSensor.add(new MetricName("skipped-records-rate", threadLevelGroup, "The average per-second number of skipped records", - streamsMetrics.tagMap()), + streamsMetrics.tagMap(threadId)), new Rate(TimeUnit.SECONDS, new WindowedCount())); skippedRecordsSensor.add(new MetricName("skipped-records-total", threadLevelGroup, "The total number of skipped records", - streamsMetrics.tagMap()), + streamsMetrics.tagMap(threadId)), new CumulativeSum()); final ThreadCache cache = new ThreadCache( new LogContext("topology-test-driver "), diff --git a/streams/test-utils/src/main/java/org/apache/kafka/streams/processor/MockProcessorContext.java b/streams/test-utils/src/main/java/org/apache/kafka/streams/processor/MockProcessorContext.java index 041bf0d65d8b8..09850e4ed5a42 100644 --- a/streams/test-utils/src/main/java/org/apache/kafka/streams/processor/MockProcessorContext.java +++ b/streams/test-utils/src/main/java/org/apache/kafka/streams/processor/MockProcessorContext.java @@ -213,9 +213,9 @@ public MockProcessorContext(final Properties config, final TaskId taskId, final this.stateDir = stateDir; final MetricConfig metricConfig = new MetricConfig(); metricConfig.recordLevel(Sensor.RecordingLevel.DEBUG); - final String threadName = "mock-processor-context-virtual-thread"; - this.metrics = new StreamsMetricsImpl(new Metrics(metricConfig), threadName, StreamsConfig.METRICS_LATEST); - ThreadMetrics.skipRecordSensor(metrics); + final String threadId = Thread.currentThread().getName(); + this.metrics = new StreamsMetricsImpl(new Metrics(metricConfig), threadId, StreamsConfig.METRICS_LATEST); + ThreadMetrics.skipRecordSensor(threadId, metrics); } @Override