cadonna commented on a change in pull request #11149: URL: https://github.com/apache/kafka/pull/11149#discussion_r695567992
########## File path: clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java ########## @@ -1925,6 +1926,56 @@ public void testCommittedAuthenticationFailure() { assertThrows(AuthenticationException.class, () -> consumer.committed(Collections.singleton(tp0)).get(tp0)); } + @Test + public void testMeasureCommitSyncDuration() { + // use a consumer that will throw to ensure we return quickly Review comment: I think we also need to test the case without failure. Otherwise, we assume in the test that the measurement is in the `finally` clause which we should not assume but rather ensure with unit tests. Same applies to the test below. ########## File path: clients/src/test/java/org/apache/kafka/clients/producer/internals/KafkaProducerMetricsTest.java ########## @@ -0,0 +1,91 @@ +/* + * 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 static org.junit.jupiter.api.Assertions.assertEquals; + +import org.apache.kafka.common.metrics.Metrics; +import org.junit.jupiter.api.Test; Review comment: ```suggestion import org.apache.kafka.common.metrics.Metrics; import org.junit.jupiter.api.Test; import static org.junit.jupiter.api.Assertions.assertEquals; ``` ########## File path: clients/src/test/java/org/apache/kafka/clients/producer/KafkaProducerTest.java ########## @@ -905,6 +971,57 @@ public void testSendTxnOffsetsWithGroupId() { } } + private double getAndAssertDuration(KafkaProducer<?, ?> producer, String name, double floor) { + double value = getMetricValue(producer, name); + assertTrue(value > floor); + return value; + } + + @Test + public void testMeasureTransactionDurations() { + Map<String, Object> configs = new HashMap<>(); + configs.put(ProducerConfig.TRANSACTIONAL_ID_CONFIG, "some.id"); + configs.put(ProducerConfig.MAX_BLOCK_MS_CONFIG, 10000); + configs.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9000"); + Time time = new MockTime(1); + MetadataResponse initialUpdateResponse = RequestTestUtils.metadataUpdateWith(1, singletonMap("topic", 1)); + ProducerMetadata metadata = newMetadata(0, Long.MAX_VALUE); + + MockClient client = new MockClient(time, metadata); + client.updateMetadata(initialUpdateResponse); + client.prepareResponse(FindCoordinatorResponse.prepareResponse(Errors.NONE, "some.id", host1)); + client.prepareResponse(initProducerIdResponse(1L, (short) 5, Errors.NONE)); + + try (KafkaProducer<String, String> producer = kafkaProducer(configs, new StringSerializer(), + new StringSerializer(), metadata, client, null, time)) { + producer.initTransactions(); + assertTrue(getMetricValue(producer, "txn-init-time-total") > 999999); Review comment: I now saw that in the consumer tests you use `Duration.ofSeconds(1).toMillis()` and `Duration.ofMillis(999).toNanos()`. This makes it already clearer. I think a variable with a meaningful name for the lower bound would make it even clearer. ########## File path: clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java ########## @@ -1925,6 +1926,56 @@ public void testCommittedAuthenticationFailure() { assertThrows(AuthenticationException.class, () -> consumer.committed(Collections.singleton(tp0)).get(tp0)); } + @Test + public void testMeasureCommitSyncDuration() { + // use a consumer that will throw to ensure we return quickly + Time time = new MockTime(Duration.ofSeconds(1).toMillis()); + SubscriptionState subscription = new SubscriptionState(new LogContext(), OffsetResetStrategy.EARLIEST); + ConsumerMetadata metadata = createMetadata(subscription); + MockClient client = new MockClient(time, metadata); + initMetadata(client, singletonMap(topic, 1)); + Node node = metadata.fetch().nodes().get(0); + ConsumerPartitionAssignor assignor = new RangeAssignor(); + client.createPendingAuthenticationError(node, 0); + final KafkaConsumer<String, String> consumer + = newConsumer(time, client, subscription, metadata, assignor, false, groupInstanceId); Review comment: Could you not just use `final KafkaConsumer<String, String> consumer = consumerWithPendingAuthenticationError();` as in the test above? ########## File path: clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java ########## @@ -1925,6 +1926,56 @@ public void testCommittedAuthenticationFailure() { assertThrows(AuthenticationException.class, () -> consumer.committed(Collections.singleton(tp0)).get(tp0)); } + @Test + public void testMeasureCommitSyncDuration() { + // use a consumer that will throw to ensure we return quickly + Time time = new MockTime(Duration.ofSeconds(1).toMillis()); + SubscriptionState subscription = new SubscriptionState(new LogContext(), OffsetResetStrategy.EARLIEST); + ConsumerMetadata metadata = createMetadata(subscription); + MockClient client = new MockClient(time, metadata); + initMetadata(client, singletonMap(topic, 1)); + Node node = metadata.fetch().nodes().get(0); + ConsumerPartitionAssignor assignor = new RangeAssignor(); + client.createPendingAuthenticationError(node, 0); + final KafkaConsumer<String, String> consumer + = newConsumer(time, client, subscription, metadata, assignor, false, groupInstanceId); + Map<TopicPartition, OffsetAndMetadata> offsets = new HashMap<>(); + offsets.put(tp0, new OffsetAndMetadata(10L)); + + try { + consumer.commitSync(offsets); + } catch (final RuntimeException e) { + } + + final Metric metric = consumer.metrics() + .get(consumer.metrics.metricName("commit-sync-time-total", "consumer-metrics")); + assertTrue((Double)metric.metricValue() >= Duration.ofMillis(999).toNanos()); + } + + @Test + public void testMeasureCommittedDuration() { + // use a consumer that will throw to ensure we return quickly + Time time = new MockTime(Duration.ofSeconds(1).toMillis()); + SubscriptionState subscription = new SubscriptionState(new LogContext(), OffsetResetStrategy.EARLIEST); + ConsumerMetadata metadata = createMetadata(subscription); + MockClient client = new MockClient(time, metadata); + initMetadata(client, singletonMap(topic, 1)); + Node node = metadata.fetch().nodes().get(0); + ConsumerPartitionAssignor assignor = new RangeAssignor(); + client.createPendingAuthenticationError(node, 0); + final KafkaConsumer<String, String> consumer + = newConsumer(time, client, subscription, metadata, assignor, false, groupInstanceId); Review comment: Could you not just use final KafkaConsumer<String, String> consumer = consumerWithPendingAuthenticationError(); as in the test above? ########## 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), + new CumulativeSum() + ); + + this.committedSensor = metrics.sensor("committed-time-total"); + this.committedSensor.add( + metrics.metricName("committed-time-total", metricGroupName), + new CumulativeSum() + ); Review comment: Could you please already open the follow-up PR with scaffolding and link it here? I think otherwise we risk to forget about it. ########## File path: streams/test-utils/src/main/java/org/apache/kafka/streams/TopologyTestDriver.java ########## @@ -1337,7 +1337,7 @@ public boolean isOpen() { public TestDriverProducer(final StreamsConfig config, final KafkaClientSupplier clientSupplier, final LogContext logContext) { - super(config, "TopologyTestDriver-StreamThread-1", clientSupplier, new TaskId(0, 0), UUID.randomUUID(), logContext); + super(config, "TopologyTestDriver-StreamThread-1", clientSupplier, new TaskId(0, 0), UUID.randomUUID(), logContext, Time.SYSTEM); Review comment: I think this should be: ```suggestion super(config, "TopologyTestDriver-StreamThread-1", clientSupplier, new TaskId(0, 0), UUID.randomUUID(), logContext, mockWallClockTime); ``` ########## File path: clients/src/test/java/org/apache/kafka/clients/producer/KafkaProducerTest.java ########## @@ -905,6 +971,57 @@ public void testSendTxnOffsetsWithGroupId() { } } + private double getAndAssertDuration(KafkaProducer<?, ?> producer, String name, double floor) { + double value = getMetricValue(producer, name); + assertTrue(value > floor); + return value; + } + + @Test + public void testMeasureTransactionDurations() { + Map<String, Object> configs = new HashMap<>(); + configs.put(ProducerConfig.TRANSACTIONAL_ID_CONFIG, "some.id"); + configs.put(ProducerConfig.MAX_BLOCK_MS_CONFIG, 10000); + configs.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9000"); + Time time = new MockTime(1); + MetadataResponse initialUpdateResponse = RequestTestUtils.metadataUpdateWith(1, singletonMap("topic", 1)); + ProducerMetadata metadata = newMetadata(0, Long.MAX_VALUE); + + MockClient client = new MockClient(time, metadata); + client.updateMetadata(initialUpdateResponse); + client.prepareResponse(FindCoordinatorResponse.prepareResponse(Errors.NONE, "some.id", host1)); + client.prepareResponse(initProducerIdResponse(1L, (short) 5, Errors.NONE)); + + try (KafkaProducer<String, String> producer = kafkaProducer(configs, new StringSerializer(), + new StringSerializer(), metadata, client, null, time)) { + producer.initTransactions(); + assertTrue(getMetricValue(producer, "txn-init-time-total") > 999999); Review comment: Got it! Could you please put the `999999` in a variable and give it a meaningful name such as `lowerBoundNs`. Maybe put also the `1` that you use in the `MockTime` constructor in a variable such as `tickMs` so that the difference in units becomes clear. BTW: you could also use `getAndAssertDuration()` here, right? -- 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