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