Browse Source

[ 1913998 ] JMX instrumentation

git-svn-id: https://svn.apache.org/repos/asf/hadoop/zookeeper/trunk@670963 13f79535-47bb-0310-9956-ffa450edef68
Benjamin Reed 17 years ago
parent
commit
f1dd874ed7
42 changed files with 2645 additions and 0 deletions
  1. 27 0
      zookeeper/java/jmx/com/yahoo/zookeeper/jmx/CommonNames.java
  2. 173 0
      zookeeper/java/jmx/com/yahoo/zookeeper/jmx/MBeanRegistry.java
  3. 34 0
      zookeeper/java/jmx/com/yahoo/zookeeper/jmx/ZKMBeanInfo.java
  4. 109 0
      zookeeper/java/jmx/com/yahoo/zookeeper/jmx/server/ConnectionBean.java
  5. 66 0
      zookeeper/java/jmx/com/yahoo/zookeeper/jmx/server/ConnectionMXBean.java
  6. 89 0
      zookeeper/java/jmx/com/yahoo/zookeeper/jmx/server/DataTreeBean.java
  7. 44 0
      zookeeper/java/jmx/com/yahoo/zookeeper/jmx/server/DataTreeMXBean.java
  8. 95 0
      zookeeper/java/jmx/com/yahoo/zookeeper/jmx/server/ZooKeeperServerBean.java
  9. 72 0
      zookeeper/java/jmx/com/yahoo/zookeeper/jmx/server/ZooKeeperServerMXBean.java
  10. 26 0
      zookeeper/java/jmx/com/yahoo/zookeeper/jmx/server/quorum/FollowerBean.java
  11. 24 0
      zookeeper/java/jmx/com/yahoo/zookeeper/jmx/server/quorum/FollowerMXBean.java
  12. 27 0
      zookeeper/java/jmx/com/yahoo/zookeeper/jmx/server/quorum/LeaderBean.java
  13. 41 0
      zookeeper/java/jmx/com/yahoo/zookeeper/jmx/server/quorum/LeaderElectionBean.java
  14. 32 0
      zookeeper/java/jmx/com/yahoo/zookeeper/jmx/server/quorum/LeaderElectionMXBean.java
  15. 24 0
      zookeeper/java/jmx/com/yahoo/zookeeper/jmx/server/quorum/LeaderMXBean.java
  16. 46 0
      zookeeper/java/jmx/com/yahoo/zookeeper/jmx/server/quorum/LocalPeerBean.java
  17. 32 0
      zookeeper/java/jmx/com/yahoo/zookeeper/jmx/server/quorum/LocalPeerMXBean.java
  18. 42 0
      zookeeper/java/jmx/com/yahoo/zookeeper/jmx/server/quorum/QuorumBean.java
  19. 31 0
      zookeeper/java/jmx/com/yahoo/zookeeper/jmx/server/quorum/QuorumMXBean.java
  20. 43 0
      zookeeper/java/jmx/com/yahoo/zookeeper/jmx/server/quorum/RemotePeerBean.java
  21. 31 0
      zookeeper/java/jmx/com/yahoo/zookeeper/jmx/server/quorum/RemotePeerMXBean.java
  22. 36 0
      zookeeper/java/jmx/com/yahoo/zookeeper/jmx/server/quorum/ServerBean.java
  23. 31 0
      zookeeper/java/jmx/com/yahoo/zookeeper/jmx/server/quorum/ServerMXBean.java
  24. 138 0
      zookeeper/java/jmx/com/yahoo/zookeeper/server/ManagedZooKeeperServer.java
  25. 111 0
      zookeeper/java/jmx/com/yahoo/zookeeper/server/ObservableDataTree.java
  26. 83 0
      zookeeper/java/jmx/com/yahoo/zookeeper/server/ObservableNIOServerCnxn.java
  27. 50 0
      zookeeper/java/jmx/com/yahoo/zookeeper/server/ObservableZooKeeperServer.java
  28. 74 0
      zookeeper/java/jmx/com/yahoo/zookeeper/server/ZooKeeperObserverNotifier.java
  29. 243 0
      zookeeper/java/jmx/com/yahoo/zookeeper/server/quorum/ManagedQuorumPeer.java
  30. 76 0
      zookeeper/java/jmx/com/yahoo/zookeeper/server/quorum/ObservableFollower.java
  31. 53 0
      zookeeper/java/jmx/com/yahoo/zookeeper/server/quorum/ObservableFollowerZooKeeperServer.java
  32. 80 0
      zookeeper/java/jmx/com/yahoo/zookeeper/server/quorum/ObservableLeader.java
  33. 53 0
      zookeeper/java/jmx/com/yahoo/zookeeper/server/quorum/ObservableLeaderZooKeeperServer.java
  34. 107 0
      zookeeper/java/jmx/com/yahoo/zookeeper/server/quorum/ObservableQuorumPeer.java
  35. 36 0
      zookeeper/java/jmx/com/yahoo/zookeeper/server/util/ConnectionObserver.java
  36. 51 0
      zookeeper/java/jmx/com/yahoo/zookeeper/server/util/DataTreeObserver.java
  37. 28 0
      zookeeper/java/jmx/com/yahoo/zookeeper/server/util/EventInfo.java
  38. 32 0
      zookeeper/java/jmx/com/yahoo/zookeeper/server/util/ObservableComponent.java
  39. 93 0
      zookeeper/java/jmx/com/yahoo/zookeeper/server/util/ObserverManager.java
  40. 67 0
      zookeeper/java/jmx/com/yahoo/zookeeper/server/util/QuorumPeerObserver.java
  41. 36 0
      zookeeper/java/jmx/com/yahoo/zookeeper/server/util/ServerObserver.java
  42. 59 0
      zookeeper/java/jmx/com/yahoo/zookeeper/server/util/ZooKeeperObserverManager.java

+ 27 - 0
zookeeper/java/jmx/com/yahoo/zookeeper/jmx/CommonNames.java

@@ -0,0 +1,27 @@
+/**
+ * Copyright 2008, Yahoo! Inc.
+ *
+ * Licensed 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 com.yahoo.zookeeper.jmx;
+
+/**
+ * A bunch of constants.
+ * TODO: will get rid of it eventually.
+ */
+public class CommonNames {
+    public static final String DOMAIN="com.yahoo.ZookeeperService";
+    public static final String DATA_TREE_KEY="DataTree";
+    public static final String STANDALONE_SERVER_KEY="StandaloneServer";
+}

+ 173 - 0
zookeeper/java/jmx/com/yahoo/zookeeper/jmx/MBeanRegistry.java

@@ -0,0 +1,173 @@
+/**
+ * Copyright 2008, Yahoo! Inc.
+ *
+ * Licensed 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 com.yahoo.zookeeper.jmx;
+
+import java.lang.management.ManagementFactory;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+
+import javax.management.MBeanServer;
+import javax.management.ObjectName;
+
+import org.apache.log4j.Logger;
+
+/**
+ * This class provides a unified interface for registering/unregistering of
+ * zookeeper MBeans with the platform MBean server. It builds a hierarchy of MBeans
+ * where each MBean represented by a filesystem-like path. Eventually, this hierarchy
+ * will be stored in the zookeeper data tree instance as a virtual data tree.
+ */
+public class MBeanRegistry {
+    private static final Logger LOG = Logger.getLogger(MBeanRegistry.class);
+    private static MBeanRegistry instance=new MBeanRegistry(); 
+    private Map<ZKMBeanInfo, String> mapBean2Path = new ConcurrentHashMap<ZKMBeanInfo, String>();
+    private Map<String, ZKMBeanInfo> mapName2Bean = new ConcurrentHashMap<String, ZKMBeanInfo>();
+
+    public static MBeanRegistry getInstance(){
+        return instance;
+    }
+    
+    /**
+     * Registers a new MBean with the platform MBean server. 
+     * @param bean the bean being registered
+     * @param parent if not null, the new bean will be registered as a child
+     * node of this parent.
+     */
+    public void register(ZKMBeanInfo bean, ZKMBeanInfo parent) {
+        assert bean != null;
+        String path = null;
+        if (parent != null) {
+            path = mapBean2Path.get(parent);
+            assert path != null;
+        }
+        path = makeFullPath(path, parent);
+        mapBean2Path.put(bean, path);
+        mapName2Bean.put(bean.getName(), bean);
+        if(bean.isHidden())
+            return;
+        MBeanServer mbs = ManagementFactory.getPlatformMBeanServer();
+        ObjectName oname = makeObjectName(path, bean);
+        try {
+            mbs.registerMBean(bean, oname);
+        } catch (Exception e) {
+            LOG.error("Failed to register MBean " + bean.getName());
+            e.printStackTrace();
+        }
+    }
+
+    /**
+     * Unregister the MBean identified by the path.
+     * @param path
+     * @param bean
+     */
+    private void unregister(String path,ZKMBeanInfo bean){
+        if(path==null)
+            return;
+        if (!bean.isHidden()) {
+            MBeanServer mbs = ManagementFactory.getPlatformMBeanServer();
+            try {
+                mbs.unregisterMBean(makeObjectName(path, bean));
+            } catch (Exception e) {
+                LOG.error("Failed to unregister MBean " + bean.getName());
+                e.printStackTrace();
+            }
+        }        
+    }
+    
+    /**
+     * Unregister MBean.
+     * @param bean
+     */
+    public void unregister(ZKMBeanInfo bean){
+        if(bean==null)
+            return;
+        String path=mapBean2Path.get(bean);
+        unregister(path,bean);
+        mapBean2Path.remove(bean);
+        mapName2Bean.remove(bean.getName());
+    }
+    /**
+     * Unregister all currently registered MBeans
+     */
+    public void unregisterAll(){
+        for(Map.Entry<ZKMBeanInfo,String> e: mapBean2Path.entrySet()){
+            unregister(e.getValue(),e.getKey());
+        }
+        mapBean2Path.clear();
+        mapName2Bean.clear();
+    }
+    /**
+     * Generate a filesystem-like path.
+     * @param prefix path prefix
+     * @param name path elements
+     * @return absolute path
+     */
+    public String makeFullPath(String prefix, String... name){
+        StringBuilder sb=new StringBuilder(prefix == null ? "/" : (prefix.equals("/")?prefix:prefix+"/"));
+        boolean first=true;
+        for (String s : name) {
+            if(s==null) continue;
+            if(!first){
+                sb.append("/");
+            }else
+                first=false;
+            sb.append(s);
+        }
+        return sb.toString();
+    }
+    
+    protected String makeFullPath(String prefix, ZKMBeanInfo bean) {
+        return makeFullPath(prefix, bean == null ? null : bean.getName());
+    }
+
+    /**
+     * This takes a path, such as /a/b/c, and converts it to 
+     * name0=a,name1=b,name2=c
+     */
+    private int tokenize(StringBuilder sb, String path, int index){
+        String[] tokens = path.split("/");
+        for (String s: tokens) {
+            if (s.isEmpty())
+                continue;
+            sb.append("name").append(index++)
+                    .append("=").append(s).append(",");
+        }
+        return index;
+    }
+    /**
+     * Builds an MBean path and creates an ObjectName instance using the path. 
+     * @param path MBean path
+     * @param bean the MBean instance
+     * @return ObjectName to be registered with the paltform MBean server
+     */
+    protected ObjectName makeObjectName(String path, ZKMBeanInfo bean) {
+        if(path==null)
+            return null;
+        StringBuilder beanName = new StringBuilder(CommonNames.DOMAIN + ":");
+        int counter=0;
+        counter=tokenize(beanName,path,counter);
+        tokenize(beanName,bean.getName(),counter);
+        beanName.deleteCharAt(beanName.length()-1);
+        try {
+            return new ObjectName(beanName.toString());
+        } catch (Exception e) {
+            e.printStackTrace();
+            assert false;
+        }
+        return null;
+    }
+}

