Browse Source

ZOOKEEPER-1321: Add number of client connections metric in JMX and srvr (Neha Narkhede via camille)

git-svn-id: https://svn.apache.org/repos/asf/zookeeper/trunk@1225200 13f79535-47bb-0310-9956-ffa450edef68
Camille Fournier 13 years ago
parent
commit
6c4b747ea5

+ 2 - 0
CHANGES.txt

@@ -162,6 +162,8 @@ IMPROVEMENTS:
 
   ZOOKEEPER-1342. quorum Listener & LearnerCnxAcceptor are missing
   thread names (Rakesh R via phunt)
+  
+  ZOOKEEPER-1321. Add number of client connections metric in JMX and srvr (Neha Narkhede via camille)
 
 Release 3.4.0 - 
 

+ 2 - 1
src/java/main/org/apache/zookeeper/server/NIOServerCnxn.java

@@ -749,7 +749,8 @@ public class NIOServerCnxn extends ServerCnxn {
 
             print("packets_received", stats.getPacketsReceived());
             print("packets_sent", stats.getPacketsSent());
-
+            print("num_alive_connections", stats.getNumAliveClientConnections());
+            
             print("outstanding_requests", stats.getOutstandingRequests());
 
             print("server_state", stats.getServerState());

+ 9 - 6
src/java/main/org/apache/zookeeper/server/NIOServerCnxnFactory.java

@@ -32,14 +32,14 @@ import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Set;
 
+import javax.security.auth.login.Configuration;
+import javax.security.auth.login.LoginException;
+
 import org.apache.zookeeper.Login;
 import org.apache.zookeeper.server.auth.SaslServerCallbackHandler;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import javax.security.auth.login.Configuration;
-import javax.security.auth.login.LoginException;
-
 public class NIOServerCnxnFactory extends ServerCnxnFactory implements Runnable {
     private static final Logger LOG = LoggerFactory.getLogger(NIOServerCnxnFactory.class);
 
@@ -78,7 +78,6 @@ public class NIOServerCnxnFactory extends ServerCnxnFactory implements Runnable
 
     int maxClientCnxns = 60;
 
-
     /**
      * Construct a new server connection factory which will accept an unlimited number
      * of concurrent connections from each client (up to the file descriptor
@@ -122,7 +121,7 @@ public class NIOServerCnxnFactory extends ServerCnxnFactory implements Runnable
     public void setMaxClientCnxnsPerHost(int max) {
         maxClientCnxns = max;
     }
-
+   
     @Override
     public void start() {
         // ensure thread is started once and only once
@@ -187,7 +186,7 @@ public class NIOServerCnxnFactory extends ServerCnxnFactory implements Runnable
             return s.size();
         }
     }
-
+    
     public void run() {
         while (!ss.socket().isClosed()) {
             try {
@@ -323,4 +322,8 @@ public class NIOServerCnxnFactory extends ServerCnxnFactory implements Runnable
         return cnxns;
     }
 
+    @Override
+    public int getNumAliveConnections() {
+    	return cnxns.size();
+    }
 }

+ 2 - 1
src/java/main/org/apache/zookeeper/server/NettyServerCnxn.java

@@ -560,7 +560,8 @@ public class NettyServerCnxn extends ServerCnxn {
 
             print("packets_received", stats.getPacketsReceived());
             print("packets_sent", stats.getPacketsSent());
-
+            print("num_alive_connections", stats.getNumAliveClientConnections());
+            
             print("outstanding_requests", stats.getOutstandingRequests());
 
             print("server_state", stats.getServerState());

+ 5 - 0
src/java/main/org/apache/zookeeper/server/NettyServerCnxnFactory.java

@@ -411,5 +411,10 @@ public class NettyServerCnxnFactory extends ServerCnxnFactory {
             }
         }
     }
+    
+    @Override
+    public int getNumAliveConnections() {
+    	return cnxns.size();
+    }
 
 }

+ 7 - 4
src/java/main/org/apache/zookeeper/server/ServerCnxnFactory.java

@@ -24,15 +24,16 @@ import java.nio.ByteBuffer;
 import java.util.HashMap;
 
 import javax.management.JMException;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import org.apache.zookeeper.jmx.MBeanRegistry;
+
 import org.apache.zookeeper.Login;
+import org.apache.zookeeper.jmx.MBeanRegistry;
 import org.apache.zookeeper.server.auth.SaslServerCallbackHandler;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 public abstract class ServerCnxnFactory {
 
-    public static final String ZOOKEEPER_SERVER_CNXN_FACTORY = "zookeeper.serverCnxnFactory";
+    public static final String ZOOKEEPER_SERVER_CNXN_FACTORY = "zookeeper.serverCnxnFactory";    
 
     public interface PacketProcessor {
         public void processPacket(ByteBuffer packet, ServerCnxn src);
@@ -49,6 +50,8 @@ public abstract class ServerCnxnFactory {
     
     public abstract Iterable<ServerCnxn> getConnections();
 
+    public abstract int getNumAliveConnections();
+    
     public abstract void closeSession(long sessionId);
 
     public abstract void configure(InetSocketAddress addr,

+ 12 - 4
src/java/main/org/apache/zookeeper/server/ServerStats.java

@@ -19,6 +19,7 @@
 package org.apache.zookeeper.server;
 
 
+
 /**
  * Basic Server Statistics
  */
@@ -29,13 +30,14 @@ public class ServerStats {
     private long minLatency = Long.MAX_VALUE;
     private long totalLatency = 0;
     private long count = 0;
-
+    
     private final Provider provider;
 
     public interface Provider {
         public long getOutstandingRequests();
         public long getLastProcessedZxid();
         public String getState();
+        public int getNumAliveConnections();
     }
     
     public ServerStats(Provider provider) {
@@ -75,9 +77,14 @@ public class ServerStats {
     }
 
     public String getServerState() {
-        return provider.getState();
+    	return provider.getState();
     }
-    
+
+    /** The number of client connections alive to this server */
+    public int getNumAliveClientConnections() {
+    	return provider.getNumAliveConnections();
+    }
+
     @Override
     public String toString(){
         StringBuilder sb = new StringBuilder();
@@ -85,6 +92,8 @@ public class ServerStats {
                 + getAvgLatency() + "/" + getMaxLatency() + "\n");
         sb.append("Received: " + getPacketsReceived() + "\n");
         sb.append("Sent: " + getPacketsSent() + "\n");
+        sb.append("Connections: " + getNumAliveClientConnections() + "\n");
+        
         if (provider != null) {
             sb.append("Outstanding: " + getOutstandingRequests() + "\n");
             sb.append("Zxid: 0x"+ Long.toHexString(getLastProcessedZxid())+ "\n");
@@ -123,7 +132,6 @@ public class ServerStats {
         packetsReceived = 0;
         packetsSent = 0;
     }
-    
     synchronized public void reset() {
         resetLatency();
         resetRequestCounters();

+ 9 - 2
src/java/main/org/apache/zookeeper/server/ZooKeeperServer.java

@@ -109,7 +109,7 @@ public class ZooKeeperServer implements SessionExpirer, ServerStats.Provider {
     private ServerCnxnFactory serverCnxnFactory;
 
     private final ServerStats serverStats;
-
+ 
     void removeCnxn(ServerCnxn cnxn) {
         zkDb.removeCnxn(cnxn);
     }
@@ -254,7 +254,6 @@ public class ZooKeeperServer implements SessionExpirer, ServerStats.Provider {
         }
     }
 
-
     /**
      * This should be called from a synchronized block on this!
      */
@@ -677,6 +676,14 @@ public class ZooKeeperServer implements SessionExpirer, ServerStats.Provider {
         return getInProcess();
     }
 
+    /**
+     * return the total number of client connections that are alive
+     * to this server
+     */
+    public int getNumAliveConnections() {
+    	return serverCnxnFactory.getNumAliveConnections();
+    }
+    
     /**
      * trunccate the log to get in sync with others
      * if in a quorum

+ 4 - 0
src/java/main/org/apache/zookeeper/server/ZooKeeperServerBean.java

@@ -140,4 +140,8 @@ public class ZooKeeperServerBean implements ZooKeeperServerMXBean, ZKMBeanInfo {
         serverStats.resetRequestCounters();
         serverStats.resetLatency();
     }
+
+	public long getNumAliveConnections() {
+		return zks.getNumAliveConnections();
+	}
 }

+ 4 - 0
src/java/main/org/apache/zookeeper/server/ZooKeeperServerMXBean.java

@@ -103,4 +103,8 @@ public interface ZooKeeperServerMXBean {
      * Reset max latency statistics only.
      */
     public void resetMaxLatency();
+    /**
+     * @return number of alive client connections
+     */
+    public long getNumAliveConnections();
 }

+ 4 - 0
src/java/test/org/apache/zookeeper/server/quorum/Zab1_0Test.java

@@ -212,6 +212,10 @@ public class Zab1_0Test {
         }
         public void closeAll() {
         }
+		@Override
+		public int getNumAliveConnections() {			
+			return 0;
+		}
     }
     static Socket[] getSocketPair() throws IOException {
         ServerSocket ss = new ServerSocket();

+ 5 - 0
src/java/test/org/apache/zookeeper/test/FourLetterWordsTest.java

@@ -94,6 +94,9 @@ public class FourLetterWordsTest extends ClientBase {
         verify("srvr", "Outstanding");
         verify("cons", "queued");
         verify("mntr", "zk_server_state\tstandalone");
+        verify("mntr", "num_alive_connections");
+        verify("stat", "Connections");
+        verify("srvr", "Connections");
     }
 
     private String sendRequest(String cmd) throws IOException {
@@ -136,6 +139,8 @@ public class FourLetterWordsTest extends ClientBase {
         line = in.readLine();
         Assert.assertTrue(Pattern.matches("^Sent: \\d+$", line));
         line = in.readLine();
+        Assert.assertTrue(Pattern.matches("^Connections: \\d+$", line));
+        line = in.readLine();
         Assert.assertTrue(Pattern.matches("^Outstanding: \\d+$", line));
         line = in.readLine();
         Assert.assertTrue(Pattern.matches("^Zxid: 0x[\\da-fA-F]+$", line));