guozhangwang commented on a change in pull request #11149: URL: https://github.com/apache/kafka/pull/11149#discussion_r697605349
########## File path: clients/src/main/java/org/apache/kafka/clients/producer/internals/KafkaProducerMetrics.java ########## @@ -0,0 +1,106 @@ +/* + * 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.clients.producer.internals; + +import org.apache.kafka.common.MetricName; +import org.apache.kafka.common.metrics.Metrics; +import org.apache.kafka.common.metrics.Sensor; +import org.apache.kafka.common.metrics.stats.CumulativeSum; + +import java.util.Map; + +public class KafkaProducerMetrics implements AutoCloseable { + + public static final String GROUP = "producer-metrics"; + private static final String FLUSH = "flush"; + private static final String TXN_INIT = "txn-init"; + private static final String TXN_BEGIN = "txn-begin"; + private static final String TXN_SEND_OFFSETS = "txn-send-offsets"; + private static final String TXN_COMMIT = "txn-commit"; + private static final String TXN_ABORT = "txn-abort"; + private static final String TOTAL_TIME_SUFFIX = "-time-total"; Review comment: Just following my other comment here: maybe we can declare it as TOTAL_NS_TIME_SUFFIX = "-time-ns-total"; ########## File path: clients/src/main/java/org/apache/kafka/clients/consumer/internals/KafkaConsumerMetrics.java ########## @@ -63,6 +66,18 @@ public KafkaConsumerMetrics(Metrics metrics, String metricGrpPrefix) { metricGroupName, "The average fraction of time the consumer's poll() is idle as opposed to waiting for the user code to process records."), new Avg()); + + this.commitSyncSensor = metrics.sensor("commit-sync-time-total"); + this.commitSyncSensor.add( + metrics.metricName("commit-sync-time-total", metricGroupName), Review comment: Could we add the description in the metricName as well indicating this is measured in nanos not millis? Ditto elsewhere. ########## File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsProducer.java ########## @@ -178,12 +184,48 @@ public void resetProducer() { throw new IllegalStateException("Expected eos-v2 to be enabled, but the processing mode was " + processingMode); } + oldProducerTotalBlockedTime += totalBlockedTime(producer); + final long start = time.nanoseconds(); producer.close(); + final long closeTime = time.nanoseconds() - start; + oldProducerTotalBlockedTime += closeTime; producer = clientSupplier.getProducer(eosV2ProducerConfigs); transactionInitialized = false; } + private double getMetricValue(final Map<MetricName, ? extends Metric> metrics, + final String name) { + final List<MetricName> found = metrics.keySet().stream() + .filter(n -> n.name().equals(name)) + .collect(Collectors.toList()); + if (found.isEmpty()) { + return 0.0; + } + if (found.size() > 1) { Review comment: Since the `metrics` registry object is not shared among all embedded producer/consumer instances, we should never get this situation. How about treating it as an illegal state exception which would be fatal and kill the node? ########## File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/metrics/ThreadMetrics.java ########## @@ -45,6 +46,8 @@ private ThreadMetrics() {} private static final String CREATE_TASK = "task-created"; private static final String CLOSE_TASK = "task-closed"; private static final String SKIP_RECORD = "skipped-records"; + private static final String BLOCKED_TIME = "blocked-time-total"; Review comment: Just follow my other comment: how about naming it as `blocked-time-ns-total`? ########## File path: clients/src/main/java/org/apache/kafka/clients/producer/internals/KafkaProducerMetrics.java ########## @@ -0,0 +1,106 @@ +/* + * 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.clients.producer.internals; + +import org.apache.kafka.common.MetricName; +import org.apache.kafka.common.metrics.Metrics; +import org.apache.kafka.common.metrics.Sensor; +import org.apache.kafka.common.metrics.stats.CumulativeSum; + +import java.util.Map; + +public class KafkaProducerMetrics implements AutoCloseable { + + public static final String GROUP = "producer-metrics"; + private static final String FLUSH = "flush"; + private static final String TXN_INIT = "txn-init"; + private static final String TXN_BEGIN = "txn-begin"; + private static final String TXN_SEND_OFFSETS = "txn-send-offsets"; + private static final String TXN_COMMIT = "txn-commit"; + private static final String TXN_ABORT = "txn-abort"; + private static final String TOTAL_TIME_SUFFIX = "-time-total"; + + private final Map<String, String> tags; + private final Metrics metrics; + private final Sensor initTimeSensor; + private final Sensor beginTxnTimeSensor; + private final Sensor flushTimeSensor; + private final Sensor sendOffsetsSensor; + private final Sensor commitTxnSensor; + private final Sensor abortTxnSensor; + + public KafkaProducerMetrics(Metrics metrics) { + this.metrics = metrics; + tags = this.metrics.config().tags(); + flushTimeSensor = newLatencySensor(FLUSH); + initTimeSensor = newLatencySensor(TXN_INIT); + beginTxnTimeSensor = newLatencySensor(TXN_BEGIN); + sendOffsetsSensor = newLatencySensor(TXN_SEND_OFFSETS); + commitTxnSensor = newLatencySensor(TXN_COMMIT); + abortTxnSensor = newLatencySensor(TXN_ABORT); + } + + @Override + public void close() { + removeMetric(FLUSH); + removeMetric(TXN_INIT); + removeMetric(TXN_BEGIN); + removeMetric(TXN_SEND_OFFSETS); + removeMetric(TXN_COMMIT); + removeMetric(TXN_ABORT); + } + + public void recordFlush(long duration) { + flushTimeSensor.record(duration); + } + + public void recordInit(long duration) { + initTimeSensor.record(duration); + } + + public void recordBeginTxn(long duration) { + beginTxnTimeSensor.record(duration); + } + + public void recordSendOffsets(long duration) { + sendOffsetsSensor.record(duration); + } + + public void recordCommitTxn(long duration) { + commitTxnSensor.record(duration); + } + + public void recordAbortTxn(long duration) { + abortTxnSensor.record(duration); + } + + private Sensor newLatencySensor(String name) { + Sensor sensor = metrics.sensor(name + TOTAL_TIME_SUFFIX); + sensor.add(metricName(name), new CumulativeSum()); + return sensor; + } + + private MetricName metricName(final String name) { + return metrics.metricName(name + TOTAL_TIME_SUFFIX, GROUP, tags); + } + + private void removeMetric(final String name) { + metrics.removeSensor(name + TOTAL_TIME_SUFFIX); Review comment: `removeSensor()` would remove its associated metrics as well, I think we do not need the second call below. ########## File path: clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java ########## @@ -590,9 +593,11 @@ else if (acks != -1) public void initTransactions() { throwIfNoTransactionManager(); throwIfProducerClosed(); + long now = time.nanoseconds(); Review comment: I took a look at these metrics and I think they should be fixed with the `-ns` suffix (I was wrong though thinking that they should be `-total-ns`, instead they should be `-ns-total`). Filed https://issues.apache.org/jira/browse/KAFKA-13243 For the newly added metrics, I'd suggest we add the -ns suffix before the "type" (-total) suffix, in which we would update the wiki page and then just send an update on the KIP thread saying we are making a small change to it. We do not need to ask for re-voting since this is a small change. ########## File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThreadTotalBlockedTime.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 StreamThreadTotalBlockedTime { + private final Consumer<?, ?> consumer; + private final Consumer<?, ?> restoreConsumer; + private final Supplier<Double> producerTotalBlockedTime; + + StreamThreadTotalBlockedTime( + final Consumer<?, ?> consumer, + final Consumer<?, ?> restoreConsumer, + final Supplier<Double> producerTotalBlockedTime) { + this.consumer = consumer; + this.restoreConsumer = restoreConsumer; + this.producerTotalBlockedTime = producerTotalBlockedTime; + } + + private double metricValue( + 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 compute() { + return metricValue(consumer.metrics(), "io-waittime-total") + + metricValue(consumer.metrics(), "iotime-total") Review comment: Not introduced in this PR at all, but I just realized we mistakenly had a bug that caused the avg and total metric names to be different: `iotime` for total and `io-time` for average. I will include that in the filed JIRA ticket as well. ########## File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/metrics/StreamsMetricsImpl.java ########## @@ -269,6 +300,12 @@ public final void removeAllThreadLevelSensors(final String threadId) { metrics.removeSensor(sensors.pop()); } } + synchronized (threadLevelMetrics) { Review comment: We should not need this block as well since now the names are consistent, the above block should be sufficient. Note that `metrics.removeSensor()` would also trigger `metrics.removeMetric` for its children metrics. -- 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