mirror of https://github.com/apache/kafka.git
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:
parent
a0d4cbec40
commit
c69a6b0e80
|
@ -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);
|
||||||
|
|
|
@ -978,6 +978,7 @@ public class KafkaStreams implements AutoCloseable {
|
||||||
streamsMetrics = new StreamsMetricsImpl(
|
streamsMetrics = new StreamsMetricsImpl(
|
||||||
metrics,
|
metrics,
|
||||||
clientId,
|
clientId,
|
||||||
|
processId.toString(),
|
||||||
time
|
time
|
||||||
);
|
);
|
||||||
|
|
||||||
|
|
|
@ -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;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -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;
|
||||||
|
|
|
@ -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")
|
||||||
|
|
|
@ -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,
|
||||||
|
|
|
@ -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());
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -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);
|
||||||
|
|
|
@ -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<>(
|
||||||
|
|
|
@ -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());
|
||||||
|
|
|
@ -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;
|
||||||
|
|
|
@ -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;
|
||||||
}
|
}
|
||||||
|
|
|
@ -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,
|
||||||
|
|
|
@ -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,
|
||||||
|
|
|
@ -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",
|
||||||
|
|
|
@ -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())),
|
||||||
|
|
|
@ -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())),
|
||||||
|
|
|
@ -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())),
|
||||||
|
|
|
@ -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);
|
||||||
|
|
|
@ -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)
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
|
|
@ -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);
|
||||||
|
|
|
@ -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);
|
||||||
|
|
|
@ -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);
|
||||||
|
|
|
@ -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(),
|
||||||
|
|
|
@ -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);
|
||||||
|
|
||||||
|
|
|
@ -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(),
|
||||||
|
|
|
@ -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);
|
||||||
|
|
|
@ -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());
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -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)
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
|
|
@ -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);
|
||||||
|
|
|
@ -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
|
||||||
)
|
)
|
||||||
);
|
);
|
||||||
|
|
|
@ -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,
|
||||||
|
|
|
@ -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);
|
||||||
|
|
|
@ -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);
|
||||||
|
|
|
@ -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);
|
||||||
|
|
|
@ -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));
|
||||||
|
|
Loading…
Reference in New Issue