cadonna commented on a change in pull request #11149:
URL: https://github.com/apache/kafka/pull/11149#discussion_r691098389



##########
File path: 
streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsThreadTotalBlockedTime.java
##########
@@ -0,0 +1,59 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.kafka.streams.processor.internals;
+
+import java.util.Map;
+import java.util.function.Supplier;
+import org.apache.kafka.clients.consumer.Consumer;
+import org.apache.kafka.common.Metric;
+import org.apache.kafka.common.MetricName;
+
+public class StreamsThreadTotalBlockedTime {
+    final Consumer<?, ?> consumer;
+    final Consumer<?, ?> restoreConsumer;
+    final Supplier<Double> producerTotalBlockedTime;
+
+    StreamsThreadTotalBlockedTime(
+        final Consumer<?, ?> consumer,
+        final Consumer<?, ?> restoreConsumer,
+        final Supplier<Double> producerTotalBlockedTime) {
+        this.consumer = consumer;
+        this.restoreConsumer = restoreConsumer;
+        this.producerTotalBlockedTime = producerTotalBlockedTime;
+    }
+
+    final double getMetricValue(

Review comment:
       Could you please make this method private? 
   Out of curiosity, why did you define this method as `final`?

##########
File path: 
streams/src/main/java/org/apache/kafka/streams/processor/internals/metrics/ThreadMetrics.java
##########
@@ -310,6 +317,28 @@ public static Sensor commitRatioSensor(final String 
threadId,
         return sensor;
     }
 
+    public static void addThreadStartTimeMetric(final String threadId,

Review comment:
       Could you please add tests in `ThreadMetricsTest`?
   A similar test is `ClientMetricsTest#shouldAddVersionMetric()`.  

##########
File path: 
streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsThreadTotalBlockedTime.java
##########
@@ -0,0 +1,59 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.kafka.streams.processor.internals;
+
+import java.util.Map;
+import java.util.function.Supplier;
+import org.apache.kafka.clients.consumer.Consumer;
+import org.apache.kafka.common.Metric;
+import org.apache.kafka.common.MetricName;
+
+public class StreamsThreadTotalBlockedTime {
+    final Consumer<?, ?> consumer;
+    final Consumer<?, ?> restoreConsumer;
+    final Supplier<Double> producerTotalBlockedTime;

Review comment:
       Could you please make this member fields private?

##########
File path: 
streams/src/main/java/org/apache/kafka/streams/processor/internals/metrics/ThreadMetrics.java
##########
@@ -310,6 +317,28 @@ public static Sensor commitRatioSensor(final String 
threadId,
         return sensor;
     }
 
+    public static void addThreadStartTimeMetric(final String threadId,
+                                                final StreamsMetricsImpl 
streamsMetrics,
+                                                final long startTime) {
+        streamsMetrics.addThreadLevelImmutableMetric(
+            THREAD_START_TIME,
+            THREAD_START_TIME_DESCRIPTION,
+            threadId,
+            startTime
+        );
+    }
+
+    public static void addThreadBlockedTimeMetric(final String threadId,

Review comment:
       Could you please add tests in `ThreadMetricsTest`?
   A similar test is `ClientMetricsTest#shouldAddStateMetric()`

##########
File path: 
streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamsProducerTest.java
##########
@@ -1121,4 +1125,60 @@ public void 
shouldResetTransactionInitializedOnResetProducer() {
         verify(mockedProducer);
     }
 
+    @Test
+    public void shouldComputeTotalBlockedTime() {
+        setProducerMetrics(nonEosMockProducer, 1, 2, 3, 4, 5, 6, 7);
+
+        final double expectedTotalBlocked = 1 + 2 + 3 + 4 + 5 + 6 + 7;
+        assertThat(nonEosStreamsProducer.totalBlockedTime(), 
equalTo(expectedTotalBlocked));
+    }
+
+    @Test
+    public void shouldComputeTotalBlockedTimeAfterReset() {
+        setProducerMetrics(eosBetaMockProducer, 1, 2, 3, 4, 5, 6, 7);
+        eosBetaStreamsProducer.resetProducer();
+
+        final double expectedTotalBlocked = 1 + 2 + 3 + 4 + 5 + 6 + 7;
+        assertThat(eosBetaStreamsProducer.totalBlockedTime(), greaterThan(2 * 
expectedTotalBlocked));

Review comment:
       Do you use `greaterThan()` here because of `double`? Maybe you should 
use `Matchers.closeTo(double, double)`.

##########
File path: 
streams/src/main/java/org/apache/kafka/streams/processor/internals/metrics/ThreadMetrics.java
##########
@@ -20,6 +20,7 @@
 import org.apache.kafka.common.metrics.Sensor.RecordingLevel;
 
 import java.util.Map;
+import 
org.apache.kafka.streams.processor.internals.StreamsThreadTotalBlockedTime;

Review comment:
       Could you please move this import after line 20? 

##########
File path: 
streams/src/main/java/org/apache/kafka/streams/processor/internals/ActiveTaskCreator.java
##########
@@ -299,4 +299,12 @@ private LogContext getLogContext(final TaskId taskId) {
         return new LogContext(logPrefix);
     }
 
+    public double totalProducerBlockedTime() {

Review comment:
       Could you please write a unit test for this method?

##########
File path: 
streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamsProducerTest.java
##########
@@ -1121,4 +1125,60 @@ public void 
shouldResetTransactionInitializedOnResetProducer() {
         verify(mockedProducer);
     }
 
+    @Test
+    public void shouldComputeTotalBlockedTime() {
+        setProducerMetrics(nonEosMockProducer, 1, 2, 3, 4, 5, 6, 7);
+
+        final double expectedTotalBlocked = 1 + 2 + 3 + 4 + 5 + 6 + 7;
+        assertThat(nonEosStreamsProducer.totalBlockedTime(), 
equalTo(expectedTotalBlocked));
+    }
+
+    @Test
+    public void shouldComputeTotalBlockedTimeAfterReset() {
+        setProducerMetrics(eosBetaMockProducer, 1, 2, 3, 4, 5, 6, 7);
+        eosBetaStreamsProducer.resetProducer();

Review comment:
       Could you add `assertThat(eosBetaStreamsProducer.totalBlockedTime(), 
equalTo(expectedTotalBlocked));` above this line?

##########
File path: 
streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsThreadTotalBlockedTime.java
##########
@@ -0,0 +1,59 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.kafka.streams.processor.internals;
+
+import java.util.Map;
+import java.util.function.Supplier;
+import org.apache.kafka.clients.consumer.Consumer;
+import org.apache.kafka.common.Metric;
+import org.apache.kafka.common.MetricName;
+
+public class StreamsThreadTotalBlockedTime {
+    final Consumer<?, ?> consumer;
+    final Consumer<?, ?> restoreConsumer;
+    final Supplier<Double> producerTotalBlockedTime;
+
+    StreamsThreadTotalBlockedTime(
+        final Consumer<?, ?> consumer,
+        final Consumer<?, ?> restoreConsumer,
+        final Supplier<Double> producerTotalBlockedTime) {
+        this.consumer = consumer;
+        this.restoreConsumer = restoreConsumer;
+        this.producerTotalBlockedTime = producerTotalBlockedTime;
+    }
+
+    final double getMetricValue(
+        final Map<MetricName, ? extends Metric> metrics,
+        final String name) {
+        return metrics.keySet().stream()
+            .filter(n -> n.name().equals(name))
+            .findFirst()
+            .map(n -> (Double) metrics.get(n).metricValue())
+            .orElse(0.0);
+    }
+
+    public double getTotalBlockedTime() {

Review comment:
       We do usually not use the `get` prefix in AK. What about renaming the 
class to `StreamThreadTotalBlockedTime` (note that I removed the `s` in 
`StreamThread` to match with the `StreamThread` class)  and the method to 
`compute()`?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscr...@kafka.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


Reply via email to