+ 34 - 0
zookeeper/java/jmx/com/yahoo/zookeeper/jmx/ZKMBeanInfo.java

@@ -0,0 +1,34 @@
+/**
+ * Copyright 2008, Yahoo! Inc.
+ *
+ * Licensed 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 com.yahoo.zookeeper.jmx;
+
+/**
+ * Zookeeper MBean info interface. MBeanRegistry uses the interface to generate
+ * JMX object name.
+ */
+public interface ZKMBeanInfo {
+    /**
+     * @return a string identifying the MBean 
+     */
+    public String getName();
+    /**
+     * If isHidden returns true, the MBean won't be registered with MBean server,
+     * and thus won't be available for management tools. Used for grouping MBeans.
+     * @return true if the MBean is hidden.
+     */
+    public boolean isHidden();
+}

+ 109 - 0
zookeeper/java/jmx/com/yahoo/zookeeper/jmx/server/ConnectionBean.java

@@ -0,0 +1,109 @@
+/**
+ * Copyright 2008, Yahoo! Inc.
+ *
+ * Licensed 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 com.yahoo.zookeeper.jmx.server;
+
+import java.util.Arrays;
+import java.util.Date;
+
+import org.apache.log4j.Logger;
+
+import com.yahoo.zookeeper.jmx.MBeanRegistry;
+import com.yahoo.zookeeper.jmx.ZKMBeanInfo;
+import com.yahoo.zookeeper.server.ServerCnxn;
+import com.yahoo.zookeeper.server.ZooKeeperServer;
+
+/**
+ * Implementation of connection MBean interface.
+ */
+public class ConnectionBean implements ConnectionMXBean, ZKMBeanInfo {
+    private static final Logger LOG = Logger.getLogger(ConnectionBean.class);
+    private ServerCnxn connection;
+    private ZooKeeperServer zk;
+    private Date timeCreated;
+    
+    public ConnectionBean(ServerCnxn connection,ZooKeeperServer zk){
+        this.connection=connection;
+        this.zk=zk;
+        timeCreated=new Date();
+    }
+    
+    public String getSessionId() {
+        return Long.toHexString(connection.getSessionId());
+    }
+
+    
+    public String getSourceIP() {
+        return connection.getRemoteAddress().getAddress().getHostAddress()+
+            ":"+connection.getRemoteAddress().getPort();
+    }
+    
+    public String getName() {
+        String ip=connection.getRemoteAddress().getAddress().getHostAddress();
+        return MBeanRegistry.getInstance().makeFullPath("Connections", ip,getSessionId());
+    }
+    
+    public boolean isHidden() {
+        return false;
+    }
+    
+    public String[] getEphemeralNodes() {
+        if(zk.dataTree!=null){
+            String[] res=zk.dataTree.getEphemerals(connection.getSessionId()).toArray(new String[0]);
+            Arrays.sort(res);
+            return res;
+        }
+        return null;
+    }
+    
+    public String getStartedTime() {
+        return timeCreated.toString();
+    }
+    
+    public void terminateSession() {
+        try {
+            zk.closeSession(connection.getSessionId());
+        } catch (Exception e) {
+            LOG.warn("Unable to closeSession() for session: "+getSessionId()+
+                    ", "+e.getMessage());
+        }
+    }
+    
+    public void terminateConnection() {
+        connection.close();
+    }
+    
+    public String toString() {
+        return "ConnectionBean{ClientIP="+getSourceIP()+",SessionId="+getSessionId()+"}";
+    }
+    
+    public long getOutstandingRequests() {
+        return connection.getStats().getOutstandingRequests();
+    }
+    
+    public long getPacketsReceived() {
+        return connection.getStats().getPacketsReceived();
+    }
+    
+    public long getPacketsSent() {
+        return connection.getStats().getPacketsSent();
+    }
+    
+    public int getSessionTimeout() {
+        return connection.getSessionTimeout();
+    }
+
+}

+ 66 - 0
zookeeper/java/jmx/com/yahoo/zookeeper/jmx/server/ConnectionMXBean.java

@@ -0,0 +1,66 @@
+/**
+ * Copyright 2008, Yahoo! Inc.
+ *
+ * Licensed 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 com.yahoo.zookeeper.jmx.server;
+
+/**
+ * This MBean represents a client connection.
+ */
+public interface ConnectionMXBean {
+    /**
+     * @return source (client) IP address
+     */
+    public String getSourceIP();
+    /**
+     * @return client's session id
+     */
+    public String getSessionId();
+    /**
+     * @return time the connection was started
+     */
+    public String getStartedTime();
+    /**
+     * @return number of ephemeral nodes owned by this connection
+     */
+    public String[] getEphemeralNodes();
+    /**
+     * @return packets received from this client
+     */
+    public long getPacketsReceived();
+    /**
+     * @return number of packets sent to this client
+     */
+    public long getPacketsSent();
+    /**
+     * @return number of requets being processed
+     */
+    public long getOutstandingRequests();
+    /**
+     * @return session timeout in ms
+     */
+    public int getSessionTimeout();
+    
+    /**
+     * Terminate this client session. The client will reconnect with a different
+     * session id.
+     */
+    public void terminateSession();
+    /**
+     * Terminate thei client connection. The client will immediately attempt to 
+     * reconnect with the same session id.
+     */
+    public void terminateConnection();
+}

+ 89 - 0
zookeeper/java/jmx/com/yahoo/zookeeper/jmx/server/DataTreeBean.java

@@ -0,0 +1,89 @@
+/**
+ * Copyright 2008, Yahoo! Inc.
+ *
+ * Licensed 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 com.yahoo.zookeeper.jmx.server;
+
+import java.io.ByteArrayOutputStream;
+
+import org.apache.log4j.Logger;
+
+import com.yahoo.jute.BinaryOutputArchive;
+import com.yahoo.zookeeper.jmx.ZKMBeanInfo;
+
+/**
+ * This class implements the data tree MBean.
+ */
+public class DataTreeBean implements DataTreeMXBean, ZKMBeanInfo {
+    private static final Logger LOG = Logger.getLogger(DataTreeBean.class);
+
+    com.yahoo.zookeeper.server.DataTree dataTree;
+    
+    public DataTreeBean(){
+    }
+
+    public DataTreeBean(com.yahoo.zookeeper.server.DataTree dataTree){
+        this.dataTree=dataTree;
+    }
+    
+    public int getNodeCount() {
+        return dataTree.getNodeCount();
+    }
+
+    /* (non-Javadoc)
+     * @see com.yahoo.zookeeper.jmx.server.DataTreeMBean#getDataSize()
+     */
+    //TODO: it's useful info but can be expensive to get
+    public long getDataSize() {
+      /*  We need a more efficient way to do this
+        ByteArrayOutputStream stream=new ByteArrayOutputStream();
+        BinaryOutputArchive oa = BinaryOutputArchive.getArchive(stream);
+        try {
+            dataTree.serialize(oa, "tree");
+        } catch (Exception e) {
+            LOG.warn("Failed to get data tree size: "+e.getMessage());            
+        }
+        return stream.size();
+      */
+        return -1;
+    }
+
+    /* (non-Javadoc)
+     * @see com.yahoo.zookeeper.jmx.server.DataTreeMBean#getEphemeralCount()
+     */
+    public int getEphemeralCount() {
+        return dataTree.getSessions().size();
+    }
+
+    /* (non-Javadoc)
+     * @see com.yahoo.zookeeper.jmx.server.DataTreeMBean#getWatchCount()
+     */
+    public int getWatchCount() {
+        return dataTree.getWatchCount();
+    }
+
+    public String getName() {
+        return "InMemoryDataTree";
+    }
+
+    public boolean isHidden() {
+        return false;
+    }
+
+    public long getLastZxid() {
+        return dataTree.lastProcessedZxid;
+    }
+
+}

+ 44 - 0
zookeeper/java/jmx/com/yahoo/zookeeper/jmx/server/DataTreeMXBean.java

@@ -0,0 +1,44 @@
+/**
+ * Copyright 2008, Yahoo! Inc.
+ *
+ * Licensed 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 com.yahoo.zookeeper.jmx.server;
+
+/**
+ * Zookeeper data tree MBean.
+ */
+public interface DataTreeMXBean {
+    /**
+     * @return number of znodes in the data tree.
+     */
+    public int getNodeCount();
+    /**
+     * @return data tree size in bytes. The size includes the znode path and 
+     * its value.
+     */
+    public long getDataSize();
+    /**
+     * @return the most recent zxid processed by the data tree.
+     */
+    public long getLastZxid();
+    /**
+     * @return number of watches set.
+     */
+    public int getWatchCount();
+    /**
+     * @return number of ephemeral nodes in the data tree
+     */
+    public int getEphemeralCount();
+}

+ 95 - 0
zookeeper/java/jmx/com/yahoo/zookeeper/jmx/server/ZooKeeperServerBean.java

