You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kafka.apache.org by mj...@apache.org on 2021/02/11 00:45:59 UTC

[kafka] branch 2.6 updated: KAFKA-12272: Fix commit-interval metrics (#10103)

This is an automated email from the ASF dual-hosted git repository.

mjsax pushed a commit to branch 2.6
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/2.6 by this push:
     new 2abd33c5 KAFKA-12272: Fix commit-interval metrics (#10103)
2abd33c5 is described below

commit 2abd33c5946847e2305dd72f230579c61e5f8b88
Author: Matthias J. Sax <ma...@confluent.io>
AuthorDate: Wed Feb 10 16:44:21 2021 -0800

    KAFKA-12272: Fix commit-interval metrics (#10103)
    
    Reviewer: A. Sophie Blee-Goldman <so...@confluent.io>
---
 .../streams/processor/internals/StreamThread.java  |   3 +-
 .../processor/internals/StreamThreadTest.java      | 120 +++++++++++++++++++--
 2 files changed, 113 insertions(+), 10 deletions(-)

diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java
index 2ca4184..d707bb5 100644
--- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java
+++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java
@@ -711,8 +711,9 @@ public class StreamThread extends Thread {
                     punctuateSensor.record(punctuateLatency / (double) punctuated, now);
                 }
 
+                final long beforeCommitMs = now;
                 final int committed = maybeCommit();
-                final long commitLatency = advanceNowAndComputeLatency();
+                final long commitLatency = Math.max(now - beforeCommitMs, 0);
                 totalCommitLatency += commitLatency;
                 if (committed > 0) {
                     commitSensor.record(commitLatency / (double) committed, now);
diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java
index 7ab5b56..5c727d2 100644
--- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java
@@ -20,6 +20,7 @@ import org.apache.kafka.clients.admin.MockAdminClient;
 import org.apache.kafka.clients.consumer.Consumer;
 import org.apache.kafka.clients.consumer.ConsumerRebalanceListener;
 import org.apache.kafka.clients.consumer.ConsumerRecord;
+import org.apache.kafka.clients.consumer.ConsumerRecords;
 import org.apache.kafka.clients.consumer.InvalidOffsetException;
 import org.apache.kafka.clients.consumer.MockConsumer;
 import org.apache.kafka.clients.consumer.OffsetResetStrategy;
@@ -110,9 +111,11 @@ import static org.apache.kafka.common.utils.Utils.mkProperties;
 import static org.apache.kafka.common.utils.Utils.mkSet;
 import static org.apache.kafka.streams.processor.internals.ClientUtils.getSharedAdminClientId;
 import static org.apache.kafka.streams.processor.internals.StateManagerUtil.CHECKPOINT_FILE_NAME;
+import static org.easymock.EasyMock.anyObject;
 import static org.easymock.EasyMock.expect;
 import static org.easymock.EasyMock.expectLastCall;
 import static org.easymock.EasyMock.mock;
+import static org.easymock.EasyMock.niceMock;
 import static org.easymock.EasyMock.verify;
 import static org.hamcrest.CoreMatchers.equalTo;
 import static org.hamcrest.CoreMatchers.not;
@@ -780,6 +783,111 @@ public class StreamThreadTest {
     }
 
     @Test
+    public void shouldRecordCommitLatency() {
+        final Consumer<byte[], byte[]> consumer = EasyMock.createNiceMock(Consumer.class);
+        expect(consumer.poll(anyObject())).andStubReturn(new ConsumerRecords<>(Collections.emptyMap()));
+        final Task task = niceMock(Task.class);
+        expect(task.id()).andStubReturn(task1);
+        expect(task.inputPartitions()).andStubReturn(Collections.singleton(t1p1));
+        final ActiveTaskCreator activeTaskCreator = mock(ActiveTaskCreator.class);
+        expect(activeTaskCreator.createTasks(anyObject(), anyObject())).andStubReturn(Collections.singleton(task));
+        expect(activeTaskCreator.producerClientIds()).andStubReturn(Collections.singleton("producerClientId"));
+        EasyMock.replay(consumer, task, activeTaskCreator);
+
+        final StreamsMetricsImpl streamsMetrics =
+            new StreamsMetricsImpl(metrics, CLIENT_ID, StreamsConfig.METRICS_LATEST);
+
+        final TaskManager taskManager = new TaskManager(
+            null,
+            null,
+            null,
+            activeTaskCreator,
+            null,
+            internalTopologyBuilder,
+            null,
+            null,
+            null
+        ) {
+            @Override
+            int commit(final Collection<Task> tasksToCommit) {
+                mockTime.sleep(10L);
+                return 1;
+            }
+        };
+        taskManager.setMainConsumer(consumer);
+
+        final StreamThread thread = new StreamThread(
+            mockTime,
+            config,
+            null,
+            consumer,
+            consumer,
+            changelogReader,
+            null,
+            taskManager,
+            streamsMetrics,
+            internalTopologyBuilder,
+            CLIENT_ID,
+            new LogContext(""),
+            new AtomicInteger(),
+            new AtomicLong(Long.MAX_VALUE)
+        );
+        thread.updateThreadMetadata("adminClientId");
+        thread.setState(StreamThread.State.STARTING);
+
+        final Map<TaskId, Set<TopicPartition>> activeTasks = new HashMap<>();
+        activeTasks.put(task1, Collections.singleton(t1p1));
+        thread.taskManager().handleAssignment(activeTasks, emptyMap());
+        thread.rebalanceListener().onPartitionsAssigned(Collections.singleton(t1p1));
+
+        assertTrue(
+            Double.isNaN(
+                (Double) streamsMetrics.metrics().get(new MetricName(
+                    "commit-latency-max",
+                    "stream-thread-metrics",
+                    "",
+                    Collections.singletonMap("thread-id", CLIENT_ID))
+                ).metricValue()
+            )
+        );
+        assertTrue(
+            Double.isNaN(
+                (Double) streamsMetrics.metrics().get(new MetricName(
+                    "commit-latency-avg",
+                    "stream-thread-metrics",
+                    "",
+                    Collections.singletonMap("thread-id", CLIENT_ID))
+                ).metricValue()
+            )
+        );
+
+        thread.runOnce();
+
+        assertThat(
+            streamsMetrics.metrics().get(
+                new MetricName(
+                    "commit-latency-max",
+                    "stream-thread-metrics",
+                    "",
+                    Collections.singletonMap("thread-id", CLIENT_ID)
+                )
+            ).metricValue(),
+            equalTo(10.0)
+        );
+        assertThat(
+            streamsMetrics.metrics().get(
+                new MetricName(
+                    "commit-latency-avg",
+                    "stream-thread-metrics",
+                    "",
+                    Collections.singletonMap("thread-id", CLIENT_ID)
+                )
+            ).metricValue(),
+            equalTo(10.0)
+        );
+    }
+
+    @Test
     public void shouldInjectSharedProducerForAllTasksUsingClientSupplierOnCreateIfEosDisabled() {
         internalTopologyBuilder.addSource(null, "source1", null, null, null, topic1);
         internalStreamsBuilder.buildAndOptimizeTopology();
@@ -1228,26 +1336,20 @@ public class StreamThreadTest {
             @Override
             public Processor<Object, Object> get() {
                 return new Processor<Object, Object>() {
-                    private ProcessorContext context;
-
                     @Override
-                    public void init(final ProcessorContext context) {
-                        this.context = context;
-                    }
+                    public void init(final ProcessorContext context) {}
 
                     @Override
                     public void process(final Object key, final Object value) {
                         if (shouldThrow.get()) {
-                            throw new TaskCorruptedException(singletonMap(task1, new HashSet<TopicPartition>(singleton(storeChangelogTopicPartition))));
+                            throw new TaskCorruptedException(singletonMap(task1, new HashSet<>(singleton(storeChangelogTopicPartition))));
                         } else {
                             processed.set(true);
                         }
                     }
 
                     @Override
-                    public void close() {
-
-                    }
+                    public void close() {}
                 };
             }
         }, "name");