KAFKA-17248 - KIP 1076 Add process-id to get carried to telemetry results (#17630)

This PR adds the processId to Kafka Streams client instance metrics

Reviewers: Matthias Sax <mjsax@apache.org>
This commit is contained in:
Bill Bejeck 2024-11-07 12:12:42 -05:00 committed by GitHub
parent a0d4cbec40
commit c69a6b0e80
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
37 changed files with 164 additions and 116 deletions

View File

@ -74,6 +74,7 @@ import java.util.Arrays;
import java.util.List; import java.util.List;
import java.util.Locale; import java.util.Locale;
import java.util.Map; import java.util.Map;
import java.util.Optional;
import java.util.Properties; import java.util.Properties;
import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentHashMap;
import java.util.stream.Collectors; import java.util.stream.Collectors;
@ -167,6 +168,11 @@ public class KafkaStreamsTelemetryIntegrationTest {
assertNotNull(mainConsumerInstanceId); assertNotNull(mainConsumerInstanceId);
LOG.info("Main consumer instance id {}", mainConsumerInstanceId); LOG.info("Main consumer instance id {}", mainConsumerInstanceId);
final String expectedProcessId = streams.metrics().values().stream()
.filter(metric -> metric.metricName().tags().containsKey("process-id"))
.map(metric -> metric.metricName().tags().get("process-id"))
.findFirst().orElseThrow();
TestUtils.waitForCondition(() -> !TelemetryPlugin.SUBSCRIBED_METRICS.get(mainConsumerInstanceId).isEmpty(), TestUtils.waitForCondition(() -> !TelemetryPlugin.SUBSCRIBED_METRICS.get(mainConsumerInstanceId).isEmpty(),
30_000, 30_000,
"Never received subscribed metrics"); "Never received subscribed metrics");
@ -185,6 +191,12 @@ public class KafkaStreamsTelemetryIntegrationTest {
final List<String> actualInstanceMetrics = TelemetryPlugin.SUBSCRIBED_METRICS.get(adminInstanceId); final List<String> actualInstanceMetrics = TelemetryPlugin.SUBSCRIBED_METRICS.get(adminInstanceId);
final List<String> expectedInstanceMetrics = Arrays.asList("org.apache.kafka.stream.alive.stream.threads", "org.apache.kafka.stream.failed.stream.threads"); final List<String> expectedInstanceMetrics = Arrays.asList("org.apache.kafka.stream.alive.stream.threads", "org.apache.kafka.stream.failed.stream.threads");
assertEquals(expectedInstanceMetrics, actualInstanceMetrics); assertEquals(expectedInstanceMetrics, actualInstanceMetrics);
TestUtils.waitForCondition(() -> TelemetryPlugin.processId != null,
30_000,
"Never received the process id");
assertEquals(expectedProcessId, TelemetryPlugin.processId);
} }
} }
@ -456,6 +468,7 @@ public class KafkaStreamsTelemetryIntegrationTest {
public static class TelemetryPlugin implements ClientTelemetry, MetricsReporter, ClientTelemetryReceiver { public static class TelemetryPlugin implements ClientTelemetry, MetricsReporter, ClientTelemetryReceiver {
public static final Map<Uuid, List<String>> SUBSCRIBED_METRICS = new ConcurrentHashMap<>(); public static final Map<Uuid, List<String>> SUBSCRIBED_METRICS = new ConcurrentHashMap<>();
public static String processId;
public TelemetryPlugin() { public TelemetryPlugin() {
} }
@ -490,10 +503,26 @@ public class KafkaStreamsTelemetryIntegrationTest {
public void exportMetrics(final AuthorizableRequestContext context, final ClientTelemetryPayload payload) { public void exportMetrics(final AuthorizableRequestContext context, final ClientTelemetryPayload payload) {
try { try {
final MetricsData data = MetricsData.parseFrom(payload.data()); final MetricsData data = MetricsData.parseFrom(payload.data());
final Optional<String> processIdOption = data.getResourceMetricsList()
.stream()
.flatMap(rm -> rm.getScopeMetricsList().stream())
.flatMap(sm -> sm.getMetricsList().stream())
.map(metric -> metric.getGauge())
.flatMap(gauge -> gauge.getDataPointsList().stream())
.flatMap(numberDataPoint -> numberDataPoint.getAttributesList().stream())
.filter(keyValue -> keyValue.getKey().equals("process_id"))
.map(keyValue -> keyValue.getValue().getStringValue())
.findFirst();
processIdOption.ifPresent(pid -> processId = pid);
final Uuid clientId = payload.clientInstanceId(); final Uuid clientId = payload.clientInstanceId();
final List<String> metricNames = data.getResourceMetricsList() final List<String> metricNames = data.getResourceMetricsList()
.stream() .stream()
.map(rm -> rm.getScopeMetricsList().get(0).getMetrics(0).getName()) .flatMap(rm -> rm.getScopeMetricsList().stream())
.flatMap(sm -> sm.getMetricsList().stream())
.map(metric -> metric.getName())
.sorted() .sorted()
.collect(Collectors.toList()); .collect(Collectors.toList());
LOG.info("Found metrics {} for clientId={}", metricNames, clientId); LOG.info("Found metrics {} for clientId={}", metricNames, clientId);

View File

@ -978,6 +978,7 @@ public class KafkaStreams implements AutoCloseable {
streamsMetrics = new StreamsMetricsImpl( streamsMetrics = new StreamsMetricsImpl(
metrics, metrics,
clientId, clientId,
processId.toString(),
time time
); );

View File

@ -87,6 +87,7 @@ public class StreamsMetricsImpl implements StreamsMetrics {
private final Metrics metrics; private final Metrics metrics;
private final Map<Sensor, Sensor> parentSensors; private final Map<Sensor, Sensor> parentSensors;
private final String clientId; private final String clientId;
private final String processId;
private final Version version; private final Version version;
private final Deque<MetricName> clientLevelMetrics = new LinkedList<>(); private final Deque<MetricName> clientLevelMetrics = new LinkedList<>();
@ -114,6 +115,7 @@ public class StreamsMetricsImpl implements StreamsMetrics {
private static final String SENSOR_INTERNAL_LABEL = "internal"; private static final String SENSOR_INTERNAL_LABEL = "internal";
public static final String CLIENT_ID_TAG = "client-id"; public static final String CLIENT_ID_TAG = "client-id";
public static final String PROCESS_ID_TAG = "process-id";
public static final String THREAD_ID_TAG = "thread-id"; public static final String THREAD_ID_TAG = "thread-id";
public static final String TASK_ID_TAG = "task-id"; public static final String TASK_ID_TAG = "task-id";
public static final String PROCESSOR_NODE_ID_TAG = "processor-node-id"; public static final String PROCESSOR_NODE_ID_TAG = "processor-node-id";
@ -162,10 +164,12 @@ public class StreamsMetricsImpl implements StreamsMetrics {
public StreamsMetricsImpl(final Metrics metrics, public StreamsMetricsImpl(final Metrics metrics,
final String clientId, final String clientId,
final String processId,
final Time time) { final Time time) {
Objects.requireNonNull(metrics, "Metrics cannot be null"); Objects.requireNonNull(metrics, "Metrics cannot be null");
this.metrics = metrics; this.metrics = metrics;
this.clientId = clientId; this.clientId = clientId;
this.processId = processId;
version = Version.LATEST; version = Version.LATEST;
rocksDBMetricsRecordingTrigger = new RocksDBMetricsRecordingTrigger(time); rocksDBMetricsRecordingTrigger = new RocksDBMetricsRecordingTrigger(time);
@ -269,6 +273,7 @@ public class StreamsMetricsImpl implements StreamsMetrics {
public Map<String, String> clientLevelTagMap() { public Map<String, String> clientLevelTagMap() {
final Map<String, String> tagMap = new LinkedHashMap<>(); final Map<String, String> tagMap = new LinkedHashMap<>();
tagMap.put(CLIENT_ID_TAG, clientId); tagMap.put(CLIENT_ID_TAG, clientId);
tagMap.put(PROCESS_ID_TAG, processId);
return tagMap; return tagMap;
} }

View File

@ -80,7 +80,7 @@ public class KStreamSessionWindowAggregateProcessorTest {
private final MockTime time = new MockTime(); private final MockTime time = new MockTime();
private final Metrics metrics = new Metrics(); private final Metrics metrics = new Metrics();
private final StreamsMetricsImpl streamsMetrics = new StreamsMetricsImpl(metrics, "test", time); private final StreamsMetricsImpl streamsMetrics = new StreamsMetricsImpl(metrics, "test", "processId", time);
private final String threadId = Thread.currentThread().getName(); private final String threadId = Thread.currentThread().getName();
private final Initializer<Long> initializer = () -> 0L; private final Initializer<Long> initializer = () -> 0L;
private final Aggregator<String, String, Long> aggregator = (aggKey, value, aggregate) -> aggregate + 1; private final Aggregator<String, String, Long> aggregator = (aggKey, value, aggregate) -> aggregate + 1;

View File

@ -72,7 +72,7 @@ public class ActiveTaskCreatorTest {
private ChangelogReader changeLogReader; private ChangelogReader changeLogReader;
private final MockClientSupplier mockClientSupplier = new MockClientSupplier(); private final MockClientSupplier mockClientSupplier = new MockClientSupplier();
private final StreamsMetricsImpl streamsMetrics = new StreamsMetricsImpl(new Metrics(), "clientId", new MockTime()); private final StreamsMetricsImpl streamsMetrics = new StreamsMetricsImpl(new Metrics(), "clientId", "processId", new MockTime());
private final Map<String, Object> properties = mkMap( private final Map<String, Object> properties = mkMap(
mkEntry(StreamsConfig.APPLICATION_ID_CONFIG, "appId"), mkEntry(StreamsConfig.APPLICATION_ID_CONFIG, "appId"),
mkEntry(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "dummy:1234") mkEntry(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "dummy:1234")

View File

@ -134,7 +134,7 @@ public class GlobalStreamThreadTest {
mockConsumer, mockConsumer,
new StateDirectory(config, time, true, false), new StateDirectory(config, time, true, false),
0, 0,
new StreamsMetricsImpl(new Metrics(), "test-client", time), new StreamsMetricsImpl(new Metrics(), "test-client", "processId", time),
time, time,
"clientId", "clientId",
stateRestoreListener, stateRestoreListener,
@ -173,7 +173,7 @@ public class GlobalStreamThreadTest {
mockConsumer, mockConsumer,
new StateDirectory(config, time, true, false), new StateDirectory(config, time, true, false),
0, 0,
new StreamsMetricsImpl(new Metrics(), "test-client", time), new StreamsMetricsImpl(new Metrics(), "test-client", "processId", time),
time, time,
"clientId", "clientId",
stateRestoreListener, stateRestoreListener,

View File

@ -23,6 +23,6 @@ import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl;
public class MockStreamsMetrics extends StreamsMetricsImpl { public class MockStreamsMetrics extends StreamsMetricsImpl {
public MockStreamsMetrics(final Metrics metrics) { public MockStreamsMetrics(final Metrics metrics) {
super(metrics, "test", new MockTime()); super(metrics, "test", "processId", new MockTime());
} }
} }

View File

@ -217,7 +217,7 @@ public class ProcessorNodeTest {
public void testMetricsWithBuiltInMetricsVersionLatest() { public void testMetricsWithBuiltInMetricsVersionLatest() {
final Metrics metrics = new Metrics(); final Metrics metrics = new Metrics();
final StreamsMetricsImpl streamsMetrics = final StreamsMetricsImpl streamsMetrics =
new StreamsMetricsImpl(metrics, "test-client", new MockTime()); new StreamsMetricsImpl(metrics, "test-client", "processId", new MockTime());
final InternalMockProcessorContext<Object, Object> context = new InternalMockProcessorContext<>(streamsMetrics); final InternalMockProcessorContext<Object, Object> context = new InternalMockProcessorContext<>(streamsMetrics);
final ProcessorNode<Object, Object, Object, Object> node = final ProcessorNode<Object, Object, Object, Object> node =
new ProcessorNode<>(NAME, new NoOpProcessor(), Collections.emptySet()); new ProcessorNode<>(NAME, new NoOpProcessor(), Collections.emptySet());
@ -301,7 +301,7 @@ public class ProcessorNodeTest {
public void testTopologyLevelClassCastExceptionDirect() { public void testTopologyLevelClassCastExceptionDirect() {
final Metrics metrics = new Metrics(); final Metrics metrics = new Metrics();
final StreamsMetricsImpl streamsMetrics = final StreamsMetricsImpl streamsMetrics =
new StreamsMetricsImpl(metrics, "test-client", new MockTime()); new StreamsMetricsImpl(metrics, "test-client", "processId", new MockTime());
final InternalMockProcessorContext<Object, Object> context = new InternalMockProcessorContext<>(streamsMetrics); final InternalMockProcessorContext<Object, Object> context = new InternalMockProcessorContext<>(streamsMetrics);
final ProcessorNode<Object, Object, Object, Object> node = final ProcessorNode<Object, Object, Object, Object> node =
new ProcessorNode<>("pname", new ClassCastProcessor(), Collections.emptySet()); new ProcessorNode<>("pname", new ClassCastProcessor(), Collections.emptySet());
@ -321,7 +321,7 @@ public class ProcessorNodeTest {
final InternalProcessorContext<Object, Object> internalProcessorContext = mock(InternalProcessorContext.class, withSettings().strictness(Strictness.LENIENT)); final InternalProcessorContext<Object, Object> internalProcessorContext = mock(InternalProcessorContext.class, withSettings().strictness(Strictness.LENIENT));
when(internalProcessorContext.taskId()).thenReturn(TASK_ID); when(internalProcessorContext.taskId()).thenReturn(TASK_ID);
when(internalProcessorContext.metrics()).thenReturn(new StreamsMetricsImpl(new Metrics(), "test-client", new MockTime())); when(internalProcessorContext.metrics()).thenReturn(new StreamsMetricsImpl(new Metrics(), "test-client", "processId", new MockTime()));
when(internalProcessorContext.topic()).thenReturn(TOPIC); when(internalProcessorContext.topic()).thenReturn(TOPIC);
when(internalProcessorContext.partition()).thenReturn(PARTITION); when(internalProcessorContext.partition()).thenReturn(PARTITION);
when(internalProcessorContext.offset()).thenReturn(OFFSET); when(internalProcessorContext.offset()).thenReturn(OFFSET);

View File

@ -72,7 +72,7 @@ public class RecordQueueTest {
private final Metrics metrics = new Metrics(); private final Metrics metrics = new Metrics();
private final StreamsMetricsImpl streamsMetrics = private final StreamsMetricsImpl streamsMetrics =
new StreamsMetricsImpl(metrics, "mock", new MockTime()); new StreamsMetricsImpl(metrics, "mock", "processId", new MockTime());
@SuppressWarnings("rawtypes") @SuppressWarnings("rawtypes")
final InternalMockProcessorContext context = new InternalMockProcessorContext<>( final InternalMockProcessorContext context = new InternalMockProcessorContext<>(

View File

@ -97,7 +97,7 @@ public class SourceNodeTest {
public void shouldExposeProcessMetrics() { public void shouldExposeProcessMetrics() {
final Metrics metrics = new Metrics(); final Metrics metrics = new Metrics();
final StreamsMetricsImpl streamsMetrics = final StreamsMetricsImpl streamsMetrics =
new StreamsMetricsImpl(metrics, "test-client", new MockTime()); new StreamsMetricsImpl(metrics, "test-client", "processId", new MockTime());
final InternalMockProcessorContext<String, String> context = new InternalMockProcessorContext<>(streamsMetrics); final InternalMockProcessorContext<String, String> context = new InternalMockProcessorContext<>(streamsMetrics);
final SourceNode<String, String> node = final SourceNode<String, String> node =
new SourceNode<>(context.currentNode().name(), new TheDeserializer(), new TheDeserializer()); new SourceNode<>(context.currentNode().name(), new TheDeserializer(), new TheDeserializer());

View File

@ -113,7 +113,7 @@ public class StandbyTaskTest {
private final MockTime time = new MockTime(); private final MockTime time = new MockTime();
private final Metrics metrics = new Metrics(new MetricConfig().recordLevel(Sensor.RecordingLevel.DEBUG), time); private final Metrics metrics = new Metrics(new MetricConfig().recordLevel(Sensor.RecordingLevel.DEBUG), time);
private final StreamsMetricsImpl streamsMetrics = new StreamsMetricsImpl(metrics, threadName, time); private final StreamsMetricsImpl streamsMetrics = new StreamsMetricsImpl(metrics, threadName, "processId", time);
private File baseDir; private File baseDir;
private StreamsConfig config; private StreamsConfig config;

View File

@ -957,7 +957,7 @@ public class StateDirectoryTest {
Mockito.when(metadata.buildSubtopology(ArgumentMatchers.any())).thenReturn(processorTopology); Mockito.when(metadata.buildSubtopology(ArgumentMatchers.any())).thenReturn(processorTopology);
Mockito.when(metadata.taskConfig(ArgumentMatchers.any())).thenReturn(topologyConfig.getTaskConfig()); Mockito.when(metadata.taskConfig(ArgumentMatchers.any())).thenReturn(topologyConfig.getTaskConfig());
directory.initializeStartupTasks(metadata, new StreamsMetricsImpl(new Metrics(), "test", time), new LogContext("test")); directory.initializeStartupTasks(metadata, new StreamsMetricsImpl(new Metrics(), "test", "processId", time), new LogContext("test"));
return store; return store;
} }

View File

@ -2487,7 +2487,7 @@ public class StreamTaskTest {
streamsMetrics, streamsMetrics,
null null
); );
final StreamsMetricsImpl metrics = new StreamsMetricsImpl(this.metrics, "test", time); final StreamsMetricsImpl metrics = new StreamsMetricsImpl(this.metrics, "test", "processId", time);
// The processor topology is missing the topics // The processor topology is missing the topics
final ProcessorTopology topology = withSources(emptyList(), mkMap()); final ProcessorTopology topology = withSources(emptyList(), mkMap());
@ -3043,7 +3043,7 @@ public class StreamTaskTest {
topology, topology,
consumer, consumer,
new TopologyConfig(null, config, new Properties()).getTaskConfig(), new TopologyConfig(null, config, new Properties()).getTaskConfig(),
new StreamsMetricsImpl(metrics, "test", time), new StreamsMetricsImpl(metrics, "test", "processId", time),
stateDirectory, stateDirectory,
cache, cache,
time, time,
@ -3080,7 +3080,7 @@ public class StreamTaskTest {
topology, topology,
consumer, consumer,
new TopologyConfig(null, config, new Properties()).getTaskConfig(), new TopologyConfig(null, config, new Properties()).getTaskConfig(),
new StreamsMetricsImpl(metrics, "test", time), new StreamsMetricsImpl(metrics, "test", "processId", time),
stateDirectory, stateDirectory,
cache, cache,
time, time,
@ -3116,7 +3116,7 @@ public class StreamTaskTest {
topology, topology,
consumer, consumer,
new TopologyConfig(null, config, new Properties()).getTaskConfig(), new TopologyConfig(null, config, new Properties()).getTaskConfig(),
new StreamsMetricsImpl(metrics, "test", time), new StreamsMetricsImpl(metrics, "test", "processId", time),
stateDirectory, stateDirectory,
cache, cache,
time, time,

View File

@ -305,6 +305,7 @@ public class StreamThreadTest {
final StreamsMetricsImpl streamsMetrics = new StreamsMetricsImpl( final StreamsMetricsImpl streamsMetrics = new StreamsMetricsImpl(
metrics, metrics,
APPLICATION_ID, APPLICATION_ID,
PROCESS_ID.toString(),
time time
); );
@ -712,6 +713,7 @@ public class StreamThreadTest {
final StreamsMetricsImpl streamsMetrics = new StreamsMetricsImpl( final StreamsMetricsImpl streamsMetrics = new StreamsMetricsImpl(
metrics, metrics,
APPLICATION_ID, APPLICATION_ID,
PROCESS_ID.toString(),
mockTime mockTime
); );
@ -776,6 +778,7 @@ public class StreamThreadTest {
final StreamsMetricsImpl streamsMetrics = new StreamsMetricsImpl( final StreamsMetricsImpl streamsMetrics = new StreamsMetricsImpl(
metrics, metrics,
APPLICATION_ID, APPLICATION_ID,
PROCESS_ID.toString(),
mockTime mockTime
); );
@ -1140,7 +1143,7 @@ public class StreamThreadTest {
final StreamsConfig config = new StreamsConfig(configProps(false, stateUpdaterEnabled, processingThreadsEnabled)); final StreamsConfig config = new StreamsConfig(configProps(false, stateUpdaterEnabled, processingThreadsEnabled));
final StreamsMetricsImpl streamsMetrics = final StreamsMetricsImpl streamsMetrics =
new StreamsMetricsImpl(metrics, CLIENT_ID, mockTime); new StreamsMetricsImpl(metrics, CLIENT_ID, PROCESS_ID.toString(), mockTime);
final TopologyMetadata topologyMetadata = new TopologyMetadata(internalTopologyBuilder, config); final TopologyMetadata topologyMetadata = new TopologyMetadata(internalTopologyBuilder, config);
topologyMetadata.buildAndRewriteTopology(); topologyMetadata.buildAndRewriteTopology();
stateDirectory = new StateDirectory(config, mockTime, true, false); stateDirectory = new StateDirectory(config, mockTime, true, false);
@ -1364,7 +1367,7 @@ public class StreamThreadTest {
final StreamsConfig config = new StreamsConfig(configProps(false, stateUpdaterEnabled, processingThreadsEnabled)); final StreamsConfig config = new StreamsConfig(configProps(false, stateUpdaterEnabled, processingThreadsEnabled));
final StreamsMetricsImpl streamsMetrics = final StreamsMetricsImpl streamsMetrics =
new StreamsMetricsImpl(metrics, CLIENT_ID, mockTime); new StreamsMetricsImpl(metrics, CLIENT_ID, PROCESS_ID.toString(), mockTime);
final TopologyMetadata topologyMetadata = new TopologyMetadata(internalTopologyBuilder, config); final TopologyMetadata topologyMetadata = new TopologyMetadata(internalTopologyBuilder, config);
topologyMetadata.buildAndRewriteTopology(); topologyMetadata.buildAndRewriteTopology();
thread = buildStreamThread(consumer, taskManager, config, topologyMetadata) thread = buildStreamThread(consumer, taskManager, config, topologyMetadata)
@ -1418,7 +1421,7 @@ public class StreamThreadTest {
} }
final StreamsMetricsImpl streamsMetrics = final StreamsMetricsImpl streamsMetrics =
new StreamsMetricsImpl(metrics, CLIENT_ID, mockTime); new StreamsMetricsImpl(metrics, CLIENT_ID, PROCESS_ID.toString(), mockTime);
final Properties props = configProps(false, stateUpdaterEnabled, processingThreadsEnabled); final Properties props = configProps(false, stateUpdaterEnabled, processingThreadsEnabled);
final StreamsConfig config = new StreamsConfig(props); final StreamsConfig config = new StreamsConfig(props);
@ -1464,7 +1467,7 @@ public class StreamThreadTest {
final StreamsConfig config = new StreamsConfig(configProps(false, stateUpdaterEnabled, processingThreadsEnabled)); final StreamsConfig config = new StreamsConfig(configProps(false, stateUpdaterEnabled, processingThreadsEnabled));
final StreamsMetricsImpl streamsMetrics = final StreamsMetricsImpl streamsMetrics =
new StreamsMetricsImpl(metrics, CLIENT_ID, mockTime); new StreamsMetricsImpl(metrics, CLIENT_ID, PROCESS_ID.toString(), mockTime);
final TopologyMetadata topologyMetadata = new TopologyMetadata(internalTopologyBuilder, config); final TopologyMetadata topologyMetadata = new TopologyMetadata(internalTopologyBuilder, config);
topologyMetadata.buildAndRewriteTopology(); topologyMetadata.buildAndRewriteTopology();
thread = buildStreamThread(consumer, taskManager, config, topologyMetadata) thread = buildStreamThread(consumer, taskManager, config, topologyMetadata)
@ -1484,7 +1487,7 @@ public class StreamThreadTest {
final StreamsConfig config = new StreamsConfig(configProps(false, stateUpdaterEnabled, processingThreadsEnabled)); final StreamsConfig config = new StreamsConfig(configProps(false, stateUpdaterEnabled, processingThreadsEnabled));
final StreamsMetricsImpl streamsMetrics = final StreamsMetricsImpl streamsMetrics =
new StreamsMetricsImpl(metrics, CLIENT_ID, mockTime); new StreamsMetricsImpl(metrics, CLIENT_ID, PROCESS_ID.toString(), mockTime);
final TopologyMetadata topologyMetadata = new TopologyMetadata(internalTopologyBuilder, config); final TopologyMetadata topologyMetadata = new TopologyMetadata(internalTopologyBuilder, config);
topologyMetadata.buildAndRewriteTopology(); topologyMetadata.buildAndRewriteTopology();
thread = buildStreamThread(consumer, taskManager, config, topologyMetadata) thread = buildStreamThread(consumer, taskManager, config, topologyMetadata)
@ -1885,6 +1888,7 @@ public class StreamThreadTest {
final StreamsMetricsImpl streamsMetrics = new StreamsMetricsImpl( final StreamsMetricsImpl streamsMetrics = new StreamsMetricsImpl(
metrics, metrics,
APPLICATION_ID, APPLICATION_ID,
PROCESS_ID.toString(),
mockTime mockTime
); );
@ -2582,7 +2586,7 @@ public class StreamThreadTest {
doThrow(new TaskMigratedException("Task lost exception", new RuntimeException())).when(taskManager).handleLostAll(); doThrow(new TaskMigratedException("Task lost exception", new RuntimeException())).when(taskManager).handleLostAll();
final StreamsMetricsImpl streamsMetrics = final StreamsMetricsImpl streamsMetrics =
new StreamsMetricsImpl(metrics, CLIENT_ID, mockTime); new StreamsMetricsImpl(metrics, CLIENT_ID, PROCESS_ID.toString(), mockTime);
final TopologyMetadata topologyMetadata = new TopologyMetadata(internalTopologyBuilder, config); final TopologyMetadata topologyMetadata = new TopologyMetadata(internalTopologyBuilder, config);
topologyMetadata.buildAndRewriteTopology(); topologyMetadata.buildAndRewriteTopology();
thread = buildStreamThread(consumer, taskManager, config, topologyMetadata) thread = buildStreamThread(consumer, taskManager, config, topologyMetadata)
@ -2612,7 +2616,7 @@ public class StreamThreadTest {
doThrow(new TaskMigratedException("Revocation non fatal exception", new RuntimeException())).when(taskManager).handleRevocation(assignedPartitions); doThrow(new TaskMigratedException("Revocation non fatal exception", new RuntimeException())).when(taskManager).handleRevocation(assignedPartitions);
final StreamsMetricsImpl streamsMetrics = final StreamsMetricsImpl streamsMetrics =
new StreamsMetricsImpl(metrics, CLIENT_ID, mockTime); new StreamsMetricsImpl(metrics, CLIENT_ID, PROCESS_ID.toString(), mockTime);
final TopologyMetadata topologyMetadata = new TopologyMetadata(internalTopologyBuilder, config); final TopologyMetadata topologyMetadata = new TopologyMetadata(internalTopologyBuilder, config);
topologyMetadata.buildAndRewriteTopology(); topologyMetadata.buildAndRewriteTopology();
thread = buildStreamThread(consumer, taskManager, config, topologyMetadata) thread = buildStreamThread(consumer, taskManager, config, topologyMetadata)
@ -2644,7 +2648,7 @@ public class StreamThreadTest {
when(taskManager.handleCorruption(corruptedTasks)).thenReturn(true); when(taskManager.handleCorruption(corruptedTasks)).thenReturn(true);
final StreamsMetricsImpl streamsMetrics = final StreamsMetricsImpl streamsMetrics =
new StreamsMetricsImpl(metrics, CLIENT_ID, mockTime); new StreamsMetricsImpl(metrics, CLIENT_ID, PROCESS_ID.toString(), mockTime);
final TopologyMetadata topologyMetadata = new TopologyMetadata(internalTopologyBuilder, config); final TopologyMetadata topologyMetadata = new TopologyMetadata(internalTopologyBuilder, config);
topologyMetadata.buildAndRewriteTopology(); topologyMetadata.buildAndRewriteTopology();
thread = new StreamThread( thread = new StreamThread(
@ -2702,7 +2706,7 @@ public class StreamThreadTest {
doThrow(new TimeoutException()).when(taskManager).handleCorruption(corruptedTasks); doThrow(new TimeoutException()).when(taskManager).handleCorruption(corruptedTasks);
final StreamsMetricsImpl streamsMetrics = final StreamsMetricsImpl streamsMetrics =
new StreamsMetricsImpl(metrics, CLIENT_ID, mockTime); new StreamsMetricsImpl(metrics, CLIENT_ID, PROCESS_ID.toString(), mockTime);
final TopologyMetadata topologyMetadata = new TopologyMetadata(internalTopologyBuilder, config); final TopologyMetadata topologyMetadata = new TopologyMetadata(internalTopologyBuilder, config);
topologyMetadata.buildAndRewriteTopology(); topologyMetadata.buildAndRewriteTopology();
thread = new StreamThread( thread = new StreamThread(
@ -2769,7 +2773,7 @@ public class StreamThreadTest {
doNothing().when(taskManager).handleLostAll(); doNothing().when(taskManager).handleLostAll();
final StreamsMetricsImpl streamsMetrics = final StreamsMetricsImpl streamsMetrics =
new StreamsMetricsImpl(metrics, CLIENT_ID, mockTime); new StreamsMetricsImpl(metrics, CLIENT_ID, PROCESS_ID.toString(), mockTime);
final TopologyMetadata topologyMetadata = new TopologyMetadata(internalTopologyBuilder, config); final TopologyMetadata topologyMetadata = new TopologyMetadata(internalTopologyBuilder, config);
topologyMetadata.buildAndRewriteTopology(); topologyMetadata.buildAndRewriteTopology();
thread = new StreamThread( thread = new StreamThread(
@ -2832,7 +2836,7 @@ public class StreamThreadTest {
doNothing().when(consumer).enforceRebalance("Active tasks corrupted"); doNothing().when(consumer).enforceRebalance("Active tasks corrupted");
final StreamsMetricsImpl streamsMetrics = final StreamsMetricsImpl streamsMetrics =
new StreamsMetricsImpl(metrics, CLIENT_ID, mockTime); new StreamsMetricsImpl(metrics, CLIENT_ID, PROCESS_ID.toString(), mockTime);
final TopologyMetadata topologyMetadata = new TopologyMetadata(internalTopologyBuilder, config); final TopologyMetadata topologyMetadata = new TopologyMetadata(internalTopologyBuilder, config);
topologyMetadata.buildAndRewriteTopology(); topologyMetadata.buildAndRewriteTopology();
thread = new StreamThread( thread = new StreamThread(
@ -2892,7 +2896,7 @@ public class StreamThreadTest {
when(taskManager.handleCorruption(corruptedTasks)).thenReturn(false); when(taskManager.handleCorruption(corruptedTasks)).thenReturn(false);
final StreamsMetricsImpl streamsMetrics = final StreamsMetricsImpl streamsMetrics =
new StreamsMetricsImpl(metrics, CLIENT_ID, mockTime); new StreamsMetricsImpl(metrics, CLIENT_ID, PROCESS_ID.toString(), mockTime);
final TopologyMetadata topologyMetadata = new TopologyMetadata(internalTopologyBuilder, config); final TopologyMetadata topologyMetadata = new TopologyMetadata(internalTopologyBuilder, config);
topologyMetadata.buildAndRewriteTopology(); topologyMetadata.buildAndRewriteTopology();
thread = new StreamThread( thread = new StreamThread(
@ -3076,7 +3080,7 @@ public class StreamThreadTest {
when(taskManager.producerMetrics()).thenReturn(dummyProducerMetrics); when(taskManager.producerMetrics()).thenReturn(dummyProducerMetrics);
final StreamsMetricsImpl streamsMetrics = final StreamsMetricsImpl streamsMetrics =
new StreamsMetricsImpl(metrics, CLIENT_ID, mockTime); new StreamsMetricsImpl(metrics, CLIENT_ID, PROCESS_ID.toString(), mockTime);
final TopologyMetadata topologyMetadata = new TopologyMetadata(internalTopologyBuilder, config); final TopologyMetadata topologyMetadata = new TopologyMetadata(internalTopologyBuilder, config);
topologyMetadata.buildAndRewriteTopology(); topologyMetadata.buildAndRewriteTopology();
thread = buildStreamThread(consumer, taskManager, config, topologyMetadata); thread = buildStreamThread(consumer, taskManager, config, topologyMetadata);
@ -3101,7 +3105,7 @@ public class StreamThreadTest {
final TaskManager taskManager = mock(TaskManager.class); final TaskManager taskManager = mock(TaskManager.class);
final StreamsMetricsImpl streamsMetrics = final StreamsMetricsImpl streamsMetrics =
new StreamsMetricsImpl(metrics, CLIENT_ID, mockTime); new StreamsMetricsImpl(metrics, CLIENT_ID, PROCESS_ID.toString(), mockTime);
final TopologyMetadata topologyMetadata = new TopologyMetadata(internalTopologyBuilder, config); final TopologyMetadata topologyMetadata = new TopologyMetadata(internalTopologyBuilder, config);
topologyMetadata.buildAndRewriteTopology(); topologyMetadata.buildAndRewriteTopology();
thread = new StreamThread( thread = new StreamThread(
@ -3157,7 +3161,7 @@ public class StreamThreadTest {
when(consumerGroupMetadata.groupInstanceId()).thenReturn(Optional.empty()); when(consumerGroupMetadata.groupInstanceId()).thenReturn(Optional.empty());
final TaskManager taskManager = mock(TaskManager.class); final TaskManager taskManager = mock(TaskManager.class);
final StreamsMetricsImpl streamsMetrics = final StreamsMetricsImpl streamsMetrics =
new StreamsMetricsImpl(metrics, CLIENT_ID, mockTime); new StreamsMetricsImpl(metrics, CLIENT_ID, PROCESS_ID.toString(), mockTime);
final TopologyMetadata topologyMetadata = new TopologyMetadata(internalTopologyBuilder, config); final TopologyMetadata topologyMetadata = new TopologyMetadata(internalTopologyBuilder, config);
topologyMetadata.buildAndRewriteTopology(); topologyMetadata.buildAndRewriteTopology();
thread = new StreamThread( thread = new StreamThread(
@ -3549,7 +3553,7 @@ public class StreamThreadTest {
"", "",
taskManager, taskManager,
null, null,
new StreamsMetricsImpl(metrics, CLIENT_ID, mockTime), new StreamsMetricsImpl(metrics, CLIENT_ID, PROCESS_ID.toString(), mockTime),
topologyMetadata, topologyMetadata,
"thread-id", "thread-id",
new LogContext(), new LogContext(),
@ -3599,7 +3603,7 @@ public class StreamThreadTest {
final LogContext logContext = new LogContext("test"); final LogContext logContext = new LogContext("test");
final Logger log = logContext.logger(StreamThreadTest.class); final Logger log = logContext.logger(StreamThreadTest.class);
final StreamsMetricsImpl streamsMetrics = final StreamsMetricsImpl streamsMetrics =
new StreamsMetricsImpl(metrics, CLIENT_ID, mockTime); new StreamsMetricsImpl(metrics, CLIENT_ID, PROCESS_ID.toString(), mockTime);
final StandbyTaskCreator standbyTaskCreator = new StandbyTaskCreator( final StandbyTaskCreator standbyTaskCreator = new StandbyTaskCreator(
new TopologyMetadata(internalTopologyBuilder, config), new TopologyMetadata(internalTopologyBuilder, config),
config, config,
@ -3658,7 +3662,7 @@ public class StreamThreadTest {
final StreamsConfig config, final StreamsConfig config,
final TopologyMetadata topologyMetadata) { final TopologyMetadata topologyMetadata) {
final StreamsMetricsImpl streamsMetrics = final StreamsMetricsImpl streamsMetrics =
new StreamsMetricsImpl(metrics, CLIENT_ID, mockTime); new StreamsMetricsImpl(metrics, CLIENT_ID, PROCESS_ID.toString(), mockTime);
return new StreamThread( return new StreamThread(
mockTime, mockTime,

View File

@ -54,6 +54,7 @@ import static org.apache.kafka.streams.processor.internals.metrics.StreamsMetric
import static org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl.LATENCY_SUFFIX; import static org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl.LATENCY_SUFFIX;
import static org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl.MAX_SUFFIX; import static org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl.MAX_SUFFIX;
import static org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl.PROCESSOR_NODE_LEVEL_GROUP; import static org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl.PROCESSOR_NODE_LEVEL_GROUP;
import static org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl.PROCESS_ID_TAG;
import static org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl.RATE_SUFFIX; import static org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl.RATE_SUFFIX;
import static org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl.ROLLUP_VALUE; import static org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl.ROLLUP_VALUE;
import static org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl.STATE_STORE_LEVEL_GROUP; import static org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl.STATE_STORE_LEVEL_GROUP;
@ -93,6 +94,7 @@ public class StreamsMetricsImplTest {
private static final String SENSOR_NAME_2 = "sensor2"; private static final String SENSOR_NAME_2 = "sensor2";
private static final String INTERNAL_PREFIX = "internal"; private static final String INTERNAL_PREFIX = "internal";
private static final String CLIENT_ID = "test-client"; private static final String CLIENT_ID = "test-client";
private static final String PROCESS_ID = "test-process";
private static final String THREAD_ID1 = "test-thread-1"; private static final String THREAD_ID1 = "test-thread-1";
private static final String TASK_ID1 = "test-task-1"; private static final String TASK_ID1 = "test-task-1";
private static final String TASK_ID2 = "test-task-2"; private static final String TASK_ID2 = "test-task-2";
@ -131,13 +133,13 @@ public class StreamsMetricsImplTest {
private final String metricNamePrefix = "metric"; private final String metricNamePrefix = "metric";
private final String group = "group"; private final String group = "group";
private final Map<String, String> tags = mkMap(mkEntry("tag", "value")); private final Map<String, String> tags = mkMap(mkEntry("tag", "value"));
private final Map<String, String> clientLevelTags = mkMap(mkEntry(CLIENT_ID_TAG, CLIENT_ID)); private final Map<String, String> clientLevelTags = mkMap(mkEntry(CLIENT_ID_TAG, CLIENT_ID), mkEntry(PROCESS_ID_TAG, PROCESS_ID));
private final MetricName metricName1 = private final MetricName metricName1 =
new MetricName(METRIC_NAME1, CLIENT_LEVEL_GROUP, DESCRIPTION1, clientLevelTags); new MetricName(METRIC_NAME1, CLIENT_LEVEL_GROUP, DESCRIPTION1, clientLevelTags);
private final MetricName metricName2 = private final MetricName metricName2 =
new MetricName(METRIC_NAME1, CLIENT_LEVEL_GROUP, DESCRIPTION2, clientLevelTags); new MetricName(METRIC_NAME1, CLIENT_LEVEL_GROUP, DESCRIPTION2, clientLevelTags);
private final MockTime time = new MockTime(0); private final MockTime time = new MockTime(0);
private final StreamsMetricsImpl streamsMetrics = new StreamsMetricsImpl(metrics, CLIENT_ID, time); private final StreamsMetricsImpl streamsMetrics = new StreamsMetricsImpl(metrics, CLIENT_ID, PROCESS_ID, time);
private static MetricConfig eqMetricConfig(final MetricConfig metricConfig) { private static MetricConfig eqMetricConfig(final MetricConfig metricConfig) {
final StringBuffer message = new StringBuffer(); final StringBuffer message = new StringBuffer();
@ -252,7 +254,7 @@ public class StreamsMetricsImplTest {
final Metrics metrics = mock(Metrics.class); final Metrics metrics = mock(Metrics.class);
final RecordingLevel recordingLevel = RecordingLevel.INFO; final RecordingLevel recordingLevel = RecordingLevel.INFO;
setupGetNewSensorTest(metrics, recordingLevel); setupGetNewSensorTest(metrics, recordingLevel);
final StreamsMetricsImpl streamsMetrics = new StreamsMetricsImpl(metrics, CLIENT_ID, time); final StreamsMetricsImpl streamsMetrics = new StreamsMetricsImpl(metrics, CLIENT_ID, PROCESS_ID, time);
final Sensor actualSensor = streamsMetrics.threadLevelSensor(THREAD_ID1, SENSOR_NAME_1, recordingLevel); final Sensor actualSensor = streamsMetrics.threadLevelSensor(THREAD_ID1, SENSOR_NAME_1, recordingLevel);
@ -264,7 +266,7 @@ public class StreamsMetricsImplTest {
final Metrics metrics = mock(Metrics.class); final Metrics metrics = mock(Metrics.class);
final RecordingLevel recordingLevel = RecordingLevel.INFO; final RecordingLevel recordingLevel = RecordingLevel.INFO;
setupGetExistingSensorTest(metrics); setupGetExistingSensorTest(metrics);
final StreamsMetricsImpl streamsMetrics = new StreamsMetricsImpl(metrics, CLIENT_ID, time); final StreamsMetricsImpl streamsMetrics = new StreamsMetricsImpl(metrics, CLIENT_ID, PROCESS_ID, time);
final Sensor actualSensor = streamsMetrics.threadLevelSensor(THREAD_ID1, SENSOR_NAME_1, recordingLevel); final Sensor actualSensor = streamsMetrics.threadLevelSensor(THREAD_ID1, SENSOR_NAME_1, recordingLevel);
@ -276,7 +278,7 @@ public class StreamsMetricsImplTest {
final Metrics metrics = mock(Metrics.class); final Metrics metrics = mock(Metrics.class);
final RecordingLevel recordingLevel = RecordingLevel.INFO; final RecordingLevel recordingLevel = RecordingLevel.INFO;
setupGetNewSensorTest(metrics, recordingLevel); setupGetNewSensorTest(metrics, recordingLevel);
final StreamsMetricsImpl streamsMetrics = new StreamsMetricsImpl(metrics, CLIENT_ID, time); final StreamsMetricsImpl streamsMetrics = new StreamsMetricsImpl(metrics, CLIENT_ID, PROCESS_ID, time);
final Sensor actualSensor = streamsMetrics.taskLevelSensor( final Sensor actualSensor = streamsMetrics.taskLevelSensor(
THREAD_ID1, THREAD_ID1,
@ -293,7 +295,7 @@ public class StreamsMetricsImplTest {
final Metrics metrics = mock(Metrics.class); final Metrics metrics = mock(Metrics.class);
final RecordingLevel recordingLevel = RecordingLevel.INFO; final RecordingLevel recordingLevel = RecordingLevel.INFO;
setupGetExistingSensorTest(metrics); setupGetExistingSensorTest(metrics);
final StreamsMetricsImpl streamsMetrics = new StreamsMetricsImpl(metrics, CLIENT_ID, time); final StreamsMetricsImpl streamsMetrics = new StreamsMetricsImpl(metrics, CLIENT_ID, PROCESS_ID, time);
final Sensor actualSensor = streamsMetrics.taskLevelSensor( final Sensor actualSensor = streamsMetrics.taskLevelSensor(
THREAD_ID1, THREAD_ID1,
@ -310,7 +312,7 @@ public class StreamsMetricsImplTest {
final Metrics metrics = mock(Metrics.class); final Metrics metrics = mock(Metrics.class);
final RecordingLevel recordingLevel = RecordingLevel.INFO; final RecordingLevel recordingLevel = RecordingLevel.INFO;
setupGetNewSensorTest(metrics, recordingLevel); setupGetNewSensorTest(metrics, recordingLevel);
final StreamsMetricsImpl streamsMetrics = new StreamsMetricsImpl(metrics, CLIENT_ID, time); final StreamsMetricsImpl streamsMetrics = new StreamsMetricsImpl(metrics, CLIENT_ID, PROCESS_ID, time);
final Sensor actualSensor = streamsMetrics.topicLevelSensor( final Sensor actualSensor = streamsMetrics.topicLevelSensor(
THREAD_ID1, THREAD_ID1,
@ -329,7 +331,7 @@ public class StreamsMetricsImplTest {
final Metrics metrics = mock(Metrics.class); final Metrics metrics = mock(Metrics.class);
final RecordingLevel recordingLevel = RecordingLevel.INFO; final RecordingLevel recordingLevel = RecordingLevel.INFO;
setupGetExistingSensorTest(metrics); setupGetExistingSensorTest(metrics);
final StreamsMetricsImpl streamsMetrics = new StreamsMetricsImpl(metrics, CLIENT_ID, time); final StreamsMetricsImpl streamsMetrics = new StreamsMetricsImpl(metrics, CLIENT_ID, PROCESS_ID, time);
final Sensor actualSensor = streamsMetrics.topicLevelSensor( final Sensor actualSensor = streamsMetrics.topicLevelSensor(
THREAD_ID1, THREAD_ID1,
@ -348,7 +350,7 @@ public class StreamsMetricsImplTest {
final Metrics metrics = mock(Metrics.class); final Metrics metrics = mock(Metrics.class);
final RecordingLevel recordingLevel = RecordingLevel.INFO; final RecordingLevel recordingLevel = RecordingLevel.INFO;
final ArgumentCaptor<String> sensorKeys = setupGetNewSensorTest(metrics, recordingLevel); final ArgumentCaptor<String> sensorKeys = setupGetNewSensorTest(metrics, recordingLevel);
final StreamsMetricsImpl streamsMetrics = new StreamsMetricsImpl(metrics, CLIENT_ID, time); final StreamsMetricsImpl streamsMetrics = new StreamsMetricsImpl(metrics, CLIENT_ID, PROCESS_ID, time);
final Sensor actualSensor = streamsMetrics.storeLevelSensor( final Sensor actualSensor = streamsMetrics.storeLevelSensor(
TASK_ID1, TASK_ID1,
@ -366,7 +368,7 @@ public class StreamsMetricsImplTest {
final Metrics metrics = mock(Metrics.class); final Metrics metrics = mock(Metrics.class);
final RecordingLevel recordingLevel = RecordingLevel.INFO; final RecordingLevel recordingLevel = RecordingLevel.INFO;
setupGetExistingSensorTest(metrics); setupGetExistingSensorTest(metrics);
final StreamsMetricsImpl streamsMetrics = new StreamsMetricsImpl(metrics, CLIENT_ID, time); final StreamsMetricsImpl streamsMetrics = new StreamsMetricsImpl(metrics, CLIENT_ID, PROCESS_ID, time);
final Sensor actualSensor = streamsMetrics.storeLevelSensor( final Sensor actualSensor = streamsMetrics.storeLevelSensor(
TASK_ID1, TASK_ID1,
@ -382,7 +384,7 @@ public class StreamsMetricsImplTest {
public void shouldUseSameStoreLevelSensorKeyWithTwoDifferentSensorNames() { public void shouldUseSameStoreLevelSensorKeyWithTwoDifferentSensorNames() {
final Metrics metrics = mock(Metrics.class); final Metrics metrics = mock(Metrics.class);
final ArgumentCaptor<String> sensorKeys = setUpSensorKeyTests(metrics); final ArgumentCaptor<String> sensorKeys = setUpSensorKeyTests(metrics);
final StreamsMetricsImpl streamsMetrics = new StreamsMetricsImpl(metrics, CLIENT_ID, time); final StreamsMetricsImpl streamsMetrics = new StreamsMetricsImpl(metrics, CLIENT_ID, PROCESS_ID, time);
streamsMetrics.storeLevelSensor(TASK_ID1, STORE_NAME1, SENSOR_NAME_1, INFO_RECORDING_LEVEL); streamsMetrics.storeLevelSensor(TASK_ID1, STORE_NAME1, SENSOR_NAME_1, INFO_RECORDING_LEVEL);
streamsMetrics.storeLevelSensor(TASK_ID1, STORE_NAME1, SENSOR_NAME_2, INFO_RECORDING_LEVEL); streamsMetrics.storeLevelSensor(TASK_ID1, STORE_NAME1, SENSOR_NAME_2, INFO_RECORDING_LEVEL);
@ -394,7 +396,7 @@ public class StreamsMetricsImplTest {
public void shouldNotUseSameStoreLevelSensorKeyWithDifferentTaskIds() { public void shouldNotUseSameStoreLevelSensorKeyWithDifferentTaskIds() {
final Metrics metrics = mock(Metrics.class); final Metrics metrics = mock(Metrics.class);
final ArgumentCaptor<String> sensorKeys = setUpSensorKeyTests(metrics); final ArgumentCaptor<String> sensorKeys = setUpSensorKeyTests(metrics);
final StreamsMetricsImpl streamsMetrics = new StreamsMetricsImpl(metrics, CLIENT_ID, time); final StreamsMetricsImpl streamsMetrics = new StreamsMetricsImpl(metrics, CLIENT_ID, PROCESS_ID, time);
streamsMetrics.storeLevelSensor(TASK_ID1, STORE_NAME1, SENSOR_NAME_1, INFO_RECORDING_LEVEL); streamsMetrics.storeLevelSensor(TASK_ID1, STORE_NAME1, SENSOR_NAME_1, INFO_RECORDING_LEVEL);
streamsMetrics.storeLevelSensor(TASK_ID2, STORE_NAME1, SENSOR_NAME_1, INFO_RECORDING_LEVEL); streamsMetrics.storeLevelSensor(TASK_ID2, STORE_NAME1, SENSOR_NAME_1, INFO_RECORDING_LEVEL);
@ -406,7 +408,7 @@ public class StreamsMetricsImplTest {
public void shouldNotUseSameStoreLevelSensorKeyWithDifferentStoreNames() { public void shouldNotUseSameStoreLevelSensorKeyWithDifferentStoreNames() {
final Metrics metrics = mock(Metrics.class); final Metrics metrics = mock(Metrics.class);
final ArgumentCaptor<String> sensorKeys = setUpSensorKeyTests(metrics); final ArgumentCaptor<String> sensorKeys = setUpSensorKeyTests(metrics);
final StreamsMetricsImpl streamsMetrics = new StreamsMetricsImpl(metrics, CLIENT_ID, time); final StreamsMetricsImpl streamsMetrics = new StreamsMetricsImpl(metrics, CLIENT_ID, PROCESS_ID, time);
streamsMetrics.storeLevelSensor(TASK_ID1, STORE_NAME1, SENSOR_NAME_1, INFO_RECORDING_LEVEL); streamsMetrics.storeLevelSensor(TASK_ID1, STORE_NAME1, SENSOR_NAME_1, INFO_RECORDING_LEVEL);
streamsMetrics.storeLevelSensor(TASK_ID1, STORE_NAME2, SENSOR_NAME_1, INFO_RECORDING_LEVEL); streamsMetrics.storeLevelSensor(TASK_ID1, STORE_NAME2, SENSOR_NAME_1, INFO_RECORDING_LEVEL);
@ -418,7 +420,7 @@ public class StreamsMetricsImplTest {
public void shouldNotUseSameStoreLevelSensorKeyWithDifferentThreadIds() throws InterruptedException { public void shouldNotUseSameStoreLevelSensorKeyWithDifferentThreadIds() throws InterruptedException {
final Metrics metrics = mock(Metrics.class); final Metrics metrics = mock(Metrics.class);
final ArgumentCaptor<String> sensorKeys = setUpSensorKeyTests(metrics); final ArgumentCaptor<String> sensorKeys = setUpSensorKeyTests(metrics);
final StreamsMetricsImpl streamsMetrics = new StreamsMetricsImpl(metrics, CLIENT_ID, time); final StreamsMetricsImpl streamsMetrics = new StreamsMetricsImpl(metrics, CLIENT_ID, PROCESS_ID, time);
streamsMetrics.storeLevelSensor(TASK_ID1, STORE_NAME1, SENSOR_NAME_1, INFO_RECORDING_LEVEL); streamsMetrics.storeLevelSensor(TASK_ID1, STORE_NAME1, SENSOR_NAME_1, INFO_RECORDING_LEVEL);
final Thread otherThread = final Thread otherThread =
@ -433,7 +435,7 @@ public class StreamsMetricsImplTest {
public void shouldUseSameStoreLevelSensorKeyWithSameSensorNames() { public void shouldUseSameStoreLevelSensorKeyWithSameSensorNames() {
final Metrics metrics = mock(Metrics.class); final Metrics metrics = mock(Metrics.class);
final ArgumentCaptor<String> sensorKeys = setUpSensorKeyTests(metrics); final ArgumentCaptor<String> sensorKeys = setUpSensorKeyTests(metrics);
final StreamsMetricsImpl streamsMetrics = new StreamsMetricsImpl(metrics, CLIENT_ID, time); final StreamsMetricsImpl streamsMetrics = new StreamsMetricsImpl(metrics, CLIENT_ID, PROCESS_ID, time);
streamsMetrics.storeLevelSensor(TASK_ID1, STORE_NAME1, SENSOR_NAME_1, INFO_RECORDING_LEVEL); streamsMetrics.storeLevelSensor(TASK_ID1, STORE_NAME1, SENSOR_NAME_1, INFO_RECORDING_LEVEL);
streamsMetrics.storeLevelSensor(TASK_ID1, STORE_NAME1, SENSOR_NAME_1, INFO_RECORDING_LEVEL); streamsMetrics.storeLevelSensor(TASK_ID1, STORE_NAME1, SENSOR_NAME_1, INFO_RECORDING_LEVEL);
@ -457,7 +459,7 @@ public class StreamsMetricsImplTest {
.thenReturn(metricName); .thenReturn(metricName);
when(metrics.metric(metricName)).thenReturn(null); when(metrics.metric(metricName)).thenReturn(null);
when(metrics.addMetricIfAbsent(eq(metricName), eqMetricConfig(metricConfig), eq(VALUE_PROVIDER))).thenReturn(null); when(metrics.addMetricIfAbsent(eq(metricName), eqMetricConfig(metricConfig), eq(VALUE_PROVIDER))).thenReturn(null);
final StreamsMetricsImpl streamsMetrics = new StreamsMetricsImpl(metrics, CLIENT_ID, time); final StreamsMetricsImpl streamsMetrics = new StreamsMetricsImpl(metrics, CLIENT_ID, PROCESS_ID, time);
streamsMetrics.addStoreLevelMutableMetric( streamsMetrics.addStoreLevelMutableMetric(
TASK_ID1, TASK_ID1,
@ -489,7 +491,7 @@ public class StreamsMetricsImplTest {
when(metrics.metricName(METRIC_NAME1, STATE_STORE_LEVEL_GROUP, DESCRIPTION1, STORE_LEVEL_TAG_MAP)) when(metrics.metricName(METRIC_NAME1, STATE_STORE_LEVEL_GROUP, DESCRIPTION1, STORE_LEVEL_TAG_MAP))
.thenReturn(metricName); .thenReturn(metricName);
when(metrics.metric(metricName)).thenReturn(null); when(metrics.metric(metricName)).thenReturn(null);
final StreamsMetricsImpl streamsMetrics = new StreamsMetricsImpl(metrics, CLIENT_ID, time); final StreamsMetricsImpl streamsMetrics = new StreamsMetricsImpl(metrics, CLIENT_ID, PROCESS_ID, time);
streamsMetrics.addStoreLevelMutableMetric( streamsMetrics.addStoreLevelMutableMetric(
TASK_ID1, TASK_ID1,
@ -537,7 +539,7 @@ public class StreamsMetricsImplTest {
@Test @Test
public void shouldRemoveStateStoreLevelSensors() { public void shouldRemoveStateStoreLevelSensors() {
final Metrics metrics = mock(Metrics.class); final Metrics metrics = mock(Metrics.class);
final StreamsMetricsImpl streamsMetrics = new StreamsMetricsImpl(metrics, CLIENT_ID, time); final StreamsMetricsImpl streamsMetrics = new StreamsMetricsImpl(metrics, CLIENT_ID, PROCESS_ID, time);
final MetricName metricName1 = final MetricName metricName1 =
new MetricName(METRIC_NAME1, STATE_STORE_LEVEL_GROUP, DESCRIPTION1, STORE_LEVEL_TAG_MAP); new MetricName(METRIC_NAME1, STATE_STORE_LEVEL_GROUP, DESCRIPTION1, STORE_LEVEL_TAG_MAP);
final MetricName metricName2 = final MetricName metricName2 =
@ -560,7 +562,7 @@ public class StreamsMetricsImplTest {
final Metrics metrics = mock(Metrics.class); final Metrics metrics = mock(Metrics.class);
final RecordingLevel recordingLevel = RecordingLevel.INFO; final RecordingLevel recordingLevel = RecordingLevel.INFO;
setupGetNewSensorTest(metrics, recordingLevel); setupGetNewSensorTest(metrics, recordingLevel);
final StreamsMetricsImpl streamsMetrics = new StreamsMetricsImpl(metrics, CLIENT_ID, time); final StreamsMetricsImpl streamsMetrics = new StreamsMetricsImpl(metrics, CLIENT_ID, PROCESS_ID, time);
final Sensor actualSensor = streamsMetrics.nodeLevelSensor( final Sensor actualSensor = streamsMetrics.nodeLevelSensor(
THREAD_ID1, THREAD_ID1,
@ -578,7 +580,7 @@ public class StreamsMetricsImplTest {
final Metrics metrics = mock(Metrics.class); final Metrics metrics = mock(Metrics.class);
final RecordingLevel recordingLevel = RecordingLevel.INFO; final RecordingLevel recordingLevel = RecordingLevel.INFO;
setupGetExistingSensorTest(metrics); setupGetExistingSensorTest(metrics);
final StreamsMetricsImpl streamsMetrics = new StreamsMetricsImpl(metrics, CLIENT_ID, time); final StreamsMetricsImpl streamsMetrics = new StreamsMetricsImpl(metrics, CLIENT_ID, PROCESS_ID, time);
final Sensor actualSensor = streamsMetrics.nodeLevelSensor( final Sensor actualSensor = streamsMetrics.nodeLevelSensor(
THREAD_ID1, THREAD_ID1,
@ -597,7 +599,7 @@ public class StreamsMetricsImplTest {
final RecordingLevel recordingLevel = RecordingLevel.INFO; final RecordingLevel recordingLevel = RecordingLevel.INFO;
final String processorCacheName = "processorNodeName"; final String processorCacheName = "processorNodeName";
setupGetNewSensorTest(metrics, recordingLevel); setupGetNewSensorTest(metrics, recordingLevel);
final StreamsMetricsImpl streamsMetrics = new StreamsMetricsImpl(metrics, CLIENT_ID, time); final StreamsMetricsImpl streamsMetrics = new StreamsMetricsImpl(metrics, CLIENT_ID, PROCESS_ID, time);
final Sensor actualSensor = streamsMetrics.cacheLevelSensor( final Sensor actualSensor = streamsMetrics.cacheLevelSensor(
THREAD_ID1, THREAD_ID1,
@ -616,7 +618,7 @@ public class StreamsMetricsImplTest {
final RecordingLevel recordingLevel = RecordingLevel.INFO; final RecordingLevel recordingLevel = RecordingLevel.INFO;
final String processorCacheName = "processorNodeName"; final String processorCacheName = "processorNodeName";
setupGetExistingSensorTest(metrics); setupGetExistingSensorTest(metrics);
final StreamsMetricsImpl streamsMetrics = new StreamsMetricsImpl(metrics, CLIENT_ID, time); final StreamsMetricsImpl streamsMetrics = new StreamsMetricsImpl(metrics, CLIENT_ID, PROCESS_ID, time);
final Sensor actualSensor = streamsMetrics.cacheLevelSensor( final Sensor actualSensor = streamsMetrics.cacheLevelSensor(
THREAD_ID1, TASK_ID1, THREAD_ID1, TASK_ID1,
@ -633,7 +635,7 @@ public class StreamsMetricsImplTest {
final Metrics metrics = mock(Metrics.class); final Metrics metrics = mock(Metrics.class);
final RecordingLevel recordingLevel = RecordingLevel.INFO; final RecordingLevel recordingLevel = RecordingLevel.INFO;
setupGetNewSensorTest(metrics, recordingLevel); setupGetNewSensorTest(metrics, recordingLevel);
final StreamsMetricsImpl streamsMetrics = new StreamsMetricsImpl(metrics, CLIENT_ID, time); final StreamsMetricsImpl streamsMetrics = new StreamsMetricsImpl(metrics, CLIENT_ID, PROCESS_ID, time);
final Sensor actualSensor = streamsMetrics.clientLevelSensor(SENSOR_NAME_1, recordingLevel); final Sensor actualSensor = streamsMetrics.clientLevelSensor(SENSOR_NAME_1, recordingLevel);
@ -645,7 +647,7 @@ public class StreamsMetricsImplTest {
final Metrics metrics = mock(Metrics.class); final Metrics metrics = mock(Metrics.class);
final RecordingLevel recordingLevel = RecordingLevel.INFO; final RecordingLevel recordingLevel = RecordingLevel.INFO;
setupGetExistingSensorTest(metrics); setupGetExistingSensorTest(metrics);
final StreamsMetricsImpl streamsMetrics = new StreamsMetricsImpl(metrics, CLIENT_ID, time); final StreamsMetricsImpl streamsMetrics = new StreamsMetricsImpl(metrics, CLIENT_ID, PROCESS_ID, time);
final Sensor actualSensor = streamsMetrics.clientLevelSensor(SENSOR_NAME_1, recordingLevel); final Sensor actualSensor = streamsMetrics.clientLevelSensor(SENSOR_NAME_1, recordingLevel);
@ -662,7 +664,7 @@ public class StreamsMetricsImplTest {
when(metrics.metricName(METRIC_NAME1, CLIENT_LEVEL_GROUP, DESCRIPTION1, clientLevelTags)) when(metrics.metricName(METRIC_NAME1, CLIENT_LEVEL_GROUP, DESCRIPTION1, clientLevelTags))
.thenReturn(metricName1); .thenReturn(metricName1);
doNothing().when(metrics).addMetric(eq(metricName1), eqMetricConfig(metricConfig), eq(immutableValue)); doNothing().when(metrics).addMetric(eq(metricName1), eqMetricConfig(metricConfig), eq(immutableValue));
final StreamsMetricsImpl streamsMetrics = new StreamsMetricsImpl(metrics, CLIENT_ID, time); final StreamsMetricsImpl streamsMetrics = new StreamsMetricsImpl(metrics, CLIENT_ID, PROCESS_ID, time);
streamsMetrics.addClientLevelImmutableMetric(METRIC_NAME1, DESCRIPTION1, recordingLevel, value); streamsMetrics.addClientLevelImmutableMetric(METRIC_NAME1, DESCRIPTION1, recordingLevel, value);
} }
@ -676,7 +678,7 @@ public class StreamsMetricsImplTest {
when(metrics.metricName(METRIC_NAME1, CLIENT_LEVEL_GROUP, DESCRIPTION1, clientLevelTags)) when(metrics.metricName(METRIC_NAME1, CLIENT_LEVEL_GROUP, DESCRIPTION1, clientLevelTags))
.thenReturn(metricName1); .thenReturn(metricName1);
doNothing().when(metrics).addMetric(eq(metricName1), eqMetricConfig(metricConfig), eq(valueProvider)); doNothing().when(metrics).addMetric(eq(metricName1), eqMetricConfig(metricConfig), eq(valueProvider));
final StreamsMetricsImpl streamsMetrics = new StreamsMetricsImpl(metrics, CLIENT_ID, time); final StreamsMetricsImpl streamsMetrics = new StreamsMetricsImpl(metrics, CLIENT_ID, PROCESS_ID, time);
streamsMetrics.addClientLevelMutableMetric(METRIC_NAME1, DESCRIPTION1, recordingLevel, valueProvider); streamsMetrics.addClientLevelMutableMetric(METRIC_NAME1, DESCRIPTION1, recordingLevel, valueProvider);
} }
@ -697,7 +699,7 @@ public class StreamsMetricsImplTest {
@Test @Test
public void shouldRemoveClientLevelMetricsAndSensors() { public void shouldRemoveClientLevelMetricsAndSensors() {
final Metrics metrics = mock(Metrics.class); final Metrics metrics = mock(Metrics.class);
final StreamsMetricsImpl streamsMetrics = new StreamsMetricsImpl(metrics, CLIENT_ID, time); final StreamsMetricsImpl streamsMetrics = new StreamsMetricsImpl(metrics, CLIENT_ID, PROCESS_ID, time);
final ArgumentCaptor<String> sensorKeys = addSensorsOnAllLevels(metrics, streamsMetrics); final ArgumentCaptor<String> sensorKeys = addSensorsOnAllLevels(metrics, streamsMetrics);
doNothing().when(metrics).removeSensor(sensorKeys.getAllValues().get(0)); doNothing().when(metrics).removeSensor(sensorKeys.getAllValues().get(0));
@ -710,7 +712,7 @@ public class StreamsMetricsImplTest {
@Test @Test
public void shouldRemoveThreadLevelSensors() { public void shouldRemoveThreadLevelSensors() {
final Metrics metrics = mock(Metrics.class); final Metrics metrics = mock(Metrics.class);
final StreamsMetricsImpl streamsMetrics = new StreamsMetricsImpl(metrics, CLIENT_ID, time); final StreamsMetricsImpl streamsMetrics = new StreamsMetricsImpl(metrics, CLIENT_ID, PROCESS_ID, time);
addSensorsOnAllLevels(metrics, streamsMetrics); addSensorsOnAllLevels(metrics, streamsMetrics);
setupRemoveSensorsTest(metrics, THREAD_ID1); setupRemoveSensorsTest(metrics, THREAD_ID1);
@ -719,7 +721,7 @@ public class StreamsMetricsImplTest {
@Test @Test
public void testNullMetrics() { public void testNullMetrics() {
assertThrows(NullPointerException.class, () -> new StreamsMetricsImpl(null, "", time)); assertThrows(NullPointerException.class, () -> new StreamsMetricsImpl(null, "", PROCESS_ID, time));
} }
@Test @Test
@ -752,7 +754,7 @@ public class StreamsMetricsImplTest {
@Test @Test
public void testMultiLevelSensorRemoval() { public void testMultiLevelSensorRemoval() {
final Metrics registry = new Metrics(); final Metrics registry = new Metrics();
final StreamsMetricsImpl metrics = new StreamsMetricsImpl(registry, THREAD_ID1, time); final StreamsMetricsImpl metrics = new StreamsMetricsImpl(registry, THREAD_ID1, PROCESS_ID, time);
for (final MetricName defaultMetric : registry.metrics().keySet()) { for (final MetricName defaultMetric : registry.metrics().keySet()) {
registry.removeMetric(defaultMetric); registry.removeMetric(defaultMetric);
} }
@ -858,7 +860,7 @@ public class StreamsMetricsImplTest {
final MockTime time = new MockTime(1); final MockTime time = new MockTime(1);
final MetricConfig config = new MetricConfig().timeWindow(1, TimeUnit.MILLISECONDS); final MetricConfig config = new MetricConfig().timeWindow(1, TimeUnit.MILLISECONDS);
final Metrics metrics = new Metrics(config, time); final Metrics metrics = new Metrics(config, time);
final StreamsMetricsImpl streamsMetrics = new StreamsMetricsImpl(metrics, "", time); final StreamsMetricsImpl streamsMetrics = new StreamsMetricsImpl(metrics, "", PROCESS_ID, time);
final String scope = "scope"; final String scope = "scope";
final String entity = "entity"; final String entity = "entity";
@ -892,7 +894,7 @@ public class StreamsMetricsImplTest {
@Test @Test
public void shouldAddLatencyRateTotalSensor() { public void shouldAddLatencyRateTotalSensor() {
final StreamsMetricsImpl streamsMetrics = new StreamsMetricsImpl(metrics, CLIENT_ID, time); final StreamsMetricsImpl streamsMetrics = new StreamsMetricsImpl(metrics, CLIENT_ID, PROCESS_ID, time);
shouldAddCustomSensor( shouldAddCustomSensor(
streamsMetrics.addLatencyRateTotalSensor(SCOPE_NAME, ENTITY_NAME, OPERATION_NAME, RecordingLevel.DEBUG), streamsMetrics.addLatencyRateTotalSensor(SCOPE_NAME, ENTITY_NAME, OPERATION_NAME, RecordingLevel.DEBUG),
streamsMetrics, streamsMetrics,
@ -907,7 +909,7 @@ public class StreamsMetricsImplTest {
@Test @Test
public void shouldAddRateTotalSensor() { public void shouldAddRateTotalSensor() {
final StreamsMetricsImpl streamsMetrics = new StreamsMetricsImpl(metrics, CLIENT_ID, time); final StreamsMetricsImpl streamsMetrics = new StreamsMetricsImpl(metrics, CLIENT_ID, PROCESS_ID, time);
shouldAddCustomSensor( shouldAddCustomSensor(
streamsMetrics.addRateTotalSensor(SCOPE_NAME, ENTITY_NAME, OPERATION_NAME, RecordingLevel.DEBUG), streamsMetrics.addRateTotalSensor(SCOPE_NAME, ENTITY_NAME, OPERATION_NAME, RecordingLevel.DEBUG),
streamsMetrics, streamsMetrics,
@ -1033,8 +1035,9 @@ public class StreamsMetricsImplTest {
public void shouldGetClientLevelTagMap() { public void shouldGetClientLevelTagMap() {
final Map<String, String> tagMap = streamsMetrics.clientLevelTagMap(); final Map<String, String> tagMap = streamsMetrics.clientLevelTagMap();
assertThat(tagMap.size(), equalTo(1)); assertThat(tagMap.size(), equalTo(2));
assertThat(tagMap.get(StreamsMetricsImpl.CLIENT_ID_TAG), equalTo(CLIENT_ID)); assertThat(tagMap.get(StreamsMetricsImpl.CLIENT_ID_TAG), equalTo(CLIENT_ID));
assertThat(tagMap.get(StreamsMetricsImpl.PROCESS_ID_TAG), equalTo(PROCESS_ID));
} }
@Test @Test
@ -1042,7 +1045,7 @@ public class StreamsMetricsImplTest {
final String taskName = "test-task"; final String taskName = "test-task";
final String storeType = "remote-window"; final String storeType = "remote-window";
final String storeName = "window-keeper"; final String storeName = "window-keeper";
final StreamsMetricsImpl streamsMetrics = new StreamsMetricsImpl(metrics, THREAD_ID1, time); final StreamsMetricsImpl streamsMetrics = new StreamsMetricsImpl(metrics, THREAD_ID1, PROCESS_ID, time);
final Map<String, String> tagMap = streamsMetrics.storeLevelTagMap(taskName, storeType, storeName); final Map<String, String> tagMap = streamsMetrics.storeLevelTagMap(taskName, storeType, storeName);
@ -1057,7 +1060,7 @@ public class StreamsMetricsImplTest {
@Test @Test
public void shouldGetCacheLevelTagMap() { public void shouldGetCacheLevelTagMap() {
final StreamsMetricsImpl streamsMetrics = final StreamsMetricsImpl streamsMetrics =
new StreamsMetricsImpl(metrics, THREAD_ID1, time); new StreamsMetricsImpl(metrics, THREAD_ID1, PROCESS_ID, time);
final String taskName = "taskName"; final String taskName = "taskName";
final String storeName = "storeName"; final String storeName = "storeName";
@ -1074,7 +1077,7 @@ public class StreamsMetricsImplTest {
@Test @Test
public void shouldGetThreadLevelTagMap() { public void shouldGetThreadLevelTagMap() {
final StreamsMetricsImpl streamsMetrics = new StreamsMetricsImpl(metrics, THREAD_ID1, time); final StreamsMetricsImpl streamsMetrics = new StreamsMetricsImpl(metrics, THREAD_ID1, PROCESS_ID, time);
final Map<String, String> tagMap = streamsMetrics.threadLevelTagMap(THREAD_ID1); final Map<String, String> tagMap = streamsMetrics.threadLevelTagMap(THREAD_ID1);
@ -1207,7 +1210,7 @@ public class StreamsMetricsImplTest {
@Test @Test
public void shouldReturnMetricsVersionCurrent() { public void shouldReturnMetricsVersionCurrent() {
assertThat( assertThat(
new StreamsMetricsImpl(metrics, THREAD_ID1, time).version(), new StreamsMetricsImpl(metrics, THREAD_ID1, PROCESS_ID, time).version(),
equalTo(Version.LATEST) equalTo(Version.LATEST)
); );
} }
@ -1266,7 +1269,7 @@ public class StreamsMetricsImplTest {
public void shouldAddThreadLevelMutableMetric() { public void shouldAddThreadLevelMutableMetric() {
final int measuredValue = 123; final int measuredValue = 123;
final StreamsMetricsImpl streamsMetrics final StreamsMetricsImpl streamsMetrics
= new StreamsMetricsImpl(metrics, THREAD_ID1, time); = new StreamsMetricsImpl(metrics, THREAD_ID1, PROCESS_ID, time);
streamsMetrics.addThreadLevelMutableMetric( streamsMetrics.addThreadLevelMutableMetric(
"foobar", "foobar",
@ -1288,7 +1291,7 @@ public class StreamsMetricsImplTest {
public void shouldCleanupThreadLevelMutableMetric() { public void shouldCleanupThreadLevelMutableMetric() {
final int measuredValue = 123; final int measuredValue = 123;
final StreamsMetricsImpl streamsMetrics final StreamsMetricsImpl streamsMetrics
= new StreamsMetricsImpl(metrics, THREAD_ID1, time); = new StreamsMetricsImpl(metrics, THREAD_ID1, PROCESS_ID, time);
streamsMetrics.addThreadLevelMutableMetric( streamsMetrics.addThreadLevelMutableMetric(
"foobar", "foobar",
"test metric", "test metric",
@ -1310,7 +1313,7 @@ public class StreamsMetricsImplTest {
public void shouldAddThreadLevelImmutableMetric() { public void shouldAddThreadLevelImmutableMetric() {
final int measuredValue = 123; final int measuredValue = 123;
final StreamsMetricsImpl streamsMetrics final StreamsMetricsImpl streamsMetrics
= new StreamsMetricsImpl(metrics, THREAD_ID1, time); = new StreamsMetricsImpl(metrics, THREAD_ID1, PROCESS_ID, time);
streamsMetrics.addThreadLevelImmutableMetric( streamsMetrics.addThreadLevelImmutableMetric(
"foobar", "foobar",
@ -1332,7 +1335,7 @@ public class StreamsMetricsImplTest {
public void shouldCleanupThreadLevelImmutableMetric() { public void shouldCleanupThreadLevelImmutableMetric() {
final int measuredValue = 123; final int measuredValue = 123;
final StreamsMetricsImpl streamsMetrics final StreamsMetricsImpl streamsMetrics
= new StreamsMetricsImpl(metrics, THREAD_ID1, time); = new StreamsMetricsImpl(metrics, THREAD_ID1, PROCESS_ID, time);
streamsMetrics.addThreadLevelImmutableMetric( streamsMetrics.addThreadLevelImmutableMetric(
"foobar", "foobar",
"test metric", "test metric",

View File

@ -1415,7 +1415,7 @@ public abstract class AbstractDualSchemaRocksDBSegmentedBytesStoreTest {
dir, dir,
Serdes.String(), Serdes.String(),
Serdes.String(), Serdes.String(),
new StreamsMetricsImpl(new Metrics(), "mock", new MockTime()), new StreamsMetricsImpl(new Metrics(), "mock", "processId", new MockTime()),
new StreamsConfig(props), new StreamsConfig(props),
MockRecordCollector::new, MockRecordCollector::new,
new ThreadCache(new LogContext("testCache "), 0, new MockStreamsMetrics(new Metrics())), new ThreadCache(new LogContext("testCache "), 0, new MockStreamsMetrics(new Metrics())),
@ -1451,7 +1451,7 @@ public abstract class AbstractDualSchemaRocksDBSegmentedBytesStoreTest {
dir, dir,
Serdes.String(), Serdes.String(),
Serdes.String(), Serdes.String(),
new StreamsMetricsImpl(new Metrics(), "mock", new MockTime()), new StreamsMetricsImpl(new Metrics(), "mock", "processId", new MockTime()),
new StreamsConfig(props), new StreamsConfig(props),
MockRecordCollector::new, MockRecordCollector::new,
new ThreadCache(new LogContext("testCache "), 0, new MockStreamsMetrics(new Metrics())), new ThreadCache(new LogContext("testCache "), 0, new MockStreamsMetrics(new Metrics())),
@ -1490,7 +1490,7 @@ public abstract class AbstractDualSchemaRocksDBSegmentedBytesStoreTest {
dir, dir,
Serdes.String(), Serdes.String(),
Serdes.String(), Serdes.String(),
new StreamsMetricsImpl(new Metrics(), "mock", new MockTime()), new StreamsMetricsImpl(new Metrics(), "mock", "processId", new MockTime()),
new StreamsConfig(props), new StreamsConfig(props),
MockRecordCollector::new, MockRecordCollector::new,
new ThreadCache(new LogContext("testCache "), 0, new MockStreamsMetrics(new Metrics())), new ThreadCache(new LogContext("testCache "), 0, new MockStreamsMetrics(new Metrics())),
@ -1531,7 +1531,7 @@ public abstract class AbstractDualSchemaRocksDBSegmentedBytesStoreTest {
dir, dir,
Serdes.String(), Serdes.String(),
Serdes.String(), Serdes.String(),
new StreamsMetricsImpl(new Metrics(), "mock", new MockTime()), new StreamsMetricsImpl(new Metrics(), "mock", "processId", new MockTime()),
new StreamsConfig(props), new StreamsConfig(props),
MockRecordCollector::new, MockRecordCollector::new,
new ThreadCache(new LogContext("testCache "), 0, new MockStreamsMetrics(new Metrics())), new ThreadCache(new LogContext("testCache "), 0, new MockStreamsMetrics(new Metrics())),

View File

@ -572,7 +572,7 @@ public abstract class AbstractRocksDBSegmentedBytesStoreTest<S extends Segment>
dir, dir,
Serdes.String(), Serdes.String(),
Serdes.String(), Serdes.String(),
new StreamsMetricsImpl(new Metrics(), "mock", new MockTime()), new StreamsMetricsImpl(new Metrics(), "mock", "processId", new MockTime()),
new StreamsConfig(props), new StreamsConfig(props),
MockRecordCollector::new, MockRecordCollector::new,
new ThreadCache(new LogContext("testCache "), 0, new MockStreamsMetrics(new Metrics())), new ThreadCache(new LogContext("testCache "), 0, new MockStreamsMetrics(new Metrics())),
@ -612,7 +612,7 @@ public abstract class AbstractRocksDBSegmentedBytesStoreTest<S extends Segment>
dir, dir,
Serdes.String(), Serdes.String(),
Serdes.String(), Serdes.String(),
new StreamsMetricsImpl(new Metrics(), "mock", new MockTime()), new StreamsMetricsImpl(new Metrics(), "mock", "processId", new MockTime()),
new StreamsConfig(props), new StreamsConfig(props),
MockRecordCollector::new, MockRecordCollector::new,
new ThreadCache(new LogContext("testCache "), 0, new MockStreamsMetrics(new Metrics())), new ThreadCache(new LogContext("testCache "), 0, new MockStreamsMetrics(new Metrics())),
@ -654,7 +654,7 @@ public abstract class AbstractRocksDBSegmentedBytesStoreTest<S extends Segment>
dir, dir,
Serdes.String(), Serdes.String(),
Serdes.String(), Serdes.String(),
new StreamsMetricsImpl(new Metrics(), "mock", new MockTime()), new StreamsMetricsImpl(new Metrics(), "mock", "processId", new MockTime()),
new StreamsConfig(props), new StreamsConfig(props),
MockRecordCollector::new, MockRecordCollector::new,
new ThreadCache(new LogContext("testCache "), 0, new MockStreamsMetrics(new Metrics())), new ThreadCache(new LogContext("testCache "), 0, new MockStreamsMetrics(new Metrics())),
@ -698,7 +698,7 @@ public abstract class AbstractRocksDBSegmentedBytesStoreTest<S extends Segment>
dir, dir,
Serdes.String(), Serdes.String(),
Serdes.String(), Serdes.String(),
new StreamsMetricsImpl(new Metrics(), "mock", new MockTime()), new StreamsMetricsImpl(new Metrics(), "mock", "processId", new MockTime()),
new StreamsConfig(props), new StreamsConfig(props),
MockRecordCollector::new, MockRecordCollector::new,
new ThreadCache(new LogContext("testCache "), 0, new MockStreamsMetrics(new Metrics())), new ThreadCache(new LogContext("testCache "), 0, new MockStreamsMetrics(new Metrics())),

View File

@ -98,7 +98,7 @@ public class ChangeLoggingKeyValueBytesStoreTest {
TestUtils.tempDirectory(), TestUtils.tempDirectory(),
Serdes.String(), Serdes.String(),
Serdes.Long(), Serdes.Long(),
new StreamsMetricsImpl(new Metrics(), "mock", new MockTime()), new StreamsMetricsImpl(new Metrics(), "mock", "processId", new MockTime()),
streamsConfig, streamsConfig,
() -> collector, () -> collector,
new ThreadCache(new LogContext("testCache "), 0, new MockStreamsMetrics(new Metrics())), new ThreadCache(new LogContext("testCache "), 0, new MockStreamsMetrics(new Metrics())),

View File

@ -113,7 +113,7 @@ public class GlobalStateStoreProviderTest {
when(mockContext.applicationId()).thenReturn("appId"); when(mockContext.applicationId()).thenReturn("appId");
when(mockContext.metrics()) when(mockContext.metrics())
.thenReturn( .thenReturn(
new StreamsMetricsImpl(new Metrics(), "threadName", new MockTime()) new StreamsMetricsImpl(new Metrics(), "threadName", "processId", new MockTime())
); );
when(mockContext.taskId()).thenReturn(new TaskId(0, 0)); when(mockContext.taskId()).thenReturn(new TaskId(0, 0));
when(mockContext.appConfigs()).thenReturn(CONFIGS); when(mockContext.appConfigs()).thenReturn(CONFIGS);

View File

@ -57,7 +57,7 @@ public class KeyValueSegmentTest {
@BeforeEach @BeforeEach
public void setUp() { public void setUp() {
metricsRecorder.init( metricsRecorder.init(
new StreamsMetricsImpl(new Metrics(), "test-client", new MockTime()), new StreamsMetricsImpl(new Metrics(), "test-client", "processId", new MockTime()),
new TaskId(0, 0) new TaskId(0, 0)
); );
} }

View File

@ -124,7 +124,7 @@ public class MeteredKeyValueStoreTest {
metrics.config().recordLevel(Sensor.RecordingLevel.DEBUG); metrics.config().recordLevel(Sensor.RecordingLevel.DEBUG);
when(context.applicationId()).thenReturn(APPLICATION_ID); when(context.applicationId()).thenReturn(APPLICATION_ID);
when(context.metrics()).thenReturn( when(context.metrics()).thenReturn(
new StreamsMetricsImpl(metrics, "test", mockTime) new StreamsMetricsImpl(metrics, "test", "processId", mockTime)
); );
when(context.taskId()).thenReturn(taskId); when(context.taskId()).thenReturn(taskId);
when(context.changelogFor(STORE_NAME)).thenReturn(CHANGELOG_TOPIC); when(context.changelogFor(STORE_NAME)).thenReturn(CHANGELOG_TOPIC);

View File

@ -127,7 +127,7 @@ public class MeteredSessionStoreTest {
metrics.config().recordLevel(Sensor.RecordingLevel.DEBUG); metrics.config().recordLevel(Sensor.RecordingLevel.DEBUG);
when(context.applicationId()).thenReturn(APPLICATION_ID); when(context.applicationId()).thenReturn(APPLICATION_ID);
when(context.metrics()) when(context.metrics())
.thenReturn(new StreamsMetricsImpl(metrics, "test", mockTime)); .thenReturn(new StreamsMetricsImpl(metrics, "test", "processId", mockTime));
when(context.taskId()).thenReturn(taskId); when(context.taskId()).thenReturn(taskId);
when(context.changelogFor(STORE_NAME)).thenReturn(CHANGELOG_TOPIC); when(context.changelogFor(STORE_NAME)).thenReturn(CHANGELOG_TOPIC);
when(innerStore.name()).thenReturn(STORE_NAME); when(innerStore.name()).thenReturn(STORE_NAME);

View File

@ -128,7 +128,7 @@ public class MeteredTimestampedKeyValueStoreTest {
setUpWithoutContext(); setUpWithoutContext();
when(context.applicationId()).thenReturn(APPLICATION_ID); when(context.applicationId()).thenReturn(APPLICATION_ID);
when(context.metrics()) when(context.metrics())
.thenReturn(new StreamsMetricsImpl(metrics, "test", mockTime)); .thenReturn(new StreamsMetricsImpl(metrics, "test", "processId", mockTime));
when(context.taskId()).thenReturn(taskId); when(context.taskId()).thenReturn(taskId);
when(context.changelogFor(STORE_NAME)).thenReturn(CHANGELOG_TOPIC); when(context.changelogFor(STORE_NAME)).thenReturn(CHANGELOG_TOPIC);
when(inner.name()).thenReturn(STORE_NAME); when(inner.name()).thenReturn(STORE_NAME);

View File

@ -78,7 +78,7 @@ public class MeteredTimestampedWindowStoreTest {
public void setUp() { public void setUp() {
final StreamsMetricsImpl streamsMetrics = final StreamsMetricsImpl streamsMetrics =
new StreamsMetricsImpl(metrics, "test", new MockTime()); new StreamsMetricsImpl(metrics, "test", "processId", new MockTime());
context = new InternalMockProcessorContext<>( context = new InternalMockProcessorContext<>(
TestUtils.tempDirectory(), TestUtils.tempDirectory(),
@ -106,7 +106,7 @@ public class MeteredTimestampedWindowStoreTest {
public void setUpWithoutContextName() { public void setUpWithoutContextName() {
final StreamsMetricsImpl streamsMetrics = final StreamsMetricsImpl streamsMetrics =
new StreamsMetricsImpl(metrics, "test", new MockTime()); new StreamsMetricsImpl(metrics, "test", "processId", new MockTime());
context = new InternalMockProcessorContext<>( context = new InternalMockProcessorContext<>(
TestUtils.tempDirectory(), TestUtils.tempDirectory(),

View File

@ -111,7 +111,7 @@ public class MeteredVersionedKeyValueStoreTest {
@BeforeEach @BeforeEach
public void setUp() { public void setUp() {
when(inner.name()).thenReturn(STORE_NAME); when(inner.name()).thenReturn(STORE_NAME);
when(context.metrics()).thenReturn(new StreamsMetricsImpl(metrics, "test", mockTime)); when(context.metrics()).thenReturn(new StreamsMetricsImpl(metrics, "test", "processId", mockTime));
when(context.applicationId()).thenReturn(APPLICATION_ID); when(context.applicationId()).thenReturn(APPLICATION_ID);
when(context.taskId()).thenReturn(TASK_ID); when(context.taskId()).thenReturn(TASK_ID);

View File

@ -117,7 +117,7 @@ public class MeteredWindowStoreTest {
@BeforeEach @BeforeEach
public void setUp() { public void setUp() {
final StreamsMetricsImpl streamsMetrics = final StreamsMetricsImpl streamsMetrics =
new StreamsMetricsImpl(metrics, "test", new MockTime()); new StreamsMetricsImpl(metrics, "test", "processId", new MockTime());
context = new InternalMockProcessorContext<>( context = new InternalMockProcessorContext<>(
TestUtils.tempDirectory(), TestUtils.tempDirectory(),
Serdes.String(), Serdes.String(),

View File

@ -919,7 +919,7 @@ public class RocksDBStoreTest extends AbstractKeyValueStoreTest {
final Metrics metrics = new Metrics(new MetricConfig().recordLevel(RecordingLevel.DEBUG)); final Metrics metrics = new Metrics(new MetricConfig().recordLevel(RecordingLevel.DEBUG));
final StreamsMetricsImpl streamsMetrics = final StreamsMetricsImpl streamsMetrics =
new StreamsMetricsImpl(metrics, "test-application", time); new StreamsMetricsImpl(metrics, "test-application", "processId", time);
context = mock(InternalMockProcessorContext.class); context = mock(InternalMockProcessorContext.class);
when(context.metrics()).thenReturn(streamsMetrics); when(context.metrics()).thenReturn(streamsMetrics);
@ -952,7 +952,7 @@ public class RocksDBStoreTest extends AbstractKeyValueStoreTest {
final Metrics metrics = new Metrics(new MetricConfig().recordLevel(RecordingLevel.INFO)); final Metrics metrics = new Metrics(new MetricConfig().recordLevel(RecordingLevel.INFO));
final StreamsMetricsImpl streamsMetrics = final StreamsMetricsImpl streamsMetrics =
new StreamsMetricsImpl(metrics, "test-application", time); new StreamsMetricsImpl(metrics, "test-application", "processId", time);
context = mock(InternalMockProcessorContext.class); context = mock(InternalMockProcessorContext.class);
when(context.metrics()).thenReturn(streamsMetrics); when(context.metrics()).thenReturn(streamsMetrics);
@ -984,7 +984,7 @@ public class RocksDBStoreTest extends AbstractKeyValueStoreTest {
final Metrics metrics = new Metrics(new MetricConfig().recordLevel(RecordingLevel.INFO)); final Metrics metrics = new Metrics(new MetricConfig().recordLevel(RecordingLevel.INFO));
final StreamsMetricsImpl streamsMetrics = final StreamsMetricsImpl streamsMetrics =
new StreamsMetricsImpl(metrics, "test-application", time); new StreamsMetricsImpl(metrics, "test-application", "processId", time);
final Properties props = StreamsTestUtils.getStreamsConfig(); final Properties props = StreamsTestUtils.getStreamsConfig();
context = mock(InternalMockProcessorContext.class); context = mock(InternalMockProcessorContext.class);

View File

@ -65,7 +65,7 @@ public class RocksDBTimeOrderedKeyValueBufferTest {
when(serdeGetter.valueSerde()).thenReturn(new Serdes.StringSerde()); when(serdeGetter.valueSerde()).thenReturn(new Serdes.StringSerde());
final Metrics metrics = new Metrics(); final Metrics metrics = new Metrics();
offset = 0; offset = 0;
streamsMetrics = new StreamsMetricsImpl(metrics, "test-client", new MockTime()); streamsMetrics = new StreamsMetricsImpl(metrics, "test-client", "processId", new MockTime());
context = new MockInternalNewProcessorContext<>(StreamsTestUtils.getStreamsConfig(), new TaskId(0, 0), TestUtils.tempDirectory()); context = new MockInternalNewProcessorContext<>(StreamsTestUtils.getStreamsConfig(), new TaskId(0, 0), TestUtils.tempDirectory());
} }

View File

@ -57,7 +57,7 @@ public class TimestampedSegmentTest {
@BeforeEach @BeforeEach
public void setUp() { public void setUp() {
metricsRecorder.init( metricsRecorder.init(
new StreamsMetricsImpl(new Metrics(), "test-client", new MockTime()), new StreamsMetricsImpl(new Metrics(), "test-client", "processId", new MockTime()),
new TaskId(0, 0) new TaskId(0, 0)
); );
} }

View File

@ -202,7 +202,7 @@ public class RocksDBMetricsRecorderGaugesTest {
private void runAndVerifySumOfProperties(final String propertyName) throws Exception { private void runAndVerifySumOfProperties(final String propertyName) throws Exception {
final StreamsMetricsImpl streamsMetrics = final StreamsMetricsImpl streamsMetrics =
new StreamsMetricsImpl(new Metrics(), "test-client", new MockTime()); new StreamsMetricsImpl(new Metrics(), "test-client", "processId", new MockTime());
final RocksDBMetricsRecorder recorder = new RocksDBMetricsRecorder(METRICS_SCOPE, STORE_NAME); final RocksDBMetricsRecorder recorder = new RocksDBMetricsRecorder(METRICS_SCOPE, STORE_NAME);
recorder.init(streamsMetrics, TASK_ID); recorder.init(streamsMetrics, TASK_ID);
@ -219,7 +219,7 @@ public class RocksDBMetricsRecorderGaugesTest {
private void runAndVerifyBlockCacheMetricsWithMultipleCaches(final String propertyName) throws Exception { private void runAndVerifyBlockCacheMetricsWithMultipleCaches(final String propertyName) throws Exception {
final StreamsMetricsImpl streamsMetrics = final StreamsMetricsImpl streamsMetrics =
new StreamsMetricsImpl(new Metrics(), "test-client", new MockTime()); new StreamsMetricsImpl(new Metrics(), "test-client", "processId", new MockTime());
final RocksDBMetricsRecorder recorder = new RocksDBMetricsRecorder(METRICS_SCOPE, STORE_NAME); final RocksDBMetricsRecorder recorder = new RocksDBMetricsRecorder(METRICS_SCOPE, STORE_NAME);
recorder.init(streamsMetrics, TASK_ID); recorder.init(streamsMetrics, TASK_ID);
@ -236,7 +236,7 @@ public class RocksDBMetricsRecorderGaugesTest {
private void runAndVerifyBlockCacheMetricsWithSingleCache(final String propertyName) throws Exception { private void runAndVerifyBlockCacheMetricsWithSingleCache(final String propertyName) throws Exception {
final StreamsMetricsImpl streamsMetrics = final StreamsMetricsImpl streamsMetrics =
new StreamsMetricsImpl(new Metrics(), "test-client", new MockTime()); new StreamsMetricsImpl(new Metrics(), "test-client", "processId", new MockTime());
final RocksDBMetricsRecorder recorder = new RocksDBMetricsRecorder(METRICS_SCOPE, STORE_NAME); final RocksDBMetricsRecorder recorder = new RocksDBMetricsRecorder(METRICS_SCOPE, STORE_NAME);
recorder.init(streamsMetrics, TASK_ID); recorder.init(streamsMetrics, TASK_ID);

View File

@ -178,7 +178,7 @@ public class RocksDBMetricsRecorderTest {
assertThrows( assertThrows(
IllegalStateException.class, IllegalStateException.class,
() -> recorder.init( () -> recorder.init(
new StreamsMetricsImpl(new Metrics(), "test-client", new MockTime()), new StreamsMetricsImpl(new Metrics(), "test-client", "processId", new MockTime()),
TASK_ID1 TASK_ID1
) )
); );

View File

@ -89,7 +89,7 @@ public class InternalMockProcessorContext<KOut, VOut>
this(null, this(null,
null, null,
null, null,
new StreamsMetricsImpl(new Metrics(), "mock", new MockTime()), new StreamsMetricsImpl(new Metrics(), "mock", "processId", new MockTime()),
new StreamsConfig(StreamsTestUtils.getStreamsConfig()), new StreamsConfig(StreamsTestUtils.getStreamsConfig()),
null, null,
null, null,
@ -106,6 +106,7 @@ public class InternalMockProcessorContext<KOut, VOut>
new StreamsMetricsImpl( new StreamsMetricsImpl(
new Metrics(), new Metrics(),
"mock", "mock",
"processId",
new MockTime() new MockTime()
), ),
config, config,
@ -138,6 +139,7 @@ public class InternalMockProcessorContext<KOut, VOut>
new StreamsMetricsImpl( new StreamsMetricsImpl(
new Metrics(), new Metrics(),
"mock", "mock",
"processId",
new MockTime() new MockTime()
), ),
config, config,
@ -155,7 +157,7 @@ public class InternalMockProcessorContext<KOut, VOut>
stateDir, stateDir,
keySerde, keySerde,
valueSerde, valueSerde,
new StreamsMetricsImpl(new Metrics(), "mock", new MockTime()), new StreamsMetricsImpl(new Metrics(), "mock", "processId", new MockTime()),
config, config,
null, null,
null, null,
@ -175,7 +177,7 @@ public class InternalMockProcessorContext<KOut, VOut>
null, null,
serdes.keySerde(), serdes.keySerde(),
serdes.valueSerde(), serdes.valueSerde(),
new StreamsMetricsImpl(metrics, "mock", new MockTime()), new StreamsMetricsImpl(metrics, "mock", "processId", new MockTime()),
new StreamsConfig(StreamsTestUtils.getStreamsConfig()), new StreamsConfig(StreamsTestUtils.getStreamsConfig()),
() -> collector, () -> collector,
null, null,
@ -192,7 +194,7 @@ public class InternalMockProcessorContext<KOut, VOut>
stateDir, stateDir,
keySerde, keySerde,
valueSerde, valueSerde,
new StreamsMetricsImpl(new Metrics(), "mock", new MockTime()), new StreamsMetricsImpl(new Metrics(), "mock", "processId", new MockTime()),
new StreamsConfig(StreamsTestUtils.getStreamsConfig()), new StreamsConfig(StreamsTestUtils.getStreamsConfig()),
() -> collector, () -> collector,
cache, cache,

View File

@ -382,6 +382,7 @@ public class TopologyTestDriver implements Closeable {
final StreamsMetricsImpl streamsMetrics = new StreamsMetricsImpl( final StreamsMetricsImpl streamsMetrics = new StreamsMetricsImpl(
metrics, metrics,
"test-client", "test-client",
"processId",
mockWallClockTime mockWallClockTime
); );
TaskMetrics.droppedRecordsSensor(threadId, TASK_ID.toString(), streamsMetrics); TaskMetrics.droppedRecordsSensor(threadId, TASK_ID.toString(), streamsMetrics);

View File

@ -242,6 +242,7 @@ public class MockProcessorContext implements ProcessorContext, RecordCollector.S
this.metrics = new StreamsMetricsImpl( this.metrics = new StreamsMetricsImpl(
new Metrics(metricConfig), new Metrics(metricConfig),
threadId, threadId,
"processId",
Time.SYSTEM Time.SYSTEM
); );
TaskMetrics.droppedRecordsSensor(threadId, taskId.toString(), metrics); TaskMetrics.droppedRecordsSensor(threadId, taskId.toString(), metrics);

View File

@ -255,6 +255,7 @@ public class MockProcessorContext<KForward, VForward> implements ProcessorContex
metrics = new StreamsMetricsImpl( metrics = new StreamsMetricsImpl(
new Metrics(metricConfig), new Metrics(metricConfig),
threadId, threadId,
"processId",
Time.SYSTEM Time.SYSTEM
); );
TaskMetrics.droppedRecordsSensor(threadId, taskId.toString(), metrics); TaskMetrics.droppedRecordsSensor(threadId, taskId.toString(), metrics);

View File

@ -232,6 +232,7 @@ public class MockProcessorContextTest {
when(mockInternalProcessorContext.metrics()).thenReturn(new StreamsMetricsImpl( when(mockInternalProcessorContext.metrics()).thenReturn(new StreamsMetricsImpl(
new Metrics(new MetricConfig()), new Metrics(new MetricConfig()),
Thread.currentThread().getName(), Thread.currentThread().getName(),
"processId",
Time.SYSTEM Time.SYSTEM
)); ));
when(mockInternalProcessorContext.taskId()).thenReturn(new TaskId(1, 1)); when(mockInternalProcessorContext.taskId()).thenReturn(new TaskId(1, 1));