@@ -0,0 +1,95 @@
+/**
+ * Copyright 2008, Yahoo! Inc.
+ *
+ * Licensed 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 com.yahoo.zookeeper.jmx.server;
+
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+import java.util.Date;
+
+import com.yahoo.zookeeper.Version;
+import com.yahoo.zookeeper.jmx.ZKMBeanInfo;
+import com.yahoo.zookeeper.server.ServerConfig;
+import com.yahoo.zookeeper.server.ServerStats;
+
+/**
+ * This class implements the zookeeper server MBean interface.
+ */
+public class ZooKeeperServerBean implements ZooKeeperServerMXBean, ZKMBeanInfo {
+    private Date startTime=new Date();
+
+    public String getClientPort() {
+        try {
+            return InetAddress.getLocalHost().getHostAddress() + ":"
+                    + ServerConfig.getClientPort();
+        } catch (UnknownHostException e) {
+            return "localhost:" + ServerConfig.getClientPort();
+        }
+    }
+    
+    public String getName() {
+        return "StandaloneServer";
+    }
+    
+    public boolean isHidden() {
+        return false;
+    }
+    
+    public String getStartTime() {
+        return startTime.toString();
+    }
+    
+    public String getVersion() {
+        return Version.getFullVersion();
+    }
+    
+    public long getAvgRequestLatency() {
+        return ServerStats.getInstance().getAvgLatency();
+    }
+    
+    public long getMaxRequestLatency() {
+        return ServerStats.getInstance().getMaxLatency();
+    }
+    
+    public long getMinRequestLatency() {
+        return ServerStats.getInstance().getMinLatency();
+    }
+    
+    public long getOutstandingRequests() {
+        return ServerStats.getInstance().getOutstandingRequests();
+    }
+    
+    public long getPacketsReceived() {
+        return ServerStats.getInstance().getPacketsReceived();
+    }
+    
+    public long getPacketsSent() {
+        return ServerStats.getInstance().getPacketsSent();
+    }
+    
+    public void resetLatency() {
+        ServerStats.getInstance().resetLatency();
+    }
+    
+    public void resetMaxLatency() {
+        ServerStats.getInstance().resetMaxLatency();
+    }
+
+    public void resetStatistics() {
+        ServerStats.getInstance().resetRequestCounters();
+        ServerStats.getInstance().resetLatency();
+    }
+}

+ 72 - 0
zookeeper/java/jmx/com/yahoo/zookeeper/jmx/server/ZooKeeperServerMXBean.java

@@ -0,0 +1,72 @@
+/**
+ * Copyright 2008, Yahoo! Inc.
+ *
+ * Licensed 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 com.yahoo.zookeeper.jmx.server;
+
+/**
+ * Zookeeper server MBean.
+ */
+public interface ZooKeeperServerMXBean {
+    /**
+     * @return the server socket port number
+     */
+    public String getClientPort();
+    /**
+     * @return the zookeeper server version
+     */
+    public String getVersion();
+    /**
+     * @return time the server was started
+     */
+    public String getStartTime();
+    /**
+     * @return min request latency in ms
+     */
+    public long getMinRequestLatency();
+    /**
+     * @return average request latency in ms
+     */
+    public long getAvgRequestLatency();
+    /**
+     * @return max request latency in ms
+     */
+    public long getMaxRequestLatency();
+    /**
+     * @return number of packets received so far
+     */
+    public long getPacketsReceived();
+    /**
+     * @return number of packets sent so far
+     */
+    public long getPacketsSent();
+    /**
+     * @return number of outstanding requests.
+     */
+    public long getOutstandingRequests();
+    
+    /**
+     * Reset packet and latency statistics 
+     */
+    public void resetStatistics();
+    /**
+     * Reset min/avg/max latency statistics
+     */
+    public void resetLatency();
+    /**
+     * Reset max latency statistics only.
+     */
+    public void resetMaxLatency();
+}

+ 26 - 0
zookeeper/java/jmx/com/yahoo/zookeeper/jmx/server/quorum/FollowerBean.java

@@ -0,0 +1,26 @@
+/**
+ * Copyright 2008, Yahoo! Inc.
+ *
+ * Licensed 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 com.yahoo.zookeeper.jmx.server.quorum;
+
+/**
+ * Follower MBean inteface implementation
+ */
+public class FollowerBean extends ServerBean implements FollowerMXBean {
+    public String getName() {
+        return "Follower";
+    }
+}

+ 24 - 0
zookeeper/java/jmx/com/yahoo/zookeeper/jmx/server/quorum/FollowerMXBean.java

@@ -0,0 +1,24 @@
+/**
+ * Copyright 2008, Yahoo! Inc.
+ *
+ * Licensed 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 com.yahoo.zookeeper.jmx.server.quorum;
+
+/**
+ * Follower MBean
+ */
+public interface FollowerMXBean extends ServerMXBean {
+
+}

+ 27 - 0
zookeeper/java/jmx/com/yahoo/zookeeper/jmx/server/quorum/LeaderBean.java

@@ -0,0 +1,27 @@
+/**
+ * Copyright 2008, Yahoo! Inc.
+ *
+ * Licensed 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 com.yahoo.zookeeper.jmx.server.quorum;
+
+/**
+ * Leader MBean interface implementation.
+ */
+public class LeaderBean extends ServerBean implements LeaderMXBean {
+
+    public String getName() {
+        return "Leader";
+    }
+}

+ 41 - 0
zookeeper/java/jmx/com/yahoo/zookeeper/jmx/server/quorum/LeaderElectionBean.java

@@ -0,0 +1,41 @@
+/**
+ * Copyright 2008, Yahoo! Inc.
+ *
+ * Licensed 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 com.yahoo.zookeeper.jmx.server.quorum;
+
+import java.util.Date;
+
+import com.yahoo.zookeeper.jmx.ZKMBeanInfo;
+
+/**
+ * Leader election MBean interface implementation
+ */
+public class LeaderElectionBean implements LeaderElectionMXBean,ZKMBeanInfo {
+    private final Date startTime=new Date();
+
+    public String getName() {
+        return "LeaderElection";
+    }
+
+    public boolean isHidden() {
+        return false;
+    }
+
+    public String getStartTime() {
+        return startTime.toString();
+    }
+
+}

+ 32 - 0
zookeeper/java/jmx/com/yahoo/zookeeper/jmx/server/quorum/LeaderElectionMXBean.java

@@ -0,0 +1,32 @@
+/**
+ * Copyright 2008, Yahoo! Inc.
+ *
+ * Licensed 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 com.yahoo.zookeeper.jmx.server.quorum;
+
+/**
+ * Leader election protocol MBean. 
+ */
+public interface LeaderElectionMXBean {
+    /**
+     * @return leader election instance name
+     */
+    public String getName();
+    /**
+     * 
+     * @return the time when the leader election started
+     */
+    public String getStartTime();
+}

+ 24 - 0
zookeeper/java/jmx/com/yahoo/zookeeper/jmx/server/quorum/LeaderMXBean.java

@@ -0,0 +1,24 @@
+/**
+ * Copyright 2008, Yahoo! Inc.
+ *
+ * Licensed 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 com.yahoo.zookeeper.jmx.server.quorum;
+
+/**
+ * Leader MBean.
+ */
+public interface LeaderMXBean extends ServerMXBean {
+
+}

+ 46 - 0
zookeeper/java/jmx/com/yahoo/zookeeper/jmx/server/quorum/LocalPeerBean.java

@@ -0,0 +1,46 @@
+/**
+ * Copyright 2008, Yahoo! Inc.
+ *
+ * Licensed 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 com.yahoo.zookeeper.jmx.server.quorum;
+
+import com.yahoo.zookeeper.jmx.server.ZooKeeperServerBean;
+import com.yahoo.zookeeper.server.quorum.QuorumPeer;
+
+/**
+ * Implementation of the local peer MBean interface.
+ */
+public class LocalPeerBean extends ZooKeeperServerBean implements LocalPeerMXBean {
+
+    private QuorumPeer peer;
+    
+    public LocalPeerBean(QuorumPeer peer){
+        this.peer=peer;
+    }
+
+    public String getName() {
+        return "replica."+peer.getId();
+    }
+
+    public boolean isHidden() {
+        return false;
+    }
+
+    public String getQuorumAddress() {
+        return peer.getQuorumAddress().getHostName()+":"+
+            peer.getQuorumAddress().getPort();
+    }
+
+}

+ 32 - 0
zookeeper/java/jmx/com/yahoo/zookeeper/jmx/server/quorum/LocalPeerMXBean.java

@@ -0,0 +1,32 @@
+/**
+ * Copyright 2008, Yahoo! Inc.
+ *
+ * Licensed 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 com.yahoo.zookeeper.jmx.server.quorum;
+
+import com.yahoo.zookeeper.jmx.server.ZooKeeperServerMXBean;
+
+/**
+ * A local zookeeper server MBean interface. Unlike the remote peer, the local
+ * peer provides complete state/statistics at runtime and can be managed (just 
+ * like a standalone zookeeper server).
+ */
+public interface LocalPeerMXBean extends ZooKeeperServerMXBean{
+    /**
+     * @return the IP address used for running quorum protocols and data 
+     * replication
+     */
+    public String getQuorumAddress();
+}

+ 42 - 0
zookeeper/java/jmx/com/yahoo/zookeeper/jmx/server/quorum/QuorumBean.java

@@ -0,0 +1,42 @@
+/**
+ * Copyright 2008, Yahoo! Inc.
+ *
+ * Licensed 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 com.yahoo.zookeeper.jmx.server.quorum;
+
+import com.yahoo.zookeeper.jmx.ZKMBeanInfo;
+import com.yahoo.zookeeper.server.quorum.QuorumPeer;
+
+public class QuorumBean implements QuorumMXBean,ZKMBeanInfo {
+
+    private QuorumPeer peer;
+    
+    public QuorumBean(QuorumPeer peer){
+        this.peer=peer;
+    }
+    
+    public String getName() {
+        return "ReplicatedServer";
+    }
+    
+    public boolean isHidden() {
+        return false;
+    }
+    
+    public int getQuorumSize() {
+        return peer.getQuorumSize();
+    }
+
+}

+ 31 - 0
zookeeper/java/jmx/com/yahoo/zookeeper/jmx/server/quorum/QuorumMXBean.java

@@ -0,0 +1,31 @@
+/**
+ * Copyright 2008, Yahoo! Inc.
+ *
+ * Licensed 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 com.yahoo.zookeeper.jmx.server.quorum;
+
+/**
+ * An MBean representing a zookeeper cluster nodes (aka quorum peers)
+ */
+public interface QuorumMXBean {
+    /**
+     * @return the name of the quorum
+     */
+    public String getName();
+    /**
+     * @return configured number of peers in the quorum
+     */
+    public int getQuorumSize();
+}

