bbejeck commented on code in PR #17781:
URL: https://github.com/apache/kafka/pull/17781#discussion_r1841165295


##########
clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java:
##########
@@ -224,6 +232,117 @@ public void cleanup() {
         }
     }
 
+    @ParameterizedTest
+    @EnumSource(GroupProtocol.class)
+    public void 
testSubscribingCustomMetricsDoesntAffectConsumerMetrics(GroupProtocol 
groupProtocol) {
+        Properties props = new Properties();
+        props.setProperty(ConsumerConfig.GROUP_PROTOCOL_CONFIG, 
groupProtocol.name());
+        props.setProperty(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, 
"localhost:9999");
+        consumer = newConsumer(props, new StringDeserializer(), new 
StringDeserializer());
+
+        Map<MetricName, KafkaMetric> customMetrics = customMetrics();
+        customMetrics.forEach((name, metric) -> 
consumer.registerMetricForSubscription(metric));
+
+        Map<MetricName, ? extends Metric> consumerMetrics = consumer.metrics();
+        customMetrics.forEach((name, metric) -> 
assertFalse(consumerMetrics.containsKey(name)));
+    }
+
+    @ParameterizedTest
+    @EnumSource(GroupProtocol.class)
+    public void 
testSubscribingCustomMetricsWithSameNameDoesntAffectConsumerMetrics(GroupProtocol
 groupProtocol) {
+        Properties props = new Properties();
+        props.setProperty(ConsumerConfig.GROUP_PROTOCOL_CONFIG, 
groupProtocol.name());
+        props.setProperty(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, 
"localhost:9999");
+        consumer = newConsumer(props, new StringDeserializer(), new 
StringDeserializer());
+
+        Map<MetricName, ? extends Metric> sortedMetrics = new 
LinkedHashMap<>(consumer.metrics());
+        KafkaMetric firstMetric = (KafkaMetric) 
sortedMetrics.entrySet().iterator().next().getValue();
+        
+        Object lock = new Object();
+        MetricName metricNameCopy = new 
MetricName(firstMetric.metricName().name(), firstMetric.metricName().group(), 
firstMetric.metricName().description(), firstMetric.metricName().tags());
+        KafkaMetric metricCopy = new KafkaMetric(lock, metricNameCopy, 
firstMetric.measurable(), firstMetric.config(), Time.SYSTEM);
+        consumer.registerMetricForSubscription(metricCopy);
+
+        sortedMetrics = new LinkedHashMap<>(consumer.metrics());
+        KafkaMetric secondMetric = (KafkaMetric) 
sortedMetrics.entrySet().iterator().next().getValue();
+        assertSame(firstMetric, secondMetric);
+        assertNotSame(firstMetric, metricCopy);
+    }
+
+    @ParameterizedTest
+    @EnumSource(GroupProtocol.class)
+    public void 
testUnsubscribingCustomMetricsWithSameNameDoesntAffectConsumerMetrics(GroupProtocol
 groupProtocol) {
+        Properties props = new Properties();
+        props.setProperty(ConsumerConfig.GROUP_PROTOCOL_CONFIG, 
groupProtocol.name());
+        props.setProperty(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, 
"localhost:9999");
+        consumer = newConsumer(props, new StringDeserializer(), new 
StringDeserializer());
+
+        Map<MetricName, ? extends Metric> sortedMetrics = new 
LinkedHashMap<>(consumer.metrics());
+        KafkaMetric firstMetric = (KafkaMetric) 
sortedMetrics.entrySet().iterator().next().getValue();
+
+        Object lock = new Object();
+        MetricName metricNameCopy = new 
MetricName(firstMetric.metricName().name(), firstMetric.metricName().group(), 
firstMetric.metricName().description(), firstMetric.metricName().tags());
+        KafkaMetric metricToRemove = new KafkaMetric(lock, metricNameCopy, 
firstMetric.measurable(), firstMetric.config(), Time.SYSTEM);
+        consumer.unregisterMetricFromSubscription(metricToRemove);
+
+        sortedMetrics = new LinkedHashMap<>(consumer.metrics());
+        KafkaMetric secondMetric = (KafkaMetric) 
sortedMetrics.entrySet().iterator().next().getValue();
+        assertSame(firstMetric, secondMetric);
+        assertNotSame(firstMetric, metricToRemove);
+    }
+
+    @ParameterizedTest
+    @EnumSource(GroupProtocol.class)
+    public void 
testShouldOnlyCallMetricReporterMetricChangeOnceWithExistingConsumerMetric(GroupProtocol
 groupProtocol) {
+        try (MockedStatic<CommonClientConfigs> mockedCommonClientConfigs = 
mockStatic(CommonClientConfigs.class, new CallsRealMethods())) {
+            ClientTelemetryReporter clientTelemetryReporter = 
mock(ClientTelemetryReporter.class);
+            clientTelemetryReporter.configure(any());
+            mockedCommonClientConfigs.when(() -> 
CommonClientConfigs.telemetryReporter(anyString(), 
any())).thenReturn(Optional.of(clientTelemetryReporter));
+
+            Properties props = new Properties();
+            props.setProperty(ConsumerConfig.GROUP_PROTOCOL_CONFIG, 
groupProtocol.name());
+            props.setProperty(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, 
"localhost:9999");
+            consumer = newConsumer(props, new StringDeserializer(), new 
StringDeserializer());
+
+            KafkaMetric existingMetric = (KafkaMetric) 
consumer.metrics().entrySet().iterator().next().getValue();
+            consumer.registerMetricForSubscription(existingMetric);
+            // This test would fail without the check as the exising metric is 
registered in the consumer on startup
+            Mockito.verify(clientTelemetryReporter, 
atMostOnce()).metricChange(existingMetric);

Review Comment:
   When the consumer starts it will register all of its metrics, calling 
`ClientTelmetryReporter.metricChange` so here we're validating that is the only 
time this method is called, without the guard condition above this test would 
fail.



-- 
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