Pārlūkot izejas kodu

ZOOKEEPER-4360: Avoid NPE during gauge execution if the leader is not set on FOLLOWER node

On a follower node, we had this error
```
ago 20, 2021 1:46:28 PM org.apache.catalina.core.StandardWrapperValve invoke
GRAVE: Servlet.service() for servlet [metrics] in context with path [/metrics] threw exception
java.lang.NullPointerException: Cannot invoke "org.apache.zookeeper.server.quorum.Leader.getProposalStats()" because the return value of "org.apache.zookeeper.server.quorum.LeaderZooKeeperServer.getLeader()" is null
        at org.apache.zookeeper.server.quorum.LeaderZooKeeperServer.lambda$registerMetrics$5(LeaderZooKeeperServer.java:122)
        at magnews.zookeeper.ZooKeeperMetricsProviderAdapter$MetricsContextImpl.lambda$registerGauge$0(ZooKeeperMetricsProviderAdapter.java:91)
```

Unfortunately, I'm not able to reproduce this error deterministically

I've added the null check on leader variable, this will cause the gauge return value to be not registered (on `DefaultMetricsProvider`) and considered as zero on prometheus

Not sure if tests were needed even if the fix is so simple, I can them if requested from reviewers

Author: Nicolò Boschi <boschi1997@gmail.com>

Reviewers: Enrico Olivelli <eolivelli@apache.org>, Damien Diederen <ddiederen@apache.org>

Closes #1743 from nicoloboschi/fix/npe-metrics
Nicolò Boschi 4 gadi atpakaļ
vecāks
revīzija
4f51567346

+ 34 - 26
zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/LeaderZooKeeperServer.java

@@ -20,6 +20,7 @@ package org.apache.zookeeper.server.quorum;
 
 import java.io.IOException;
 import java.util.concurrent.TimeUnit;
+import java.util.function.Function;
 import javax.management.JMException;
 import org.apache.zookeeper.KeeperException.SessionExpiredException;
 import org.apache.zookeeper.jmx.MBeanRegistry;
@@ -99,34 +100,41 @@ public class LeaderZooKeeperServer extends QuorumZooKeeperServer {
         super.registerMetrics();
 
         MetricsContext rootContext = ServerMetrics.getMetrics().getMetricsProvider().getRootContext();
-
-        rootContext.registerGauge("learners", () -> {
-            return getLeader().getLearners().size();
-        });
-        rootContext.registerGauge("synced_followers", () -> {
-            return getLeader().getForwardingFollowers().size();
-        });
-        rootContext.registerGauge("synced_non_voting_followers", () -> {
-            return getLeader().getNonVotingFollowers().size();
-        });
-
+        rootContext.registerGauge("learners", gaugeWithLeader(
+                (leader) -> leader.getLearners().size())
+        );
+        rootContext.registerGauge("synced_followers", gaugeWithLeader(
+                (leader) -> leader.getForwardingFollowers().size()
+        ));
+        rootContext.registerGauge("synced_non_voting_followers", gaugeWithLeader(
+                (leader) -> leader.getNonVotingFollowers().size()
+        ));
         rootContext.registerGauge("synced_observers", self::getSynced_observers_metric);
+        rootContext.registerGauge("pending_syncs", gaugeWithLeader(
+                (leader) -> leader.getNumPendingSyncs()
+        ));
+        rootContext.registerGauge("leader_uptime", gaugeWithLeader(
+                (leader) -> leader.getUptime()
+        ));
+        rootContext.registerGauge("last_proposal_size", gaugeWithLeader(
+                (leader) -> leader.getProposalStats().getLastBufferSize()
+        ));
+        rootContext.registerGauge("max_proposal_size", gaugeWithLeader(
+                (leader) -> leader.getProposalStats().getMaxBufferSize()
+        ));
+        rootContext.registerGauge("min_proposal_size", gaugeWithLeader(
+                (leader) -> leader.getProposalStats().getMinBufferSize()
+        ));
+    }
 
-        rootContext.registerGauge("pending_syncs", () -> {
-            return getLeader().getNumPendingSyncs();
-        });
-        rootContext.registerGauge("leader_uptime", () -> {
-            return getLeader().getUptime();
-        });
-        rootContext.registerGauge("last_proposal_size", () -> {
-            return getLeader().getProposalStats().getLastBufferSize();
-        });
-        rootContext.registerGauge("max_proposal_size", () -> {
-            return getLeader().getProposalStats().getMaxBufferSize();
-        });
-        rootContext.registerGauge("min_proposal_size", () -> {
-            return getLeader().getProposalStats().getMinBufferSize();
-        });
+    private org.apache.zookeeper.metrics.Gauge gaugeWithLeader(Function<Leader, Number> supplier) {
+        return () -> {
+            final Leader leader = getLeader();
+            if (leader == null) {
+                return null;
+            }
+            return supplier.apply(leader);
+        };
     }
 
     @Override