+ 43 - 0
zookeeper/java/jmx/com/yahoo/zookeeper/jmx/server/quorum/RemotePeerBean.java

@@ -0,0 +1,43 @@
+/**
+ * Copyright 2008, Yahoo! Inc.
+ *
+ * Licensed 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 com.yahoo.zookeeper.jmx.server.quorum;
+
+import com.yahoo.zookeeper.jmx.ZKMBeanInfo;
+import com.yahoo.zookeeper.server.quorum.QuorumPeer;
+
+/**
+ * A remote peer bean only provides limited information about the remote peer,
+ * and the peer cannot be managed remotely. 
+ */
+public class RemotePeerBean implements RemotePeerMXBean,ZKMBeanInfo {
+    private QuorumPeer.QuorumServer peer;
+    
+    public RemotePeerBean(QuorumPeer.QuorumServer peer){
+        this.peer=peer;
+    }
+    public String getName() {
+        return "replica."+peer.id;
+    }
+    public boolean isHidden() {
+        return false;
+    }
+
+    public String getQuorumAddress() {
+        return peer.addr.getHostName()+":"+peer.addr.getPort();
+    }
+
+}

+ 31 - 0
zookeeper/java/jmx/com/yahoo/zookeeper/jmx/server/quorum/RemotePeerMXBean.java

@@ -0,0 +1,31 @@
+/**
+ * Copyright 2008, Yahoo! Inc.
+ *
+ * Licensed 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 com.yahoo.zookeeper.jmx.server.quorum;
+
+/**
+ * A proxy for a remote quorum peer.
+ */
+public interface RemotePeerMXBean {
+    /**
+     * @return name of the peer
+     */
+    public String getName();
+    /**
+     * @return IP address of the quorum peer 
+     */
+    public String getQuorumAddress();
+}

+ 36 - 0
zookeeper/java/jmx/com/yahoo/zookeeper/jmx/server/quorum/ServerBean.java

@@ -0,0 +1,36 @@
+/**
+ * Copyright 2008, Yahoo! Inc.
+ *
+ * Licensed 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 com.yahoo.zookeeper.jmx.server.quorum;
+
+import java.util.Date;
+
+import com.yahoo.zookeeper.jmx.ZKMBeanInfo;
+
+/**
+ * An abstract base class for the leader and follower MBeans.
+ */
+public abstract class ServerBean implements ServerMXBean, ZKMBeanInfo {
+    private final Date startTime=new Date();
+    
+    public boolean isHidden() {
+        return false;
+    }
+
+    public String getStartTime() {
+        return startTime.toString();
+    }
+}

+ 31 - 0
zookeeper/java/jmx/com/yahoo/zookeeper/jmx/server/quorum/ServerMXBean.java

@@ -0,0 +1,31 @@
+/**
+ * Copyright 2008, Yahoo! Inc.
+ *
+ * Licensed 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 com.yahoo.zookeeper.jmx.server.quorum;
+
+/**
+ * A quorum server MBean.
+ */
+public interface ServerMXBean {
+    /**
+     * @return name of the server MBean
+     */
+    public String getName();
+    /**
+     * @return the start time the server 
+     */
+    public String getStartTime();
+}

+ 138 - 0
zookeeper/java/jmx/com/yahoo/zookeeper/server/ManagedZooKeeperServer.java

@@ -0,0 +1,138 @@
+/**
+ * Copyright 2008, Yahoo! Inc.
+ *
+ * Licensed 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 com.yahoo.zookeeper.server;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.concurrent.ConcurrentHashMap;
+
+import org.apache.log4j.Logger;
+
+import com.yahoo.zookeeper.jmx.MBeanRegistry;
+import com.yahoo.zookeeper.jmx.server.ConnectionBean;
+import com.yahoo.zookeeper.jmx.server.DataTreeBean;
+import com.yahoo.zookeeper.jmx.server.ZooKeeperServerBean;
+import com.yahoo.zookeeper.server.util.ConnectionObserver;
+import com.yahoo.zookeeper.server.util.ObserverManager;
+import com.yahoo.zookeeper.server.util.ServerObserver;
+import com.yahoo.zookeeper.server.util.ZooKeeperObserverManager;
+
+import static com.yahoo.zookeeper.server.ServerConfig.getClientPort;
+
+/**
+ * This class launches a standalone zookeeper server with JMX support
+ * enabled. The users can connect to the server JVM and manage the server state 
+ * (such as currently open client connections) and view runtime statistics using 
+ * one of existing GUI JMX consoles (jconsole, for example). Please refer to 
+ * the JDK vendor documentation for further information on how to enable JMX 
+ * support in the JVM.
+ * <p>
+ * The server provides following MBeans:
+ * <ul>
+ *  <li>Zookeeper server MBean -- provides various configuraton data and runtime 
+ *  statistics, see {@link ZooKeeperServerMXBean}
+ *  <li>Data tree MBean -- provides runtime data tree statistics, see 
+ *  {@link DataTreeMXBean}
+ *  <li>Client connection MBean -- provides runtime statistics as well as 
+ *  connection management operations, see {@link ConnectionMXBean}
+ * </ul>
+ * The client connection is a dynamic resource and therefore the connection
+ * MBeans are dynamically created and destroyed as the clients connect to and 
+ * disconnect from the server.
+ */
+public class ManagedZooKeeperServer extends ObservableZooKeeperServer {
+    private static final Logger LOG = Logger.getLogger(ManagedZooKeeperServer.class);
+
+    private ZooKeeperServerBean svrBean;
+
+    private class ManagedServerObserver implements ServerObserver {
+        private DataTreeBean dataTreeBean;
+
+        public void onShutdown(ZooKeeperServer server) {
+            MBeanRegistry.getInstance().unregister(dataTreeBean);
+            MBeanRegistry.getInstance().unregister(svrBean);
+        }
+
+        public void onStartup(ZooKeeperServer server) {
+            try {
+                svrBean = new ZooKeeperServerBean();
+                MBeanRegistry.getInstance().register(svrBean, null);
+                dataTreeBean = new DataTreeBean(server.dataTree);
+                MBeanRegistry.getInstance().register(dataTreeBean, svrBean);
+            } catch (Exception e) {
+                LOG.warn("Failed to register Standalone ZooKeeperServerMBean "
+                                + e.getMessage());
+            }
+        }
+    }
+
+    private class ManagedConnectionObserver implements ConnectionObserver {
+        private ConcurrentHashMap<ServerCnxn,ConnectionBean> map=
+            new ConcurrentHashMap<ServerCnxn,ConnectionBean>();
+
+        public void onClose(ServerCnxn sc) {
+            ConnectionBean b=map.remove(sc);
+            if(b!=null){
+                ZooTrace.logTraceMessage(LOG, ZooTrace.JMX_TRACE_MASK,
+                        "Un-registering a ConnectionBean: "+b);
+                MBeanRegistry.getInstance().unregister(b);
+            }
+        }
+
+        public void onNew(ServerCnxn sc) {
+            ConnectionBean b=new ConnectionBean(sc,ManagedZooKeeperServer.this);
+            map.put(sc, b);
+            ZooTrace.logTraceMessage(LOG, ZooTrace.JMX_TRACE_MASK,
+                    "Registering new ConnectionBean: "+b);
+            MBeanRegistry.getInstance().register(b, svrBean);            
+        }
+    }
+
+    public ManagedZooKeeperServer(File dataDir, File dataLogDir, 
+            int tickTime,DataTreeBuilder treeBuilder) throws IOException {
+        super(dataDir, dataLogDir, tickTime,treeBuilder);
+        ObserverManager.getInstance().add(new ManagedServerObserver());
+        ObserverManager.getInstance().add(new ManagedConnectionObserver());
+    }
+
+    public ManagedZooKeeperServer(DataTreeBuilder treeBuilder) throws IOException {
+        super(treeBuilder);
+        ObserverManager.getInstance().add(new ManagedServerObserver());
+        ObserverManager.getInstance().add(new ManagedConnectionObserver());
+    }
+
+    /**
+     * To start the server specify the client port number and the data directory
+     * on the command line.
+     * @see ServerConfig#parse(String[])
+     * @param args command line parameters.
+     */
+    public static void main(String[] args) {
+        ServerConfig.parse(args);
+        ZooKeeperObserverManager.setAsConcrete();
+        runStandalone(new Factory() {
+            public NIOServerCnxn.Factory createConnectionFactory()throws IOException {
+                return new ObservableNIOServerCnxn.Factory(getClientPort());
+            }
+            public ZooKeeperServer createServer() throws IOException {
+                // TODO: we may want to build an observable/managed data tree here instead
+                return new ManagedZooKeeperServer(new BasicDataTreeBuilder());
+            }
+        });
+    }
+
+}

+ 111 - 0
zookeeper/java/jmx/com/yahoo/zookeeper/server/ObservableDataTree.java

