I'm not sure I agree in the assumption that monitoring pulls happen infrequently... On Dec 28, 2011 4:24 PM, "Patrick Hunt" <[email protected]> wrote:
> They seem like distinct changes to me. In particular getting the size > is going to happen infrequently (monitoring pull) so I don't see a > problem fixing the existing patch in the same way the code currently > handles cnxns access, with a separate jira to do the refactoring. Am I > missing something? > > Patrick > > On Wed, Dec 28, 2011 at 1:10 PM, Camille Fournier <[email protected]> > wrote: > > I don't think creating it as a flat sync is a good idea, so if we want > this > > I think we need to refactor that structure to be concurrent. > > > > C > > On Dec 28, 2011 2:24 PM, "Patrick Hunt" <[email protected]> wrote: > > > >> I think it needs to be fixed. It's obviously incorrect. Also if > >> someone changes the underlying implementation at some point it might > >> bite them. > >> > >> Given it wasn't applied to 3.4 branch yet I'd suggest revert, fix the > >> patch, then reapply to both branches. (that's easiest/bulletproof imo) > >> > >> I'd recommend a new ticket to fix the issues you identified (refactor, > >> etc...) Neha was looking for further items to work on, this would be a > >> good one. > >> > >> Patrick > >> > >> On Wed, Dec 28, 2011 at 11:18 AM, Camille Fournier <[email protected]> > >> wrote: > >> > Also, Pat, the JIRA is still open. I will look to revert in a minute > >> > although I think the sync changes may necessitate a new JIRA entirely. > >> > > >> > On Wed, Dec 28, 2011 at 1:31 PM, Camille Fournier <[email protected] > > > >> wrote: > >> >> After looking for a few mins, here are my observations: > >> >> > >> >> The implementation of HashSet in jdk 1.6 uses an underlying HashMap, > >> which > >> >> uses a bog-standard int for the size. So, I completely agree that we > can > >> >> get invalid results for a point in time. But does anyone really care > >> about > >> >> the *exact* moment-in-time number of cnxns in the system, in a system > >> where > >> >> cnxns are coming and going? At best you'll see a value that will > likely > >> be > >> >> out of date the moment you read it. > >> >> > >> >> But, if we're going to do this, I do think there may be a case for > >> >> refactoring to a concurrent data structure given that we are randomly > >> >> syncing on the cnxns set in a way that makes adding additional > >> interactions > >> >> with it error prone in this way. And we should definitely remove that > >> >> getConnections method, if anyone ever iterated over that set they > would > >> be > >> >> in a sad concurrency situation. > >> >> Seems possibly worth a whole new ticket to do this. What do you > think? > >> >> > >> >> Thanks, > >> >> C > >> >> > >> >> On Wed, Dec 28, 2011 at 1:12 PM, Camille Fournier < > [email protected] > >> >wrote: > >> >> > >> >>> I'm not sure it's meaningful enough to be worth the sync overhead. > We > >> >>> should look into that. The alternative if we must is using a proper > >> >>> concurrent collection. Neha, any thoughts? > >> >>> > >> >>> C > >> >>> On Dec 28, 2011 12:45 PM, "Patrick Hunt" <[email protected]> wrote: > >> >>> > >> >>>> I believe there is a bug in this commit. The "cnxns" size() call is > >> >>>> not being synchronized. This will lead to invalid results at best, > at > >> >>>> worst outright failure (hard to say w/o knowing the implementation > of > >> >>>> HashSet). > >> >>>> > >> >>>> Camille can you work with Neha to get this fixed? Perhaps in the > >> >>>> meantime (if it's going to take a while) you can revert this > change, > >> >>>> re-open the jira, update the patch, and reapply at some later time? > >> >>>> > >> >>>> Patrick > >> >>>> > >> >>>> On Wed, Dec 28, 2011 at 6:55 AM, <[email protected]> wrote: > >> >>>> > Author: camille > >> >>>> > Date: Wed Dec 28 14:55:37 2011 > >> >>>> > New Revision: 1225200 > >> >>>> > > >> >>>> > URL: http://svn.apache.org/viewvc?rev=1225200&view=rev > >> >>>> > Log: > >> >>>> > ZOOKEEPER-1321: Add number of client connections metric in JMX > and > >> srvr > >> >>>> (Neha Narkhede via camille) > >> >>>> > > >> >>>> > Modified: > >> >>>> > zookeeper/trunk/CHANGES.txt > >> >>>> > > >> >>>> > >> > zookeeper/trunk/src/java/main/org/apache/zookeeper/server/NIOServerCnxn.java > >> >>>> > > >> >>>> > >> > > zookeeper/trunk/src/java/main/org/apache/zookeeper/server/NIOServerCnxnFactory.java > >> >>>> > > >> >>>> > >> > > zookeeper/trunk/src/java/main/org/apache/zookeeper/server/NettyServerCnxn.java > >> >>>> > > >> >>>> > >> > > zookeeper/trunk/src/java/main/org/apache/zookeeper/server/NettyServerCnxnFactory.java > >> >>>> > > >> >>>> > >> > > zookeeper/trunk/src/java/main/org/apache/zookeeper/server/ServerCnxnFactory.java > >> >>>> > > >> >>>> > >> > zookeeper/trunk/src/java/main/org/apache/zookeeper/server/ServerStats.java > >> >>>> > > >> >>>> > >> > > zookeeper/trunk/src/java/main/org/apache/zookeeper/server/ZooKeeperServer.java > >> >>>> > > >> >>>> > >> > > zookeeper/trunk/src/java/main/org/apache/zookeeper/server/ZooKeeperServerBean.java > >> >>>> > > >> >>>> > >> > > zookeeper/trunk/src/java/main/org/apache/zookeeper/server/ZooKeeperServerMXBean.java > >> >>>> > > >> >>>> > >> > > zookeeper/trunk/src/java/test/org/apache/zookeeper/server/quorum/Zab1_0Test.java > >> >>>> > > >> >>>> > >> > > zookeeper/trunk/src/java/test/org/apache/zookeeper/test/FourLetterWordsTest.java > >> >>>> > > >> >>>> > Modified: zookeeper/trunk/CHANGES.txt > >> >>>> > URL: > >> >>>> > >> > http://svn.apache.org/viewvc/zookeeper/trunk/CHANGES.txt?rev=1225200&r1=1225199&r2=1225200&view=diff > >> >>>> > > >> >>>> > >> > ============================================================================== > >> >>>> > --- zookeeper/trunk/CHANGES.txt (original) > >> >>>> > +++ zookeeper/trunk/CHANGES.txt Wed Dec 28 14:55:37 2011 > >> >>>> > @@ -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 - > >> >>>> > > >> >>>> > > >> >>>> > Modified: > >> >>>> > >> > zookeeper/trunk/src/java/main/org/apache/zookeeper/server/NIOServerCnxn.java > >> >>>> > URL: > >> >>>> > >> > http://svn.apache.org/viewvc/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/NIOServerCnxn.java?rev=1225200&r1=1225199&r2=1225200&view=diff > >> >>>> > > >> >>>> > >> > ============================================================================== > >> >>>> > --- > >> >>>> > >> > zookeeper/trunk/src/java/main/org/apache/zookeeper/server/NIOServerCnxn.java > >> >>>> (original) > >> >>>> > +++ > >> >>>> > >> > zookeeper/trunk/src/java/main/org/apache/zookeeper/server/NIOServerCnxn.java > >> >>>> Wed Dec 28 14:55:37 2011 > >> >>>> > @@ -749,7 +749,8 @@ public class NIOServerCnxn extends Serve > >> >>>> > > >> >>>> > 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()); > >> >>>> > > >> >>>> > Modified: > >> >>>> > >> > zookeeper/trunk/src/java/main/org/apache/zookeeper/server/NIOServerCnxnFactory.java > >> >>>> > URL: > >> >>>> > >> > http://svn.apache.org/viewvc/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/NIOServerCnxnFactory.java?rev=1225200&r1=1225199&r2=1225200&view=diff > >> >>>> > > >> >>>> > >> > ============================================================================== > >> >>>> > --- > >> >>>> > >> > zookeeper/trunk/src/java/main/org/apache/zookeeper/server/NIOServerCnxnFactory.java > >> >>>> (original) > >> >>>> > +++ > >> >>>> > >> > zookeeper/trunk/src/java/main/org/apache/zookeeper/server/NIOServerCnxnFactory.java > >> >>>> Wed Dec 28 14:55:37 2011 > >> >>>> > @@ -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 extend > >> >>>> > > >> >>>> > 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 extend > >> >>>> > 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 extend > >> >>>> > return s.size(); > >> >>>> > } > >> >>>> > } > >> >>>> > - > >> >>>> > + > >> >>>> > public void run() { > >> >>>> > while (!ss.socket().isClosed()) { > >> >>>> > try { > >> >>>> > @@ -323,4 +322,8 @@ public class NIOServerCnxnFactory extend > >> >>>> > return cnxns; > >> >>>> > } > >> >>>> > > >> >>>> > + @Override > >> >>>> > + public int getNumAliveConnections() { > >> >>>> > + return cnxns.size(); > >> >>>> > + } > >> >>>> > } > >> >>>> > > >> >>>> > Modified: > >> >>>> > >> > zookeeper/trunk/src/java/main/org/apache/zookeeper/server/NettyServerCnxn.java > >> >>>> > URL: > >> >>>> > >> > http://svn.apache.org/viewvc/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/NettyServerCnxn.java?rev=1225200&r1=1225199&r2=1225200&view=diff > >> >>>> > > >> >>>> > >> > ============================================================================== > >> >>>> > --- > >> >>>> > >> > zookeeper/trunk/src/java/main/org/apache/zookeeper/server/NettyServerCnxn.java > >> >>>> (original) > >> >>>> > +++ > >> >>>> > >> > zookeeper/trunk/src/java/main/org/apache/zookeeper/server/NettyServerCnxn.java > >> >>>> Wed Dec 28 14:55:37 2011 > >> >>>> > @@ -560,7 +560,8 @@ public class NettyServerCnxn extends Ser > >> >>>> > > >> >>>> > 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()); > >> >>>> > > >> >>>> > Modified: > >> >>>> > >> > zookeeper/trunk/src/java/main/org/apache/zookeeper/server/NettyServerCnxnFactory.java > >> >>>> > URL: > >> >>>> > >> > http://svn.apache.org/viewvc/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/NettyServerCnxnFactory.java?rev=1225200&r1=1225199&r2=1225200&view=diff > >> >>>> > > >> >>>> > >> > ============================================================================== > >> >>>> > --- > >> >>>> > >> > zookeeper/trunk/src/java/main/org/apache/zookeeper/server/NettyServerCnxnFactory.java > >> >>>> (original) > >> >>>> > +++ > >> >>>> > >> > zookeeper/trunk/src/java/main/org/apache/zookeeper/server/NettyServerCnxnFactory.java > >> >>>> Wed Dec 28 14:55:37 2011 > >> >>>> > @@ -411,5 +411,10 @@ public class NettyServerCnxnFactory exte > >> >>>> > } > >> >>>> > } > >> >>>> > } > >> >>>> > + > >> >>>> > + @Override > >> >>>> > + public int getNumAliveConnections() { > >> >>>> > + return cnxns.size(); > >> >>>> > + } > >> >>>> > > >> >>>> > } > >> >>>> > > >> >>>> > Modified: > >> >>>> > >> > zookeeper/trunk/src/java/main/org/apache/zookeeper/server/ServerCnxnFactory.java > >> >>>> > URL: > >> >>>> > >> > http://svn.apache.org/viewvc/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/ServerCnxnFactory.java?rev=1225200&r1=1225199&r2=1225200&view=diff > >> >>>> > > >> >>>> > >> > ============================================================================== > >> >>>> > --- > >> >>>> > >> > zookeeper/trunk/src/java/main/org/apache/zookeeper/server/ServerCnxnFactory.java > >> >>>> (original) > >> >>>> > +++ > >> >>>> > >> > zookeeper/trunk/src/java/main/org/apache/zookeeper/server/ServerCnxnFactory.java > >> >>>> Wed Dec 28 14:55:37 2011 > >> >>>> > @@ -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, > >> >>>> > > >> >>>> > Modified: > >> >>>> > >> > zookeeper/trunk/src/java/main/org/apache/zookeeper/server/ServerStats.java > >> >>>> > URL: > >> >>>> > >> > http://svn.apache.org/viewvc/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/ServerStats.java?rev=1225200&r1=1225199&r2=1225200&view=diff > >> >>>> > > >> >>>> > >> > ============================================================================== > >> >>>> > --- > >> >>>> > >> > zookeeper/trunk/src/java/main/org/apache/zookeeper/server/ServerStats.java > >> >>>> (original) > >> >>>> > +++ > >> >>>> > >> > zookeeper/trunk/src/java/main/org/apache/zookeeper/server/ServerStats.java > >> >>>> Wed Dec 28 14:55:37 2011 > >> >>>> > @@ -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(); > >> >>>> > > >> >>>> > Modified: > >> >>>> > >> > zookeeper/trunk/src/java/main/org/apache/zookeeper/server/ZooKeeperServer.java > >> >>>> > URL: > >> >>>> > >> > http://svn.apache.org/viewvc/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/ZooKeeperServer.java?rev=1225200&r1=1225199&r2=1225200&view=diff > >> >>>> > > >> >>>> > >> > ============================================================================== > >> >>>> > --- > >> >>>> > >> > zookeeper/trunk/src/java/main/org/apache/zookeeper/server/ZooKeeperServer.java > >> >>>> (original) > >> >>>> > +++ > >> >>>> > >> > zookeeper/trunk/src/java/main/org/apache/zookeeper/server/ZooKeeperServer.java > >> >>>> Wed Dec 28 14:55:37 2011 > >> >>>> > @@ -109,7 +109,7 @@ public class ZooKeeperServer implements > >> >>>> > private ServerCnxnFactory serverCnxnFactory; > >> >>>> > > >> >>>> > private final ServerStats serverStats; > >> >>>> > - > >> >>>> > + > >> >>>> > void removeCnxn(ServerCnxn cnxn) { > >> >>>> > zkDb.removeCnxn(cnxn); > >> >>>> > } > >> >>>> > @@ -254,7 +254,6 @@ public class ZooKeeperServer implements > >> >>>> > } > >> >>>> > } > >> >>>> > > >> >>>> > - > >> >>>> > /** > >> >>>> > * This should be called from a synchronized block on this! > >> >>>> > */ > >> >>>> > @@ -678,6 +677,14 @@ public class ZooKeeperServer implements > >> >>>> > } > >> >>>> > > >> >>>> > /** > >> >>>> > + * 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 > >> >>>> > * @param zxid the zxid that it needs to get in sync > >> >>>> > > >> >>>> > Modified: > >> >>>> > >> > zookeeper/trunk/src/java/main/org/apache/zookeeper/server/ZooKeeperServerBean.java > >> >>>> > URL: > >> >>>> > >> > http://svn.apache.org/viewvc/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/ZooKeeperServerBean.java?rev=1225200&r1=1225199&r2=1225200&view=diff > >> >>>> > > >> >>>> > >> > ============================================================================== > >> >>>> > --- > >> >>>> > >> > zookeeper/trunk/src/java/main/org/apache/zookeeper/server/ZooKeeperServerBean.java > >> >>>> (original) > >> >>>> > +++ > >> >>>> > >> > zookeeper/trunk/src/java/main/org/apache/zookeeper/server/ZooKeeperServerBean.java > >> >>>> Wed Dec 28 14:55:37 2011 > >> >>>> > @@ -140,4 +140,8 @@ public class ZooKeeperServerBean impleme > >> >>>> > serverStats.resetRequestCounters(); > >> >>>> > serverStats.resetLatency(); > >> >>>> > } > >> >>>> > + > >> >>>> > + public long getNumAliveConnections() { > >> >>>> > + return zks.getNumAliveConnections(); > >> >>>> > + } > >> >>>> > } > >> >>>> > > >> >>>> > Modified: > >> >>>> > >> > zookeeper/trunk/src/java/main/org/apache/zookeeper/server/ZooKeeperServerMXBean.java > >> >>>> > URL: > >> >>>> > >> > http://svn.apache.org/viewvc/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/ZooKeeperServerMXBean.java?rev=1225200&r1=1225199&r2=1225200&view=diff > >> >>>> > > >> >>>> > >> > ============================================================================== > >> >>>> > --- > >> >>>> > >> > zookeeper/trunk/src/java/main/org/apache/zookeeper/server/ZooKeeperServerMXBean.java > >> >>>> (original) > >> >>>> > +++ > >> >>>> > >> > zookeeper/trunk/src/java/main/org/apache/zookeeper/server/ZooKeeperServerMXBean.java > >> >>>> Wed Dec 28 14:55:37 2011 > >> >>>> > @@ -103,4 +103,8 @@ public interface ZooKeeperServerMXBean { > >> >>>> > * Reset max latency statistics only. > >> >>>> > */ > >> >>>> > public void resetMaxLatency(); > >> >>>> > + /** > >> >>>> > + * @return number of alive client connections > >> >>>> > + */ > >> >>>> > + public long getNumAliveConnections(); > >> >>>> > } > >> >>>> > > >> >>>> > Modified: > >> >>>> > >> > zookeeper/trunk/src/java/test/org/apache/zookeeper/server/quorum/Zab1_0Test.java > >> >>>> > URL: > >> >>>> > >> > http://svn.apache.org/viewvc/zookeeper/trunk/src/java/test/org/apache/zookeeper/server/quorum/Zab1_0Test.java?rev=1225200&r1=1225199&r2=1225200&view=diff > >> >>>> > > >> >>>> > >> > ============================================================================== > >> >>>> > --- > >> >>>> > >> > zookeeper/trunk/src/java/test/org/apache/zookeeper/server/quorum/Zab1_0Test.java > >> >>>> (original) > >> >>>> > +++ > >> >>>> > >> > zookeeper/trunk/src/java/test/org/apache/zookeeper/server/quorum/Zab1_0Test.java > >> >>>> Wed Dec 28 14:55:37 2011 > >> >>>> > @@ -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(); > >> >>>> > > >> >>>> > Modified: > >> >>>> > >> > zookeeper/trunk/src/java/test/org/apache/zookeeper/test/FourLetterWordsTest.java > >> >>>> > URL: > >> >>>> > >> > http://svn.apache.org/viewvc/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/FourLetterWordsTest.java?rev=1225200&r1=1225199&r2=1225200&view=diff > >> >>>> > > >> >>>> > >> > ============================================================================== > >> >>>> > --- > >> >>>> > >> > zookeeper/trunk/src/java/test/org/apache/zookeeper/test/FourLetterWordsTest.java > >> >>>> (original) > >> >>>> > +++ > >> >>>> > >> > zookeeper/trunk/src/java/test/org/apache/zookeeper/test/FourLetterWordsTest.java > >> >>>> Wed Dec 28 14:55:37 2011 > >> >>>> > @@ -94,6 +94,9 @@ public class FourLetterWordsTest extends > >> >>>> > 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 > >> >>>> > 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)); > >> >>>> > > >> >>>> > > >> >>>> > >> >>> > >> >
