mumrah commented on code in PR #13438:
URL: https://github.com/apache/kafka/pull/13438#discussion_r1147915745


##########
metadata/src/main/java/org/apache/kafka/controller/metrics/ControllerMetadataMetrics.java:
##########
@@ -0,0 +1,211 @@
+/*
+ * 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.controller.metrics;
+
+import com.yammer.metrics.core.Gauge;
+import com.yammer.metrics.core.MetricName;
+import com.yammer.metrics.core.MetricsRegistry;
+import org.apache.kafka.server.metrics.KafkaYammerMetrics;
+
+import java.util.Arrays;
+import java.util.Optional;
+import java.util.concurrent.atomic.AtomicInteger;
+
+/**
+ * These are the metrics which are managed by the ControllerServer class. They 
generally pertain to
+ * aspects of the metadata, like how many topics or partitions we have.
+ * All of these except MetadataErrorCount are managed by 
ControllerMetadataMetricsPublisher.
+ *
+ * IMPORTANT: Metrics which are managed by the QuorumController class itself 
should go in
+ * @link{org.apache.kafka.controller.metrics.QuorumControllerMetrics}, not 
here.
+ */
+public final class ControllerMetadataMetrics implements AutoCloseable {
+    private final static MetricName FENCED_BROKER_COUNT = getMetricName(
+        "KafkaController", "FencedBrokerCount");
+    private final static MetricName ACTIVE_BROKER_COUNT = getMetricName(
+        "KafkaController", "ActiveBrokerCount");
+    private final static MetricName GLOBAL_TOPIC_COUNT = getMetricName(
+        "KafkaController", "GlobalTopicCount");
+    private final static MetricName GLOBAL_PARTITION_COUNT = getMetricName(
+        "KafkaController", "GlobalPartitionCount");
+    private final static MetricName OFFLINE_PARTITION_COUNT = getMetricName(
+        "KafkaController", "OfflinePartitionsCount");
+    private final static MetricName PREFERRED_REPLICA_IMBALANCE_COUNT = 
getMetricName(
+        "KafkaController", "PreferredReplicaImbalanceCount");
+    private final static MetricName METADATA_ERROR_COUNT = getMetricName(
+        "KafkaController", "MetadataErrorCount");
+
+    private final Optional<MetricsRegistry> registry;
+    private final AtomicInteger fencedBrokerCount = new AtomicInteger(0);
+    private final AtomicInteger activeBrokerCount = new AtomicInteger(0);
+    private final AtomicInteger globalTopicCount = new AtomicInteger(0);
+    private final AtomicInteger globalPartitionCount = new AtomicInteger(0);
+    private final AtomicInteger offlinePartitionCount = new AtomicInteger(0);
+    private final AtomicInteger preferredReplicaImbalanceCount = new 
AtomicInteger(0);
+    private final AtomicInteger metadataErrorCount = new AtomicInteger(0);
+
+    /**
+     * Create a new ControllerMetadataMetrics object.
+     *
+     * @param registry  The metrics registry, or Optional.empty if this is a 
test and we don't have one.
+     */
+    public ControllerMetadataMetrics(Optional<MetricsRegistry> registry) {
+        this.registry = registry;
+        registry.ifPresent(r -> r.newGauge(FENCED_BROKER_COUNT, new 
Gauge<Integer>() {
+            @Override
+            public Integer value() {
+                return fencedBrokerCount();
+            }
+        }));
+        registry.ifPresent(r -> r.newGauge(ACTIVE_BROKER_COUNT, new 
Gauge<Integer>() {
+            @Override
+            public Integer value() {
+                return activeBrokerCount();
+            }
+        }));
+        registry.ifPresent(r -> r.newGauge(GLOBAL_TOPIC_COUNT, new 
Gauge<Integer>() {
+            @Override
+            public Integer value() {
+                return globalTopicCount();
+            }
+        }));
+        registry.ifPresent(r -> r.newGauge(GLOBAL_PARTITION_COUNT, new 
Gauge<Integer>() {
+            @Override
+            public Integer value() {
+                return globalPartitionCount();
+            }
+        }));
+        registry.ifPresent(r -> r.newGauge(OFFLINE_PARTITION_COUNT, new 
Gauge<Integer>() {
+            @Override
+            public Integer value() {
+                return offlinePartitionCount();
+            }
+        }));
+        registry.ifPresent(r -> r.newGauge(PREFERRED_REPLICA_IMBALANCE_COUNT, 
new Gauge<Integer>() {
+            @Override
+            public Integer value() {
+                return preferredReplicaImbalanceCount();
+            }
+        }));
+        registry.ifPresent(r -> r.newGauge(METADATA_ERROR_COUNT, new 
Gauge<Integer>() {
+            @Override
+            public Integer value() {
+                return metadataErrorCount();
+            }
+        }));
+    }
+
+    public void setFencedBrokerCount(int brokerCount) {
+        this.fencedBrokerCount.set(brokerCount);
+    }
+
+    public void addToFencedBrokerCount(int brokerCountDelta) {
+        this.fencedBrokerCount.addAndGet(brokerCountDelta);
+    }
+
+    public int fencedBrokerCount() {
+        return this.fencedBrokerCount.get();
+    }
+
+    public void setActiveBrokerCount(int brokerCount) {
+        this.activeBrokerCount.set(brokerCount);
+    }
+
+    public void addToActiveBrokerCount(int brokerCountDelta) {
+        this.activeBrokerCount.addAndGet(brokerCountDelta);
+    }
+
+    public int activeBrokerCount() {
+        return this.activeBrokerCount.get();
+    }
+
+    public void setGlobalTopicCount(int topicCount) {
+        this.globalTopicCount.set(topicCount);
+    }
+
+    public void addToGlobalTopicCount(int topicCountDelta) {
+        this.globalTopicCount.addAndGet(topicCountDelta);
+    }
+
+    public int globalTopicCount() {
+        return this.globalTopicCount.get();
+    }
+
+    public void setGlobalPartitionCount(int partitionCount) {
+        this.globalPartitionCount.set(partitionCount);
+    }
+
+    public void addToGlobalPartitionCount(int partitionCountDelta) {
+        this.globalPartitionCount.addAndGet(partitionCountDelta);
+    }
+
+    public int globalPartitionCount() {
+        return this.globalPartitionCount.get();
+    }
+
+    public void setOfflinePartitionCount(int offlinePartitions) {
+        this.offlinePartitionCount.set(offlinePartitions);
+    }
+
+    public void addToOfflinePartitionCount(int offlinePartitionsDelta) {
+        this.offlinePartitionCount.addAndGet(offlinePartitionsDelta);
+    }
+
+    public int offlinePartitionCount() {
+        return this.offlinePartitionCount.get();
+    }
+
+    public void setPreferredReplicaImbalanceCount(int replicaImbalances) {
+        this.preferredReplicaImbalanceCount.set(replicaImbalances);
+    }
+
+    public void addToPreferredReplicaImbalanceCount(int 
replicaImbalancesCount) {
+        this.preferredReplicaImbalanceCount.addAndGet(replicaImbalancesCount);
+    }
+
+    public int preferredReplicaImbalanceCount() {
+        return this.preferredReplicaImbalanceCount.get();
+    }
+
+    public void incrementMetadataErrorCount() {
+        this.metadataErrorCount.getAndIncrement();
+    }
+
+    public int metadataErrorCount() {
+        return this.metadataErrorCount.get();
+    }
+
+    @Override
+    public void close() {
+        registry.ifPresent(r -> {
+            Arrays.asList(
+                FENCED_BROKER_COUNT,
+                ACTIVE_BROKER_COUNT,
+                GLOBAL_TOPIC_COUNT,
+                GLOBAL_PARTITION_COUNT,
+                OFFLINE_PARTITION_COUNT,
+                PREFERRED_REPLICA_IMBALANCE_COUNT,
+                METADATA_ERROR_COUNT
+                ).forEach(r::removeMetric);

Review Comment:
   nit: indent



##########
metadata/src/main/java/org/apache/kafka/controller/metrics/ControllerMetadataMetricsPublisher.java:
##########
@@ -0,0 +1,156 @@
+/*
+ * 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.controller.metrics;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.image.MetadataDelta;
+import org.apache.kafka.image.MetadataImage;
+import org.apache.kafka.image.TopicDelta;
+import org.apache.kafka.image.TopicImage;
+import org.apache.kafka.image.loader.LoaderManifest;
+import org.apache.kafka.image.publisher.MetadataPublisher;
+import org.apache.kafka.metadata.BrokerRegistration;
+import org.apache.kafka.metadata.PartitionRegistration;
+import org.apache.kafka.server.fault.FaultHandler;
+
+import java.util.Map.Entry;
+import java.util.Optional;
+
+
+/**
+ * This publisher translates metadata updates sent by MetadataLoader into 
changes to controller
+ * metrics. Like all MetadataPublisher objects, it only receives notifications 
about events that
+ * have been persisted to the metadata log. So on the active controller, it 
will run slightly
+ * behind the latest in-memory state which has not yet been fully persisted to 
the log. This is
+ * reasonable for metrics, which don't need up-to-the-millisecond update 
latency.
+ *
+ * NOTE: the ZK controller has some special rules for calculating 
preferredReplicaImbalanceCount
+ * which we haven't implemented here. Specifically, the ZK controller 
considers reassigning
+ * partitions to always have their preferred leader, even if they don't.
+ * All other metrics should be the same, as far as is possible.
+ */
+public class ControllerMetadataMetricsPublisher implements MetadataPublisher {
+    private final ControllerMetadataMetrics metrics;
+    private final FaultHandler faultHandler;
+    private MetadataImage prevImage = MetadataImage.EMPTY;
+
+    public ControllerMetadataMetricsPublisher(
+        ControllerMetadataMetrics metrics,
+        FaultHandler faultHandler
+    ) {
+        this.metrics = metrics;
+        this.faultHandler = faultHandler;
+    }
+
+    @Override
+    public String name() {
+        return "ControllerMetadataMetricsPublisher";
+    }
+
+    @Override
+    public void onMetadataUpdate(
+            MetadataDelta delta,
+            MetadataImage newImage,
+            LoaderManifest manifest

Review Comment:
   nit: indent should be 4 here



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