@@ -0,0 +1,111 @@
+/**
+ * Copyright 2008, Yahoo! Inc.
+ *
+ * Licensed 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 com.yahoo.zookeeper.server;
+
+import java.util.ArrayList;
+
+import com.yahoo.zookeeper.KeeperException;
+import com.yahoo.zookeeper.data.ACL;
+import com.yahoo.zookeeper.data.Stat;
+import com.yahoo.zookeeper.server.util.DataTreeObserver;
+import com.yahoo.zookeeper.server.util.EventInfo;
+import com.yahoo.zookeeper.server.util.ObservableComponent;
+import com.yahoo.zookeeper.server.util.ObserverManager;
+
+/**
+ * An observable data tree notifies its observers about any data modification
+ * operation, including znode creation, znode update operation, znode deletion 
+ * and znode's ACL update.
+ * 
+ * In order to be able to receive connection notification, applications
+ * are required to implement {@link ObservableDataTree} interface and register
+ * its instance with {@link ObserverManager}.
+ */
+public class ObservableDataTree extends DataTree implements ObservableComponent {
+    private enum Event {
+        ADD() {
+            public void dispatch(DataTree dt,DataNode node,DataTreeObserver o) {
+                o.onAdd(dt,node);
+            }
+        },
+        DELETE() {
+            public void dispatch(DataTree dt,DataNode node, DataTreeObserver o) {
+                o.onDelete(dt,node);
+            }
+        },
+        UPDATE() {
+            public void dispatch(DataTree dt,DataNode node, DataTreeObserver o) {
+                o.onUpdate(dt,node);
+            }
+        },
+        UPDATE_ACL() {
+            public void dispatch(DataTree dt,DataNode node, DataTreeObserver o) {
+                o.onUpdateACL(dt,node);
+            }
+        };
+        public abstract void dispatch(DataTree dt,DataNode node, DataTreeObserver o);
+    }
+
+    private class TreeEventInfo implements EventInfo{
+        private Event ev;
+        private DataNode node;
+        TreeEventInfo(Event ev, DataNode node){
+            this.ev=ev;
+            this.node=node;
+        }
+        public void dispatch(Object source, Object observer){
+            ev.dispatch((ObservableDataTree)source, node, (DataTreeObserver)observer);
+        }
+    }
+
+    public void dispatchEvent(Object observer, Object args) {
+        ((EventInfo)args).dispatch(this,observer);
+    }
+
+
+    public String createNode(String path, byte[] data, ArrayList<ACL> acl,
+            long ephemeralOwner, long zxid, long time) throws KeeperException {
+        String result=super.createNode(path, data, acl, ephemeralOwner, zxid, time);
+        ObserverManager.getInstance().notifyObservers(this,
+                new TreeEventInfo(Event.ADD,getNode(result)));
+        return result;
+    }
+
+    public void deleteNode(String path) throws KeeperException.NoNodeException {
+        DataNode deleted=getNode(path);
+        super.deleteNode(path);
+        ObserverManager.getInstance().notifyObservers(this,
+                new TreeEventInfo(Event.DELETE,deleted));
+    }
+
+    public Stat setACL(String path, ArrayList<ACL> acl, int version)
+            throws KeeperException {
+        Stat stat=super.setACL(path, acl, version);
+        ObserverManager.getInstance().notifyObservers(this,
+                new TreeEventInfo(Event.UPDATE_ACL,getNode(path)));
+        return stat;
+    }
+
+
+    public Stat setData(String path, byte[] data, int version, long zxid,
+            long time) throws KeeperException.NoNodeException {
+        Stat stat=super.setData(path, data, version, zxid, time);
+        ObserverManager.getInstance().notifyObservers(this,
+                new TreeEventInfo(Event.UPDATE,getNode(path)));
+        return stat;
+    }    
+}

+ 83 - 0
zookeeper/java/jmx/com/yahoo/zookeeper/server/ObservableNIOServerCnxn.java

@@ -0,0 +1,83 @@
+/**
+ * Copyright 2008, Yahoo! Inc.
+ *
+ * Licensed 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 com.yahoo.zookeeper.server;
+
+import java.io.IOException;
+import java.nio.channels.SelectionKey;
+import java.nio.channels.SocketChannel;
+
+import com.yahoo.zookeeper.server.util.ConnectionObserver;
+import com.yahoo.zookeeper.server.util.ObservableComponent;
+import com.yahoo.zookeeper.server.util.ObserverManager;
+
+/**
+ * This class implements an observable server connection. It supports two
+ * types of connection events: a new connection event and a connection closed 
+ * event.
+ * 
+ * In order to be able to receive connection notification, applications
+ * are required to implement {@link ConnectionObserver} interface and register
+ * its instance with {@link ObserverManager}.
+ */
+public class ObservableNIOServerCnxn extends NIOServerCnxn implements ObservableComponent {
+    private enum ConnectionEvent {
+        NEW(){
+            public void dispatch(ServerCnxn c, ConnectionObserver o){
+                o.onNew(c);
+            }
+        },
+        CLOSE(){
+            public void dispatch(ServerCnxn c, ConnectionObserver o){
+                o.onClose(c);
+            }            
+        };
+        public abstract void dispatch(ServerCnxn c, ConnectionObserver o);
+    }
+
+    static public class Factory extends NIOServerCnxn.Factory{
+        @Override
+        protected NIOServerCnxn createConnection(SocketChannel sock,
+                SelectionKey sk) throws IOException {
+            return new ObservableNIOServerCnxn(zks,sock,sk,this);
+        }
+
+        public Factory(int port) throws IOException {
+            super(port);
+        }
+    }
+    
+    public ObservableNIOServerCnxn(ZooKeeperServer zk, SocketChannel sock,
+            SelectionKey sk, Factory factory) throws IOException {
+        super(zk, sock, sk, factory);
+    }
+
+    public void close() {
+        ObserverManager.getInstance().notifyObservers(this,ConnectionEvent.CLOSE);
+        super.close();
+    }
+
+    public void dispatchEvent(Object observer, Object args) {
+        ConnectionEvent ev=(ConnectionEvent)args;
+        ev.dispatch(this, (ConnectionObserver)observer);
+    }
+
+    public void finishSessionInit(boolean valid) {
+        super.finishSessionInit(valid);
+        if(valid && !closed)
+            ObserverManager.getInstance().notifyObservers(this,ConnectionEvent.NEW);
+    }
+}

+ 50 - 0
zookeeper/java/jmx/com/yahoo/zookeeper/server/ObservableZooKeeperServer.java

@@ -0,0 +1,50 @@
+/**
+ * Copyright 2008, Yahoo! Inc.
+ *
+ * Licensed 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 com.yahoo.zookeeper.server;
+
+import java.io.File;
+import java.io.IOException;
+/**
+ * The observable server broadcast notifications when its state changes. 
+ * 
+ * The code interested in receiving the notification must implement 
+ * the {@link ServerObserver} interface and register the instance with 
+ * {@link ObserverManager}.
+ */
+public class ObservableZooKeeperServer extends ZooKeeperServer{
+
+    private ZooKeeperObserverNotifier notifier=new ZooKeeperObserverNotifier(this);
+    
+    public ObservableZooKeeperServer(File dataDir, File dataLogDir, 
+            int tickTime,DataTreeBuilder treeBuilder) throws IOException {
+        super(dataDir, dataLogDir, tickTime,treeBuilder);
+    }
+
+    public ObservableZooKeeperServer(DataTreeBuilder treeBuilder) throws IOException {
+        super(treeBuilder);
+    }
+
+    public void shutdown() {
+        notifier.notifyShutdown();
+        super.shutdown();
+    }
+
+    public void startup() throws IOException, InterruptedException {
+        super.startup();
+        notifier.notifyStarted();
+    }
+}

+ 74 - 0
zookeeper/java/jmx/com/yahoo/zookeeper/server/ZooKeeperObserverNotifier.java

@@ -0,0 +1,74 @@
+/**
+ * Copyright 2008, Yahoo! Inc.
+ *
+ * Licensed 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 com.yahoo.zookeeper.server;
+
+import com.yahoo.zookeeper.server.util.ObservableComponent;
+import com.yahoo.zookeeper.server.util.ObserverManager;
+import com.yahoo.zookeeper.server.util.ServerObserver;
+
+/**
+ * The class is responsible for triggering and dispatching of server life cycle events.
+ * <p>
+ * In order to make itself observable, the server class creates an instance of 
+ * ZooKeeperObserverNotifier and then, calls the notifyStarted or notifyShutdown 
+ * methods to notify all registered observers of a startup or shutdown event 
+ * correspondingly.
+ * 
+ * Application wishing to be notified of a server state change should implement
+ * {@link ServerObserver} interface and register an instance of the interface
+ * with {@link ObserverManager}.
+ */
+public class ZooKeeperObserverNotifier implements ObservableComponent {
+    private enum Event {
+        STARTUP() {
+            public void dispatch(ZooKeeperServer s, ServerObserver o) {
+                o.onStartup(s);
+            }
+        },
+        SHUTDOWN() {
+            public void dispatch(ZooKeeperServer s, ServerObserver o) {
+                o.onShutdown(s);
+            }
+        };
+        public abstract void dispatch(ZooKeeperServer s, ServerObserver o);
+    }
+
+    private ZooKeeperServer server;
+    
+    public ZooKeeperObserverNotifier(ZooKeeperServer server) {
+        this.server=server;
+    }
+
+    /**
+     * Generate a startup event.
+     */
+    public void notifyStarted(){
+        ObserverManager.getInstance().notifyObservers(this,Event.STARTUP);        
+    }
+    /**
+     * Generate a shutdown event.
+     */
+    public void notifyShutdown(){
+        ObserverManager.getInstance().notifyObservers(this,Event.SHUTDOWN);        
+    }
+    
+    public void dispatchEvent(Object observer, Object args) {
+        Event ev=(Event)args;
+        ev.dispatch(server,(ServerObserver)observer);
+    }
+
+}

+ 243 - 0
zookeeper/java/jmx/com/yahoo/zookeeper/server/quorum/ManagedQuorumPeer.java

@@ -0,0 +1,243 @@
+/**
+ * Copyright 2008, Yahoo! Inc.
+ *
+ * Licensed 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 com.yahoo.zookeeper.server.quorum;
+
+import static com.yahoo.zookeeper.server.ServerConfig.getClientPort;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.concurrent.ConcurrentHashMap;
+
+import org.apache.log4j.Logger;
+
+import com.yahoo.zookeeper.jmx.MBeanRegistry;
+import com.yahoo.zookeeper.jmx.ZKMBeanInfo;
+import com.yahoo.zookeeper.jmx.server.ConnectionBean;
+import com.yahoo.zookeeper.jmx.server.DataTreeBean;
+import com.yahoo.zookeeper.jmx.server.quorum.FollowerBean;
+import com.yahoo.zookeeper.jmx.server.quorum.LeaderBean;
+import com.yahoo.zookeeper.jmx.server.quorum.LeaderElectionBean;
+import com.yahoo.zookeeper.jmx.server.quorum.LocalPeerBean;
+import com.yahoo.zookeeper.jmx.server.quorum.QuorumBean;
+import com.yahoo.zookeeper.jmx.server.quorum.RemotePeerBean;
+import com.yahoo.zookeeper.jmx.server.quorum.ServerBean;
+import com.yahoo.zookeeper.server.ManagedZooKeeperServer;
+import com.yahoo.zookeeper.server.NIOServerCnxn;
+import com.yahoo.zookeeper.server.ObservableNIOServerCnxn;
+import com.yahoo.zookeeper.server.ServerCnxn;
+import com.yahoo.zookeeper.server.ZooKeeperServer;
+import com.yahoo.zookeeper.server.ZooTrace;
+import com.yahoo.zookeeper.server.util.ConnectionObserver;
+import com.yahoo.zookeeper.server.util.ObserverManager;
+import com.yahoo.zookeeper.server.util.QuorumPeerObserver;
+import com.yahoo.zookeeper.server.util.ServerObserver;
+import com.yahoo.zookeeper.server.util.ZooKeeperObserverManager;
+
+/**
+ * This class launches a replicated zookeeper server with JMX support
+ * enabled. The users can connect to the server JVM and manage 
+ * the server state (such as currently open client connections) and view runtime
+ * statistics using one of existing GUI JMX consoles (jconsole, for example).
+ * Please refer to the JDK vendor documentation for further information on how
+ * to enable JMX support in the JVM.
+ * <p>
+ * The server provides following MBeans:
+ * <ul>
+ *  <li>Quorum MBean -- provides quorum runtime statistics, see {@link QuorumMXBean}.
+ *  <li>Peer MBean -- provides information about quorum peers (local and remote),
+ *  see {@link LocalPeerMXBean} and {@link RemotePeerMXBean}.
+ *  <li>Leader election MBean -- provides runtime info on leader election protocol,
+ *  see {@link LeaderElectionMXBean}
+ *  <li>Zookeeper server MBean -- provides various configuraton data and runtime 
+ *  statistics, see {@link ZooKeeperServerMXBean}
+ *  <li>Data tree MBean -- provides runtime data tree statistics, see 
+ *  {@link DataTreeMXBean}
+ *  <li>Client connection MBean -- provides runtime statistics as well as 
+ *  connection management operations, see {@link ConnectionMXBean}
+ * </ul>
+ * The client connection is a dynamic resource and therefore the connection
+ * MBeans are dynamically created and destroyed as the clients connect to and 
+ * disconnect from the server.
+ */
+public class ManagedQuorumPeer extends ObservableQuorumPeer {
+    private static final Logger LOG = Logger.getLogger(ManagedQuorumPeer.class);
+
+    private QuorumBean quorumBean;
+    private LocalPeerBean localPeerBean;
+    private ServerBean svrBean;
+    private LeaderElectionBean leBean;
+    
+    // tracking state of the quorum peer
+    private class ManagedQuorumPeerObserver implements QuorumPeerObserver {
+        public void onFollowerShutdown(QuorumPeer qp, Follower follower) {
+            ZooTrace.logTraceMessage(LOG, ZooTrace.JMX_TRACE_MASK,
+                    "Follower shutdown "+follower);
+            MBeanRegistry.getInstance().unregister(svrBean);
+            svrBean=null;
+        }
+
+        public void onFollowerStarted(QuorumPeer qp, Follower newFollower) {
+            ZooTrace.logTraceMessage(LOG, ZooTrace.JMX_TRACE_MASK,
+                    "Follower started "+newFollower);
+            MBeanRegistry.getInstance().unregister(leBean);
+            leBean=null;
+            svrBean=new FollowerBean();
+            MBeanRegistry.getInstance().register(svrBean, localPeerBean);
+        }
+
+        public void onLeaderElectionStarted(QuorumPeer qp) {
+            ZooTrace.logTraceMessage(LOG, ZooTrace.JMX_TRACE_MASK,
+                    "Running leader election protocol...");
+            leBean=new LeaderElectionBean();
+            MBeanRegistry.getInstance().register(leBean, localPeerBean);            
+        }
+
+        public void onLeaderShutdown(QuorumPeer qp, Leader leader) {
+            ZooTrace.logTraceMessage(LOG, ZooTrace.JMX_TRACE_MASK,
+                    "Leader shutdown "+leader);
+            MBeanRegistry.getInstance().unregister(svrBean);
+            svrBean=null;
+        }
+
+        public void onLeaderStarted(QuorumPeer qp, Leader newLeader) {
+            ZooTrace.logTraceMessage(LOG, ZooTrace.JMX_TRACE_MASK,
+                    "Leader started "+newLeader);
+            MBeanRegistry.getInstance().unregister(leBean);
+            leBean=null;
+            svrBean=new LeaderBean();
+            MBeanRegistry.getInstance().register(svrBean, localPeerBean);
+        }
+
+        public void onShutdown(QuorumPeer qp) {
+            ZooTrace.logTraceMessage(LOG, ZooTrace.JMX_TRACE_MASK,
+                    "Shutting down quorum peer");
+            MBeanRegistry.getInstance().unregisterAll();
+        }
+
+        public void onStartup(QuorumPeer qp) {
+            ZooTrace.logTraceMessage(LOG, ZooTrace.JMX_TRACE_MASK,
+                    "Starting quorum peer");
+            quorumBean=new QuorumBean(qp);
+            MBeanRegistry.getInstance().register(quorumBean, null);
+            for(QuorumServer s: qp.quorumPeers){
+                ZKMBeanInfo p;
+                if(qp.getId()==s.id)
+                    p=localPeerBean=new LocalPeerBean(qp);
+                else
+                    p=new RemotePeerBean(s);
+                MBeanRegistry.getInstance().register(p, quorumBean);
+            }
+        }
+    }
+
+    // on client connect/disconnect this observer will register/unregister 
+    // a connection MBean with the MBean server
+    private class ManagedConnectionObserver implements ConnectionObserver {
+        private ConcurrentHashMap<ServerCnxn,ConnectionBean> map=
+            new ConcurrentHashMap<ServerCnxn,ConnectionBean>();
+
+        public void onClose(ServerCnxn sc) {
+            ConnectionBean b=map.remove(sc);
+            if(b!=null){
+                ZooTrace.logTraceMessage(LOG, ZooTrace.JMX_TRACE_MASK,
+                        "Un-registering a ConnectionBean: "+b);
+                MBeanRegistry.getInstance().unregister(b);
+            }
+        }
+
+        public void onNew(ServerCnxn sc) {
+            ConnectionBean b=new ConnectionBean(sc,getActiveServer());
+            map.put(sc, b);
+            ZooTrace.logTraceMessage(LOG, ZooTrace.JMX_TRACE_MASK,
+                    "Registering new ConnectionBean: "+b);
+            MBeanRegistry.getInstance().register(b, localPeerBean);            
+        }
+    }
+
+    // this observer tracks the state of the zookeeper server 
+    private class ManagedServerObserver implements ServerObserver {
+        private DataTreeBean dataTreeBean;
+
+        public void onShutdown(ZooKeeperServer server) {
+            ZooTrace.logTraceMessage(LOG, ZooTrace.JMX_TRACE_MASK,
+                    "Shutdown zookeeper server: "+server);
+            MBeanRegistry.getInstance().unregister(dataTreeBean);
+            dataTreeBean=null;
+        }
+
+        public void onStartup(ZooKeeperServer server) {
+            ZooTrace.logTraceMessage(LOG, ZooTrace.JMX_TRACE_MASK,
+                    "Started new zookeeper server: "+server);
+            try {
+                dataTreeBean = new DataTreeBean(server.dataTree);
+                MBeanRegistry.getInstance().register(dataTreeBean, svrBean);
+            } catch (Exception e) {
+                LOG.warn("Failed to register Standalone ZooKeeperServerMBean "
+                                + e.getMessage());
+            }
+        }
+    }
+
+    private void setupObservers(){
+        ObserverManager.getInstance().add(new ManagedQuorumPeerObserver());
+        ObserverManager.getInstance().add(new ManagedServerObserver());        
+        ObserverManager.getInstance().add(new ManagedConnectionObserver());        
+    }
+    
+    public ManagedQuorumPeer(ArrayList<QuorumServer> quorumPeers, File dataDir,
+            File dataLogDir,int electionAlg, int electionPort,long myid,    int tickTime, 
+            int initLimit, int syncLimit,NIOServerCnxn.Factory cnxnFactory) 
+                throws IOException {
+        super(quorumPeers, dataDir, dataLogDir,electionAlg, electionPort,myid,
+                tickTime, initLimit, syncLimit,cnxnFactory);
+        setupObservers();
+    }
+
+    public ManagedQuorumPeer(NIOServerCnxn.Factory cnxnFactory) throws IOException {
+        super(cnxnFactory);
+        setupObservers();
+    }
+
+    /**
+     * To start the replicated server specify the configuration file name on the
+     * command line.
+     * @param args command line
+     */
+    public static void main(String[] args) {
+        if (args.length == 2) {
+            ManagedZooKeeperServer.main(args);
+            return;
+        }
+        QuorumPeerConfig.parse(args);
+        if (!QuorumPeerConfig.isStandalone()) {
+            ZooKeeperObserverManager.setAsConcrete();
+            runPeer(new QuorumPeer.Factory() {
+                public QuorumPeer create(NIOServerCnxn.Factory cnxnFactory)
+                        throws IOException {
+                    return new ManagedQuorumPeer(cnxnFactory);
+                }
+                public NIOServerCnxn.Factory createConnectionFactory() throws IOException {
+                    return new ObservableNIOServerCnxn.Factory(getClientPort());
+                }
+            });
+        }else{
+            // there is only server in the quorum -- run as standalone
+            ManagedZooKeeperServer.main(args);
+        }
+    }
+}

+ 76 - 0
zookeeper/java/jmx/com/yahoo/zookeeper/server/quorum/ObservableFollower.java

@@ -0,0 +1,76 @@
+/**
+ * Copyright 2008, Yahoo! Inc.
+ *
+ * Licensed 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 com.yahoo.zookeeper.server.quorum;
+
+import com.yahoo.zookeeper.server.util.EventInfo;
+import com.yahoo.zookeeper.server.util.ObserverManager;
+import com.yahoo.zookeeper.server.util.QuorumPeerObserver;
+
+/**
+ * This observable follower implementation notifies its registered observers
+ * of its important life cycle events: startup and shutdown.
+ * <p>
+ * In order to be able to receive leader notifications, application must 
+ * implement {@link QuorumPeerObserver} and register an instance of the interface
+ * with {@link ObserverManager}.
+ */
+public class ObservableFollower extends Follower {
+
+    private enum Event{
+        STARTUP() {
+            public void dispatch(ObservableQuorumPeer peer,
+                    QuorumPeerObserver ob,Follower follower) {
+                ob.onFollowerStarted(peer,follower);
+            }
+        },
+        SHUTDOWN() {
+            public void dispatch(ObservableQuorumPeer peer,
+                    QuorumPeerObserver ob,Follower follower) {
+                ob.onFollowerShutdown(peer,follower);
+            }
+        };
+        public abstract void dispatch(ObservableQuorumPeer peer,
+                QuorumPeerObserver ob,Follower follower);
+    }
+
+    public ObservableFollower(QuorumPeer self, FollowerZooKeeperServer zk) {
+        super(self, zk);
+    }
+
+    static private class PeerEvent implements EventInfo{
+        private Event ev;
+        private Follower info;
+        PeerEvent(Event ev,Follower info){
+            this.ev=ev;
+            this.info=info;
+        }
+        public void dispatch(Object source, Object ob) {
+            ev.dispatch((ObservableQuorumPeer)source,(QuorumPeerObserver)ob,info);
+        }
+    }
+    void followLeader() throws InterruptedException {
+        try{
+            ObserverManager.getInstance().notifyObservers((ObservableQuorumPeer)self, 
+                    new PeerEvent(Event.STARTUP,this));
+            super.followLeader();
+        }finally{
+            ObserverManager.getInstance().notifyObservers((ObservableQuorumPeer)self, 
+                    new PeerEvent(Event.SHUTDOWN,this));            
+        }
+    }
+
+}

+ 53 - 0
zookeeper/java/jmx/com/yahoo/zookeeper/server/quorum/ObservableFollowerZooKeeperServer.java

@@ -0,0 +1,53 @@
+/**
+ * Copyright 2008, Yahoo! Inc.
+ *
+ * Licensed 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 com.yahoo.zookeeper.server.quorum;
+
+import java.io.File;
+import java.io.IOException;
+
+import com.yahoo.zookeeper.server.ZooKeeperObserverNotifier;
+import com.yahoo.zookeeper.server.util.ObserverManager;
+import com.yahoo.zookeeper.server.util.ServerObserver;
+
+/**
+ * This observable follower server class notifies the registered observers
+ * whenever the server instance is started or stopped.
+ * <p>
+ * Application wishing to be notified of a server state change should implement
+ * {@link ServerObserver} interface and register an instance of the interface
+ * with {@link ObserverManager}.
+ */
+public class ObservableFollowerZooKeeperServer extends FollowerZooKeeperServer {
+
+    private ZooKeeperObserverNotifier notifier;
+
+    public ObservableFollowerZooKeeperServer(File dataDir, File dataLogDir,
+            QuorumPeer self, DataTreeBuilder treeBuilder) throws IOException {
+        super(dataDir, dataLogDir, self, treeBuilder);
+        notifier=new ZooKeeperObserverNotifier(this);
+    }
+    
+    public void shutdown() {
+        notifier.notifyShutdown();
+        super.shutdown();
+    }
+
+    public void startup() throws IOException, InterruptedException {
+        super.startup();
+        notifier.notifyStarted();
+    }
+}

+ 80 - 0
zookeeper/java/jmx/com/yahoo/zookeeper/server/quorum/ObservableLeader.java

@@ -0,0 +1,80 @@
+/**
+ * Copyright 2008, Yahoo! Inc.
+ *
+ * Licensed 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 com.yahoo.zookeeper.server.quorum;
+
+import java.io.IOException;
+
+import com.yahoo.zookeeper.server.util.EventInfo;
+import com.yahoo.zookeeper.server.util.ObserverManager;
+import com.yahoo.zookeeper.server.util.QuorumPeerObserver;
+
+/**
+ * This observable leader implementation notifies its registered observers
+ * of its important life cycle events: startup and shutdown.
+ * <p>
+ * In order to be able to receive leader notifications, application must 
+ * implement {@link QuorumPeerObserver} and register an instance of the interface
+ * with {@link ObserverManager}.
+ */
+public class ObservableLeader extends Leader{
+
+    private enum Event{
+        STARTUP() {
+            public void dispatch(ObservableQuorumPeer peer,
+                    QuorumPeerObserver ob,Leader leader) {
+                ob.onLeaderStarted(peer,leader);
+            }
+        },
+        SHUTDOWN() {
+            public void dispatch(ObservableQuorumPeer peer,
+                    QuorumPeerObserver ob,Leader leader) {
+                ob.onLeaderShutdown(peer,leader);
+            }
+        };
+        public abstract void dispatch(ObservableQuorumPeer peer,
+                QuorumPeerObserver ob,Leader leader);
+    }
+
+    public ObservableLeader(QuorumPeer self, LeaderZooKeeperServer zk)
+            throws IOException {
+        super(self, zk);
+    }
+
+    static private class PeerEvent implements EventInfo{
+        private Event ev;
+        private Leader info;
+        PeerEvent(Event ev,Leader info){
+            this.ev=ev;
+            this.info=info;
+        }
+        public void dispatch(Object source, Object ob) {
+            ev.dispatch((ObservableQuorumPeer)source,(QuorumPeerObserver)ob,info);
+        }
+    }
+    // QuorumPeer calls this method as soon as the peer has been elected a leader
+    void lead() throws IOException, InterruptedException {
+        try{
+            ObserverManager.getInstance().notifyObservers((ObservableQuorumPeer)self, 
+                    new PeerEvent(Event.STARTUP,this));
+            super.lead();
+        }finally{
+            ObserverManager.getInstance().notifyObservers((ObservableQuorumPeer)self, 
+                    new PeerEvent(Event.SHUTDOWN,this));            
+        }
+    }
+
+}

+ 53 - 0
zookeeper/java/jmx/com/yahoo/zookeeper/server/quorum/ObservableLeaderZooKeeperServer.java

@@ -0,0 +1,53 @@
+/**
+ * Copyright 2008, Yahoo! Inc.
+ *
+ * Licensed 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 com.yahoo.zookeeper.server.quorum;
+
+import java.io.File;
+import java.io.IOException;
+
+import com.yahoo.zookeeper.server.ZooKeeperObserverNotifier;
+import com.yahoo.zookeeper.server.util.ObserverManager;
+import com.yahoo.zookeeper.server.util.ServerObserver;
+
+/**
+ * This observable leader server class notifies the registered observers
+ * whenever the server is started or stopped.
+ * <p>
+ * Application wishing to be notified of a server state change should implement
+ * {@link ServerObserver} interface and register an instance of the interface
+ * with {@link ObserverManager}.
+ */
+public class ObservableLeaderZooKeeperServer extends LeaderZooKeeperServer {
+
+    private ZooKeeperObserverNotifier notifier;
+
+    public ObservableLeaderZooKeeperServer(File dataDir, File dataLogDir,
+            QuorumPeer self, DataTreeBuilder treeBuilder) throws IOException {
+        super(dataDir, dataLogDir, self, treeBuilder);
+        notifier=new ZooKeeperObserverNotifier(this);
+    }
+
+    public void shutdown() {
+        notifier.notifyShutdown();
+        super.shutdown();
+    }
+
+    public void startup() throws IOException, InterruptedException {
+        super.startup();
+        notifier.notifyStarted();
+    }
+}

+ 107 - 0
zookeeper/java/jmx/com/yahoo/zookeeper/server/quorum/ObservableQuorumPeer.java

@@ -0,0 +1,107 @@
+/**
+ * Copyright 2008, Yahoo! Inc.
+ *
+ * Licensed 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 com.yahoo.zookeeper.server.quorum;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+
+import com.yahoo.zookeeper.server.NIOServerCnxn;
+import com.yahoo.zookeeper.server.ZooKeeperServer;
+import com.yahoo.zookeeper.server.util.EventInfo;
+import com.yahoo.zookeeper.server.util.ObservableComponent;
+import com.yahoo.zookeeper.server.util.ObserverManager;
+import com.yahoo.zookeeper.server.util.QuorumPeerObserver;
+
+/**
+ * The observable quorum peer sends notifications to all registered observers
+ * when its state changes. Events of interest include peer startup, shutdown and
+ * the initiation of a leader election protocol.
+ * <p>
+ * In order to be able to receive QuorumPeer notifications, application must 
+ * implement {@link QuorumPeerObserver} and register an instance of the interface
+ * with {@link ObserverManager}.
+ */
+public class ObservableQuorumPeer extends QuorumPeer implements ObservableComponent{
+
+    private enum Event{
+        STARTUP(){
+            public void dispatch(ObservableQuorumPeer peer,QuorumPeerObserver ob){
+                ob.onStartup(peer);
+            }
+        },
+        SHUTDOWN(){
+            public void dispatch(ObservableQuorumPeer peer,QuorumPeerObserver ob){
+                ob.onShutdown(peer);                
+            }            
+        },
+        LEADER_ELECTION(){
+            public void dispatch(ObservableQuorumPeer peer,QuorumPeerObserver ob){
+                ob.onLeaderElectionStarted(peer);                
+            }
+        };
+        public abstract void dispatch(ObservableQuorumPeer peer,QuorumPeerObserver ob);
+    }
+    
+    public ObservableQuorumPeer(ArrayList<QuorumServer> quorumPeers,
+            File dataDir, File dataLogDir, int electionAlg,    int electionPort,long myid, 
+            int tickTime, int initLimit, int syncLimit,NIOServerCnxn.Factory cnxnFactory)
+            throws IOException {
+        super(quorumPeers, dataDir, dataLogDir,electionAlg,electionPort, myid,
+                tickTime, initLimit, syncLimit,cnxnFactory);
+    }
+
+    public ObservableQuorumPeer(NIOServerCnxn.Factory cnxnFactory) throws IOException {
+        super(cnxnFactory);
+    }
+
+    // instantiate an observable follower
+    protected Follower makeFollower(File dataDir,File dataLogDir) throws IOException {
+        return new ObservableFollower(this, new ObservableFollowerZooKeeperServer(dataDir,
+                dataLogDir, this,new ZooKeeperServer.BasicDataTreeBuilder()));
+    }
+
+    // instantiate an observable leader
+    protected Leader makeLeader(File dataDir,File dataLogDir) throws IOException {
+        return new ObservableLeader(this, new ObservableLeaderZooKeeperServer(dataDir, 
+                dataLogDir,this,new ZooKeeperServer.BasicDataTreeBuilder()));
+    }
+
+    public void run() {
+        try {
+            ObserverManager.getInstance().notifyObservers(this, Event.STARTUP);
+            super.run();
+        } finally {
+            ObserverManager.getInstance().notifyObservers(this, Event.SHUTDOWN);
+        }
+    }
+
+    public void dispatchEvent(Object observer, Object args) {
+        if(args instanceof ObservableQuorumPeer.Event)
+            ((Event)args).dispatch(this,(QuorumPeerObserver)observer);
+        else
+            ((EventInfo)args).dispatch(this,observer);        
+    }
+
+    // this method is called by the base class when leader election is about to
+    // start; override the method to send a notification before election protocol 
+    // started
+    protected Election makeLEStrategy() {
+        ObserverManager.getInstance().notifyObservers(this,Event.LEADER_ELECTION);
+        return super.makeLEStrategy();
+    }
+}

+ 36 - 0
zookeeper/java/jmx/com/yahoo/zookeeper/server/util/ConnectionObserver.java

@@ -0,0 +1,36 @@
+/**
+ * Copyright 2008, Yahoo! Inc.
+ *
+ * Licensed 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 com.yahoo.zookeeper.server.util;
+
+import com.yahoo.zookeeper.server.ServerCnxn;
+
+/**
+ * Application must implement this interface and register its instance with
+ * the {@link ObserverManager}.
+ */
+public interface ConnectionObserver {
+    /**
+     * A new client connection started
+     * @param sc the new connection instance
+     */
+    public void onNew(ServerCnxn sc);
+    /**
+     * A client connected closed
+     * @param sc the connection instance
+     */
+    public void onClose(ServerCnxn sc);
+}

+ 51 - 0
zookeeper/java/jmx/com/yahoo/zookeeper/server/util/DataTreeObserver.java

@@ -0,0 +1,51 @@
+/**
+ * Copyright 2008, Yahoo! Inc.
+ *
+ * Licensed 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 com.yahoo.zookeeper.server.util;
+
+import com.yahoo.zookeeper.server.DataNode;
+import com.yahoo.zookeeper.server.DataTree;
+
+/**
+ * Application must implement this interface and register its instance with
+ * the {@link ObserverManager}.
+ */
+public interface DataTreeObserver {
+    /**
+     * A new znode has been created.
+     * @param dt the data tree instance
+     * @param node the new znode
+     */
+    public void onAdd(DataTree dt,DataNode node);
+    /**
+     * A znode has been deleted.
+     * @param dt the data tree instance
+     * @param node the deleted znode
+     */
+    public void onDelete(DataTree dt,DataNode node);
+    /**
+     * A znode value has changed.
+     * @param dt the data tree instance
+     * @param node the znode whose value's changed 
+     */
+    public void onUpdate(DataTree dt, DataNode node);
+    /**
+     * A znode's ACL has been modified. 
+     * @param dt the data tree instance
+     * @param node the znode whose ACL has changed
+     */
+    public void onUpdateACL(DataTree dt, DataNode node);
+}

+ 28 - 0
zookeeper/java/jmx/com/yahoo/zookeeper/server/util/EventInfo.java

@@ -0,0 +1,28 @@
+/**
+ * Copyright 2008, Yahoo! Inc.
+ *
+ * Licensed 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 com.yahoo.zookeeper.server.util;
+
+/**
+ * A helper interface used to make event generation and dispatching more 
+ * consistent. An instance of the interface is normally used as the second 
+ * argument to the {@link ObserverManager#notifyObservers()} call.
+ * 
+ * @see ObservableDataTree implements the interface 
+ */
+public interface EventInfo {
+    public void dispatch(Object source, Object observer);
+}

+ 32 - 0
zookeeper/java/jmx/com/yahoo/zookeeper/server/util/ObservableComponent.java

@@ -0,0 +1,32 @@
+/**
+ * Copyright 2008, Yahoo! Inc.
+ *
+ * Licensed 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 com.yahoo.zookeeper.server.util;
+
+/**
+ * An observable component is responsible for decoding and dispatching its events
+ * to an observer. Normally, an observable component is also responsible for
+ * triggering of its events.
+ */
+public interface ObservableComponent {
+    /**
+     * Dispatch an event to the observer.
+     * 
+     * @param observer the observer to be notified
+     * @param args application specific event payload
+     */
+    public void dispatchEvent(Object observer,Object args);
+}

+ 93 - 0
zookeeper/java/jmx/com/yahoo/zookeeper/server/util/ObserverManager.java

@@ -0,0 +1,93 @@
+/**
+ * Copyright 2008, Yahoo! Inc.
+ *
+ * Licensed 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 com.yahoo.zookeeper.server.util;
+
+import java.util.Collections;
+import java.util.Map;
+import java.util.Set;
+import java.util.WeakHashMap;
+
+/**
+ * Abstract observer manager -- a liason between an observable component 
+ * (event producer) and one or more event listeners (event consumers). It takes 
+ * care of the bookeeping chores and of broadcasting events to the observers.
+ * <p>
+ * Observers register themselves using the add() method.
+ */
+public abstract class ObserverManager {
+    private static ObserverManager instance=null;
+    private Map<ObservableComponent,Set<Object>> observableCache=
+        Collections.synchronizedMap(new WeakHashMap<ObservableComponent,Set<Object>>());
+    
+    protected abstract Set<Object> getObserverList(Object key);
+    
+    protected static void setInstance(ObserverManager newInstance){
+        instance=newInstance;
+    }
+    
+    public static ObserverManager getInstance(){
+        assert instance!=null;
+        return instance;
+    }
+    /**
+     * Add an observer to the list of registered observers. 
+     * @param observer to be registered
+     */
+    public void add(Object observer) {
+        Set<Object> ob=getObserverList(observer);
+        assert ob!=null;
+        synchronized(ob){
+            ob.add(observer);
+        }
+    }
+
+    /**
+     * An ObservableComponent will call this method to notify its observers
+     * about an event.
+     * @param source the ObservableComponent instance
+     * @param args application event-specific payload (not used by the ObserverManager) 
+     */
+    public void notifyObservers(ObservableComponent source, Object args) {
+        Set<Object> obs = observableCache.get(source);
+        if(obs==null){
+            obs=getObserverList(source);
+            observableCache.put(source, obs);
+        }
+        synchronized (obs) {
+            for (Object o : obs) {
+                try {
+                    source.dispatchEvent(o, args);
+                } catch (Exception e) {
+                    e.printStackTrace();
+                }
+            }
+        }
+    }
+
+    /**
+     * Unregister the observer.
+     * @param observer to be unregistered.
+     */
+    public void remove(Object observer) {
+        Set<Object> ob=getObserverList(observer);
+        if(ob!=null){
+            synchronized(ob){
+                ob.remove(observer);
+            }
+        }
+    }
+}

+ 67 - 0
zookeeper/java/jmx/com/yahoo/zookeeper/server/util/QuorumPeerObserver.java

@@ -0,0 +1,67 @@
+/**
+ * Copyright 2008, Yahoo! Inc.
+ *
+ * Licensed 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 com.yahoo.zookeeper.server.util;
+
+import com.yahoo.zookeeper.server.quorum.Follower;
+import com.yahoo.zookeeper.server.quorum.Leader;
+import com.yahoo.zookeeper.server.quorum.QuorumPeer;
+
+/**
+* Application must implement this interface and register its instance with
+* the {@link ObserverManager}.
+*/
+public interface QuorumPeerObserver {
+    /**
+     * The local quorum peer qp has started.
+     * @param qp the quorum peer.
+     */
+    public void onStartup(QuorumPeer qp);
+    /**
+     * The local quorum peer is about to shutdown.
+     * @param qp the quorum peer.
+     */
+    public void onShutdown(QuorumPeer qp);
+    /**
+     * Leader election protocol has started
+     * @param qp quorum peer running the protocol
+     */
+    public void onLeaderElectionStarted(QuorumPeer qp);
+    /**
+     * A new leader has been elected.
+     * @param qp quorum peer hosting the leader
+     * @param newLeader the new leader
+     */
+    public void onLeaderStarted(QuorumPeer qp, Leader newLeader);
+    /**
+     * The leader instance is shutting down.
+     * @param qp the quorum peer hosting the leader.
+     * @param leader the leader instance
+     */
+    public void onLeaderShutdown(QuorumPeer qp, Leader leader);
+    /**
+     * A new follower has started.
+     * @param qp the quorum peer instance
+     * @param newFollower the new follower instance
+     */
+    public void onFollowerStarted(QuorumPeer qp, Follower newFollower);
+    /**
+     * The follower is shutting down.
+     * @param qp the quorum peer instance
+     * @param follower the follower instance
+     */
+    public void onFollowerShutdown(QuorumPeer qp, Follower follower);
+}

+ 36 - 0
zookeeper/java/jmx/com/yahoo/zookeeper/server/util/ServerObserver.java

@@ -0,0 +1,36 @@
+/**
+ * Copyright 2008, Yahoo! Inc.
+ *
+ * Licensed 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 com.yahoo.zookeeper.server.util;
+
+import com.yahoo.zookeeper.server.ZooKeeperServer;
+
+/**
+ * Application must implement this interface and register its instance with
+ * the {@link ObserverManager}.
+ */
+public interface ServerObserver {
+    /**
+     * The server just started.
+     * @param server the new fully initialized instance of the server
+     */
+    public void onStartup(ZooKeeperServer server);
+    /**
+     * Tne server is about to shutdown.
+     * @param server instance of zookeeper server
+     */
+    public void onShutdown(ZooKeeperServer server);
+}

+ 59 - 0
zookeeper/java/jmx/com/yahoo/zookeeper/server/util/ZooKeeperObserverManager.java

@@ -0,0 +1,59 @@
+/**
+ * Copyright 2008, Yahoo! Inc.
+ *
+ * Licensed 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 com.yahoo.zookeeper.server.util;
+
+import java.util.HashSet;
+import java.util.Set;
+
+import com.yahoo.zookeeper.server.DataTree;
+import com.yahoo.zookeeper.server.ServerCnxn;
+import com.yahoo.zookeeper.server.ZooKeeperObserverNotifier;
+import com.yahoo.zookeeper.server.quorum.QuorumPeer;
+
+/**
+ * Zookeeper specific implementation of ObserverManager. It implements a mapping
+ * of observer classes to a set of observer instances.
+ */
+public class ZooKeeperObserverManager extends ObserverManager {
+
+    private ZooKeeperObserverManager(){}
+    
+    /**
+     * Explicitly set this class as a concrete instance of ObserverManager.
+     */
+    public static void setAsConcrete(){
+        setInstance(new ZooKeeperObserverManager());
+    }
+    
+    protected Set<Object> getObserverList(Object ob){
+        if(ob instanceof ConnectionObserver || ob instanceof ServerCnxn)
+            return connectionObservers;
+        else if(ob instanceof ServerObserver || ob instanceof ZooKeeperObserverNotifier)
+            return serverObservers;
+        else if(ob instanceof QuorumPeerObserver || ob instanceof QuorumPeer)
+            return quorumPeerObservers;
+        else if(ob instanceof DataTreeObserver || ob instanceof DataTree)
+            return dataTreeObservers;
+        assert false;
+        return null;
+    }
+    
+    private Set<Object> serverObservers=new HashSet<Object>();
+    private Set<Object> connectionObservers=new HashSet<Object>();
+    private Set<Object> dataTreeObservers=new HashSet<Object>();
+    private Set<Object> quorumPeerObservers=new HashSet<Object>();
+}