Browse Source

ZOOKEEPER-733. use netty to handle client connections

git-svn-id: https://svn.apache.org/repos/asf/hadoop/zookeeper/trunk@986575 13f79535-47bb-0310-9956-ffa450edef68
Patrick D. Hunt 15 years ago
parent
commit
30cff66e5e
63 changed files with 3117 additions and 997 deletions
  1. 2 0
      CHANGES.txt
  2. 5 0
      build.xml
  3. 10 2
      ivy.xml
  4. 37 0
      src/docs/src/documentation/content/xdocs/zookeeperAdmin.xml
  5. 202 0
      src/java/libtest/accessive.LICENSE.txt
  6. BIN
      src/java/libtest/accessive.jar
  7. 15 0
      src/java/main/org/apache/zookeeper/ClientCnxn.java
  8. 1 1
      src/java/main/org/apache/zookeeper/ZooKeeper.java
  9. 6 7
      src/java/main/org/apache/zookeeper/server/ConnectionBean.java
  10. 5 10
      src/java/main/org/apache/zookeeper/server/FinalRequestProcessor.java
  11. 93 643
      src/java/main/org/apache/zookeeper/server/NIOServerCnxn.java
  12. 306 0
      src/java/main/org/apache/zookeeper/server/NIOServerCnxnFactory.java
  13. 820 0
      src/java/main/org/apache/zookeeper/server/NettyServerCnxn.java
  14. 396 0
      src/java/main/org/apache/zookeeper/server/NettyServerCnxnFactory.java
  15. 3 1
      src/java/main/org/apache/zookeeper/server/PrepRequestProcessor.java
  16. 381 57
      src/java/main/org/apache/zookeeper/server/ServerCnxn.java
  17. 134 0
      src/java/main/org/apache/zookeeper/server/ServerCnxnFactory.java
  18. 2 0
      src/java/main/org/apache/zookeeper/server/SessionTrackerImpl.java
  19. 68 0
      src/java/main/org/apache/zookeeper/server/Stats.java
  20. 1 0
      src/java/main/org/apache/zookeeper/server/SyncRequestProcessor.java
  21. 182 22
      src/java/main/org/apache/zookeeper/server/ZooKeeperServer.java
  22. 3 3
      src/java/main/org/apache/zookeeper/server/ZooKeeperServerBean.java
  23. 3 2
      src/java/main/org/apache/zookeeper/server/ZooKeeperServerMain.java
  24. 2 2
      src/java/main/org/apache/zookeeper/server/auth/DigestAuthenticationProvider.java
  25. 3 3
      src/java/main/org/apache/zookeeper/server/auth/IPAuthenticationProvider.java
  26. 1 0
      src/java/main/org/apache/zookeeper/server/quorum/CommitProcessor.java
  27. 2 2
      src/java/main/org/apache/zookeeper/server/quorum/FastLeaderElection.java
  28. 4 5
      src/java/main/org/apache/zookeeper/server/quorum/Follower.java
  29. 1 0
      src/java/main/org/apache/zookeeper/server/quorum/FollowerRequestProcessor.java
  30. 2 1
      src/java/main/org/apache/zookeeper/server/quorum/FollowerZooKeeperServer.java
  31. 4 1
      src/java/main/org/apache/zookeeper/server/quorum/Leader.java
  32. 1 1
      src/java/main/org/apache/zookeeper/server/quorum/LeaderZooKeeperServer.java
  33. 3 4
      src/java/main/org/apache/zookeeper/server/quorum/Learner.java
  34. 1 1
      src/java/main/org/apache/zookeeper/server/quorum/LearnerZooKeeperServer.java
  35. 1 7
      src/java/main/org/apache/zookeeper/server/quorum/LocalPeerBean.java
  36. 1 0
      src/java/main/org/apache/zookeeper/server/quorum/ObserverRequestProcessor.java
  37. 6 0
      src/java/main/org/apache/zookeeper/server/quorum/ProposalRequestProcessor.java
  38. 5 3
      src/java/main/org/apache/zookeeper/server/quorum/QuorumCnxManager.java
  39. 23 20
      src/java/main/org/apache/zookeeper/server/quorum/QuorumPeer.java
  40. 4 4
      src/java/main/org/apache/zookeeper/server/quorum/QuorumPeerMain.java
  41. 1 3
      src/java/test/org/apache/zookeeper/server/CRCTest.java
  42. 2 4
      src/java/test/org/apache/zookeeper/server/InvalidSnapshotTest.java
  43. 80 76
      src/java/test/org/apache/zookeeper/test/ACLTest.java
  44. 0 2
      src/java/test/org/apache/zookeeper/test/AsyncHammerTest.java
  45. 58 36
      src/java/test/org/apache/zookeeper/test/ClientBase.java
  46. 3 3
      src/java/test/org/apache/zookeeper/test/ClientPortBindTest.java
  47. 9 0
      src/java/test/org/apache/zookeeper/test/DisconnectableZooKeeper.java
  48. 2 4
      src/java/test/org/apache/zookeeper/test/InvalidSnapshotTest.java
  49. 6 3
      src/java/test/org/apache/zookeeper/test/LENonTerminateTest.java
  50. 43 0
      src/java/test/org/apache/zookeeper/test/NioNettySuiteBase.java
  51. 30 0
      src/java/test/org/apache/zookeeper/test/NioNettySuiteHammerTest.java
  52. 37 0
      src/java/test/org/apache/zookeeper/test/NioNettySuiteTest.java
  53. 3 5
      src/java/test/org/apache/zookeeper/test/OOMTest.java
  54. 15 1
      src/java/test/org/apache/zookeeper/test/ObserverTest.java
  55. 2 4
      src/java/test/org/apache/zookeeper/test/PurgeTxnTest.java
  56. 6 0
      src/java/test/org/apache/zookeeper/test/QuorumBase.java
  57. 17 9
      src/java/test/org/apache/zookeeper/test/QuorumTest.java
  58. 4 6
      src/java/test/org/apache/zookeeper/test/RecoveryTest.java
  59. 2 5
      src/java/test/org/apache/zookeeper/test/RepeatStartupTest.java
  60. 44 26
      src/java/test/org/apache/zookeeper/test/SessionTest.java
  61. 10 3
      src/java/test/org/apache/zookeeper/test/TruncateTest.java
  62. 2 4
      src/java/test/org/apache/zookeeper/test/UpgradeTest.java
  63. 2 1
      src/java/test/org/apache/zookeeper/test/WatcherTest.java

+ 2 - 0
CHANGES.txt

@@ -107,6 +107,8 @@ IMPROVEMENTS:
 
 
   ZOOKEEPER-809. Improved REST Interface (Andrei Savu via phunt)
   ZOOKEEPER-809. Improved REST Interface (Andrei Savu via phunt)
 
 
+  ZOOKEEPER-733. use netty to handle client connections (breed and phunt)
+
 NEW FEATURES:
 NEW FEATURES:
   ZOOKEEPER-729. Java client API to recursively delete a subtree.
   ZOOKEEPER-729. Java client API to recursively delete a subtree.
   (Kay Kay via henry)
   (Kay Kay via henry)

+ 5 - 0
build.xml

@@ -62,6 +62,7 @@
 
 
     <property name="test.java.build.dir" value="${build.dir}/test"/>
     <property name="test.java.build.dir" value="${build.dir}/test"/>
     <property name="test.java.classes" value="${test.java.build.dir}/classes"/>
     <property name="test.java.classes" value="${test.java.build.dir}/classes"/>
+    <property name="test.lib.dir" value="${src.dir}/java/libtest" />
     <property name="test.src.dir" value="${src.dir}/java/test"/>
     <property name="test.src.dir" value="${src.dir}/java/test"/>
     <property name="systest.src.dir" value="${src.dir}/java/systest"/>
     <property name="systest.src.dir" value="${src.dir}/java/systest"/>
     <property name="test.log.dir" value="${test.java.build.dir}/logs" />
     <property name="test.log.dir" value="${test.java.build.dir}/logs" />
@@ -335,6 +336,10 @@
     </target>
     </target>
 
 
     <target name="compile-test" depends="ivy-retrieve-test,compile">
     <target name="compile-test" depends="ivy-retrieve-test,compile">
+      <copy todir="${ivy.test.lib}">
+        <fileset dir="${test.lib.dir}" includes="*.jar"/>
+      </copy>
+
       <mkdir dir="${test.java.classes}"/>
       <mkdir dir="${test.java.classes}"/>
       <javac srcdir="${test.src.dir}" destdir="${test.java.classes}"
       <javac srcdir="${test.src.dir}" destdir="${test.java.classes}"
              target="${javac.target}" debug="on">
              target="${javac.target}" debug="on">

+ 10 - 2
ivy.xml

@@ -32,10 +32,18 @@
     <conf name="releaseaudit" visibility="private" description="Artifacts required for releaseaudit target"/>
     <conf name="releaseaudit" visibility="private" description="Artifacts required for releaseaudit target"/>
   </configurations>
   </configurations>
 
 
+  <publications>
+    <artifact name='org.jboss.netty' type='jar' ext='jar' />
+  </publications>
+
   <dependencies>
   <dependencies>
     <!-- transitive false turns off dependency checking, log4j deps seem borked -->
     <!-- transitive false turns off dependency checking, log4j deps seem borked -->
-    <dependency org="log4j" name="log4j" rev="1.2.15" transitive="false"/>
-    <dependency org="jline" name="jline" rev="0.9.94" transitive="false"/>
+    <dependency org="log4j" name="log4j" rev="1.2.15" transitive="false" conf="default"/>
+    <dependency org="jline" name="jline" rev="0.9.94" transitive="false" conf="default"/>
+
+    <dependency org="org.jboss.netty" name="netty" conf="default" rev="3.1.5.GA">
+      <artifact name="netty" type="jar" conf="default"/>
+    </dependency>
 
 
     <dependency org="junit" name="junit" rev="4.8.1" conf="test->default"/>
     <dependency org="junit" name="junit" rev="4.8.1" conf="test->default"/>
     <dependency org="checkstyle" name="checkstyle" rev="5.0"
     <dependency org="checkstyle" name="checkstyle" rev="5.0"

+ 37 - 0
src/docs/src/documentation/content/xdocs/zookeeperAdmin.xml

@@ -1016,6 +1016,43 @@ server.3=zoo3:2888:3888</programlisting>
           </varlistentry>
           </varlistentry>
         </variablelist>
         </variablelist>
       </section>
       </section>
+
+      <section>
+        <title>Communication using the Netty framework</title>
+
+        <para><emphasis role="bold">New in
+            3.4:</emphasis> <ulink url="http://jboss.org/netty">Netty</ulink>
+            is an NIO based client/server communication framework, it
+            simplifies (over NIO being used directly) many of the
+            complexities of network level communication for java
+            applications. Additionally the Netty framework has built
+            in support for encryption (SSL) and authentication
+            (certificates). These are optional features and can be
+            turned on or off individually.
+        </para>
+        <para>Prior to version 3.4 ZooKeeper has always used NIO
+            directly, however in versions 3.4 and later Netty is
+            supported as an option to NIO (replaces). NIO continues to
+            be the default, however Netty based communication can be
+            used in place of NIO by setting the environment variable
+            "zookeeper.serverCnxnFactory" to
+            "org.apache.zookeeper.server.NettyServerCnxnFactory". You
+            have the option of setting this on either the client(s) or
+            server(s), typically you would want to set this on both,
+            however that is at your discretion.
+        </para>
+        <para>
+          TBD - tuning options for netty - currently there are none that are netty specific but we should add some. Esp around max bound on the number of reader worker threads netty creates.
+        </para>
+        <para>
+          TBD - how to manage encryption
+        </para>
+        <para>
+          TBD - how to manage certificates
+        </para>
+
+      </section>
+
     </section>
     </section>
 
 
     <section id="sc_zkCommands">
     <section id="sc_zkCommands">

+ 202 - 0
src/java/libtest/accessive.LICENSE.txt

@@ -0,0 +1,202 @@
+
+                                 Apache License
+                           Version 2.0, January 2004
+                        http://www.apache.org/licenses/
+
+   TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION
+
+   1. Definitions.
+
+      "License" shall mean the terms and conditions for use, reproduction,
+      and distribution as defined by Sections 1 through 9 of this document.
+
+      "Licensor" shall mean the copyright owner or entity authorized by
+      the copyright owner that is granting the License.
+
+      "Legal Entity" shall mean the union of the acting entity and all
+      other entities that control, are controlled by, or are under common
+      control with that entity. For the purposes of this definition,
+      "control" means (i) the power, direct or indirect, to cause the
+      direction or management of such entity, whether by contract or
+      otherwise, or (ii) ownership of fifty percent (50%) or more of the
+      outstanding shares, or (iii) beneficial ownership of such entity.
+
+      "You" (or "Your") shall mean an individual or Legal Entity
+      exercising permissions granted by this License.
+
+      "Source" form shall mean the preferred form for making modifications,
+      including but not limited to software source code, documentation
+      source, and configuration files.
+
+      "Object" form shall mean any form resulting from mechanical
+      transformation or translation of a Source form, including but
+      not limited to compiled object code, generated documentation,
+      and conversions to other media types.
+
+      "Work" shall mean the work of authorship, whether in Source or
+      Object form, made available under the License, as indicated by a
+      copyright notice that is included in or attached to the work
+      (an example is provided in the Appendix below).
+
+      "Derivative Works" shall mean any work, whether in Source or Object
+      form, that is based on (or derived from) the Work and for which the
+      editorial revisions, annotations, elaborations, or other modifications
+      represent, as a whole, an original work of authorship. For the purposes
+      of this License, Derivative Works shall not include works that remain
+      separable from, or merely link (or bind by name) to the interfaces of,
+      the Work and Derivative Works thereof.
+
+      "Contribution" shall mean any work of authorship, including
+      the original version of the Work and any modifications or additions
+      to that Work or Derivative Works thereof, that is intentionally
+      submitted to Licensor for inclusion in the Work by the copyright owner
+      or by an individual or Legal Entity authorized to submit on behalf of
+      the copyright owner. For the purposes of this definition, "submitted"
+      means any form of electronic, verbal, or written communication sent
+      to the Licensor or its representatives, including but not limited to
+      communication on electronic mailing lists, source code control systems,
+      and issue tracking systems that are managed by, or on behalf of, the
+      Licensor for the purpose of discussing and improving the Work, but
+      excluding communication that is conspicuously marked or otherwise
+      designated in writing by the copyright owner as "Not a Contribution."
+
+      "Contributor" shall mean Licensor and any individual or Legal Entity
+      on behalf of whom a Contribution has been received by Licensor and
+      subsequently incorporated within the Work.
+
+   2. Grant of Copyright License. Subject to the terms and conditions of
+      this License, each Contributor hereby grants to You a perpetual,
+      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+      copyright license to reproduce, prepare Derivative Works of,
+      publicly display, publicly perform, sublicense, and distribute the
+      Work and such Derivative Works in Source or Object form.
+
+   3. Grant of Patent License. Subject to the terms and conditions of
+      this License, each Contributor hereby grants to You a perpetual,
+      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+      (except as stated in this section) patent license to make, have made,
+      use, offer to sell, sell, import, and otherwise transfer the Work,
+      where such license applies only to those patent claims licensable
+      by such Contributor that are necessarily infringed by their
+      Contribution(s) alone or by combination of their Contribution(s)
+      with the Work to which such Contribution(s) was submitted. If You
+      institute patent litigation against any entity (including a
+      cross-claim or counterclaim in a lawsuit) alleging that the Work
+      or a Contribution incorporated within the Work constitutes direct
+      or contributory patent infringement, then any patent licenses
+      granted to You under this License for that Work shall terminate
+      as of the date such litigation is filed.
+
+   4. Redistribution. You may reproduce and distribute copies of the
+      Work or Derivative Works thereof in any medium, with or without
+      modifications, and in Source or Object form, provided that You
+      meet the following conditions:
+
+      (a) You must give any other recipients of the Work or
+          Derivative Works a copy of this License; and
+
+      (b) You must cause any modified files to carry prominent notices
+          stating that You changed the files; and
+
+      (c) You must retain, in the Source form of any Derivative Works
+          that You distribute, all copyright, patent, trademark, and
+          attribution notices from the Source form of the Work,
+          excluding those notices that do not pertain to any part of
+          the Derivative Works; and
+
+      (d) If the Work includes a "NOTICE" text file as part of its
+          distribution, then any Derivative Works that You distribute must
+          include a readable copy of the attribution notices contained
+          within such NOTICE file, excluding those notices that do not
+          pertain to any part of the Derivative Works, in at least one
+          of the following places: within a NOTICE text file distributed
+          as part of the Derivative Works; within the Source form or
+          documentation, if provided along with the Derivative Works; or,
+          within a display generated by the Derivative Works, if and
+          wherever such third-party notices normally appear. The contents
+          of the NOTICE file are for informational purposes only and
+          do not modify the License. You may add Your own attribution
+          notices within Derivative Works that You distribute, alongside
+          or as an addendum to the NOTICE text from the Work, provided
+          that such additional attribution notices cannot be construed
+          as modifying the License.
+
+      You may add Your own copyright statement to Your modifications and
+      may provide additional or different license terms and conditions
+      for use, reproduction, or distribution of Your modifications, or
+      for any such Derivative Works as a whole, provided Your use,
+      reproduction, and distribution of the Work otherwise complies with
+      the conditions stated in this License.
+
+   5. Submission of Contributions. Unless You explicitly state otherwise,
+      any Contribution intentionally submitted for inclusion in the Work
+      by You to the Licensor shall be under the terms and conditions of
+      this License, without any additional terms or conditions.
+      Notwithstanding the above, nothing herein shall supersede or modify
+      the terms of any separate license agreement you may have executed
+      with Licensor regarding such Contributions.
+
+   6. Trademarks. This License does not grant permission to use the trade
+      names, trademarks, service marks, or product names of the Licensor,
+      except as required for reasonable and customary use in describing the
+      origin of the Work and reproducing the content of the NOTICE file.
+
+   7. Disclaimer of Warranty. Unless required by applicable law or
+      agreed to in writing, Licensor provides the Work (and each
+      Contributor provides its Contributions) on an "AS IS" BASIS,
+      WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
+      implied, including, without limitation, any warranties or conditions
+      of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A
+      PARTICULAR PURPOSE. You are solely responsible for determining the
+      appropriateness of using or redistributing the Work and assume any
+      risks associated with Your exercise of permissions under this License.
+
+   8. Limitation of Liability. In no event and under no legal theory,
+      whether in tort (including negligence), contract, or otherwise,
+      unless required by applicable law (such as deliberate and grossly
+      negligent acts) or agreed to in writing, shall any Contributor be
+      liable to You for damages, including any direct, indirect, special,
+      incidental, or consequential damages of any character arising as a
+      result of this License or out of the use or inability to use the
+      Work (including but not limited to damages for loss of goodwill,
+      work stoppage, computer failure or malfunction, or any and all
+      other commercial damages or losses), even if such Contributor
+      has been advised of the possibility of such damages.
+
+   9. Accepting Warranty or Additional Liability. While redistributing
+      the Work or Derivative Works thereof, You may choose to offer,
+      and charge a fee for, acceptance of support, warranty, indemnity,
+      or other liability obligations and/or rights consistent with this
+      License. However, in accepting such obligations, You may act only
+      on Your own behalf and on Your sole responsibility, not on behalf
+      of any other Contributor, and only if You agree to indemnify,
+      defend, and hold each Contributor harmless for any liability
+      incurred by, or claims asserted against, such Contributor by reason
+      of your accepting any such warranty or additional liability.
+
+   END OF TERMS AND CONDITIONS
+
+   APPENDIX: How to apply the Apache License to your work.
+
+      To apply the Apache License to your work, attach the following
+      boilerplate notice, with the fields enclosed by brackets "[]"
+      replaced with your own identifying information. (Don't include
+      the brackets!)  The text should be enclosed in the appropriate
+      comment syntax for the file format. We also recommend that a
+      file or class name and description of purpose be included on the
+      same "printed page" as the copyright notice for easier
+      identification within third-party archives.
+
+   Copyright 1999-2005 The Apache Software Foundation
+
+   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.

BIN
src/java/libtest/accessive.jar


+ 15 - 0
src/java/main/org/apache/zookeeper/ClientCnxn.java

@@ -688,6 +688,15 @@ public class ClientCnxn {
         }
         }
 
 
         void readConnectResult() throws IOException {
         void readConnectResult() throws IOException {
+            if (LOG.isTraceEnabled()) {
+                StringBuffer buf = new StringBuffer("0x[");
+                for (byte b : incomingBuffer.array()) {
+                    buf.append(Integer.toHexString(b) + ",");
+                }
+                buf.append("]");
+                LOG.trace("readConnectRestult " + incomingBuffer.remaining() 
+                        + " " + buf.toString());
+            }
             ByteBufferInputStream bbis = new ByteBufferInputStream(
             ByteBufferInputStream bbis = new ByteBufferInputStream(
                     incomingBuffer);
                     incomingBuffer);
             BinaryInputArchive bbia = BinaryInputArchive.getArchive(bbis);
             BinaryInputArchive bbia = BinaryInputArchive.getArchive(bbis);
@@ -1154,7 +1163,13 @@ public class ClientCnxn {
             }
             }
             cleanup();
             cleanup();
             try {
             try {
+                if (LOG.isTraceEnabled()) {
+                    LOG.trace("Doing client selector close");
+                }
                 selector.close();
                 selector.close();
+                if (LOG.isTraceEnabled()) {
+                    LOG.trace("Closed client selector");
+                }
             } catch (IOException e) {
             } catch (IOException e) {
                 LOG.warn("Ignoring exception during selector close", e);
                 LOG.warn("Ignoring exception during selector close", e);
             }
             }

+ 1 - 1
src/java/main/org/apache/zookeeper/ZooKeeper.java

@@ -438,7 +438,7 @@ public class ZooKeeper {
         LOG.info("Initiating client connection, connectString=" + connectString
         LOG.info("Initiating client connection, connectString=" + connectString
                 + " sessionTimeout=" + sessionTimeout
                 + " sessionTimeout=" + sessionTimeout
                 + " watcher=" + watcher
                 + " watcher=" + watcher
-                + " sessionId=" + sessionId
+                + " sessionId=" + Long.toHexString(sessionId)
                 + " sessionPasswd="
                 + " sessionPasswd="
                 + (sessionPasswd == null ? "<null>" : "<hidden>"));
                 + (sessionPasswd == null ? "<null>" : "<hidden>"));
 
 

+ 6 - 7
src/java/main/org/apache/zookeeper/server/ConnectionBean.java

@@ -29,7 +29,6 @@ import javax.management.ObjectName;
 import org.apache.log4j.Logger;
 import org.apache.log4j.Logger;
 import org.apache.zookeeper.jmx.MBeanRegistry;
 import org.apache.zookeeper.jmx.MBeanRegistry;
 import org.apache.zookeeper.jmx.ZKMBeanInfo;
 import org.apache.zookeeper.jmx.ZKMBeanInfo;
-import org.apache.zookeeper.server.NIOServerCnxn.CnxnStats;
 
 
 /**
 /**
  * Implementation of connection MBean interface.
  * Implementation of connection MBean interface.
@@ -38,7 +37,7 @@ public class ConnectionBean implements ConnectionMXBean, ZKMBeanInfo {
     private static final Logger LOG = Logger.getLogger(ConnectionBean.class);
     private static final Logger LOG = Logger.getLogger(ConnectionBean.class);
 
 
     private final ServerCnxn connection;
     private final ServerCnxn connection;
-    private final CnxnStats stats;
+    private final Stats stats;
 
 
     private final ZooKeeperServer zk;
     private final ZooKeeperServer zk;
     
     
@@ -47,10 +46,10 @@ public class ConnectionBean implements ConnectionMXBean, ZKMBeanInfo {
 
 
     public ConnectionBean(ServerCnxn connection,ZooKeeperServer zk){
     public ConnectionBean(ServerCnxn connection,ZooKeeperServer zk){
         this.connection = connection;
         this.connection = connection;
-        this.stats = (CnxnStats)connection.getStats();
+        this.stats = connection;
         this.zk = zk;
         this.zk = zk;
         
         
-        InetSocketAddress sockAddr = connection.getRemoteAddress();
+        InetSocketAddress sockAddr = connection.getRemoteSocketAddress();
         if (sockAddr == null) {
         if (sockAddr == null) {
             remoteIP = "Unknown";
             remoteIP = "Unknown";
         } else {
         } else {
@@ -69,7 +68,7 @@ public class ConnectionBean implements ConnectionMXBean, ZKMBeanInfo {
     }
     }
 
 
     public String getSourceIP() {
     public String getSourceIP() {
-        InetSocketAddress sockAddr = connection.getRemoteAddress();
+        InetSocketAddress sockAddr = connection.getRemoteSocketAddress();
         if (sockAddr == null) {
         if (sockAddr == null) {
             return null;
             return null;
         }
         }
@@ -88,7 +87,7 @@ public class ConnectionBean implements ConnectionMXBean, ZKMBeanInfo {
     
     
     public String[] getEphemeralNodes() {
     public String[] getEphemeralNodes() {
         if(zk.getZKDatabase()  !=null){
         if(zk.getZKDatabase()  !=null){
-            String[] res= zk.getZKDatabase().getEphemerals(sessionId)
+            String[] res = zk.getZKDatabase().getEphemerals(sessionId)
                 .toArray(new String[0]);
                 .toArray(new String[0]);
             Arrays.sort(res);
             Arrays.sort(res);
             return res;
             return res;
@@ -114,7 +113,7 @@ public class ConnectionBean implements ConnectionMXBean, ZKMBeanInfo {
     }
     }
 
 
     public void resetCounters() {
     public void resetCounters() {
-        stats.reset();
+        stats.resetStats();
     }
     }
 
 
     @Override
     @Override

+ 5 - 10
src/java/main/org/apache/zookeeper/server/FinalRequestProcessor.java

@@ -49,8 +49,6 @@ import org.apache.zookeeper.proto.SetWatches;
 import org.apache.zookeeper.proto.SyncRequest;
 import org.apache.zookeeper.proto.SyncRequest;
 import org.apache.zookeeper.proto.SyncResponse;
 import org.apache.zookeeper.proto.SyncResponse;
 import org.apache.zookeeper.server.DataTree.ProcessTxnResult;
 import org.apache.zookeeper.server.DataTree.ProcessTxnResult;
-import org.apache.zookeeper.server.NIOServerCnxn.CnxnStats;
-import org.apache.zookeeper.server.NIOServerCnxn.Factory;
 import org.apache.zookeeper.server.ZooKeeperServer.ChangeRecord;
 import org.apache.zookeeper.server.ZooKeeperServer.ChangeRecord;
 import org.apache.zookeeper.txn.CreateSessionTxn;
 import org.apache.zookeeper.txn.CreateSessionTxn;
 import org.apache.zookeeper.txn.ErrorTxn;
 import org.apache.zookeeper.txn.ErrorTxn;
@@ -122,7 +120,7 @@ public class FinalRequestProcessor implements RequestProcessor {
         }
         }
 
 
         if (request.hdr != null && request.hdr.getType() == OpCode.closeSession) {
         if (request.hdr != null && request.hdr.getType() == OpCode.closeSession) {
-            Factory scxn = zks.getServerCnxnFactory();
+            ServerCnxnFactory scxn = zks.getServerCnxnFactory();
             // this might be possible since
             // this might be possible since
             // we might just be playing diffs from the leader
             // we might just be playing diffs from the leader
             if (scxn != null && request.cnxn == null) {
             if (scxn != null && request.cnxn == null) {
@@ -164,8 +162,7 @@ public class FinalRequestProcessor implements RequestProcessor {
                 zks.serverStats().updateLatency(request.createTime);
                 zks.serverStats().updateLatency(request.createTime);
 
 
                 lastOp = "PING";
                 lastOp = "PING";
-                ((CnxnStats)cnxn.getStats())
-                .updateForResponse(request.cxid, request.zxid, lastOp,
+                cnxn.updateStatsForResponse(request.cxid, request.zxid, lastOp,
                         request.createTime, System.currentTimeMillis());
                         request.createTime, System.currentTimeMillis());
 
 
                 cnxn.sendResponse(new ReplyHeader(-2,
                 cnxn.sendResponse(new ReplyHeader(-2,
@@ -176,11 +173,10 @@ public class FinalRequestProcessor implements RequestProcessor {
                 zks.serverStats().updateLatency(request.createTime);
                 zks.serverStats().updateLatency(request.createTime);
 
 
                 lastOp = "SESS";
                 lastOp = "SESS";
-                ((CnxnStats)cnxn.getStats())
-                .updateForResponse(request.cxid, request.zxid, lastOp,
+                cnxn.updateStatsForResponse(request.cxid, request.zxid, lastOp,
                         request.createTime, System.currentTimeMillis());
                         request.createTime, System.currentTimeMillis());
 
 
-                cnxn.finishSessionInit(true);
+                zks.finishSessionInit(request.cnxn, true);
                 return;
                 return;
             }
             }
             case OpCode.create: {
             case OpCode.create: {
@@ -359,8 +355,7 @@ public class FinalRequestProcessor implements RequestProcessor {
             new ReplyHeader(request.cxid, request.zxid, err.intValue());
             new ReplyHeader(request.cxid, request.zxid, err.intValue());
 
 
         zks.serverStats().updateLatency(request.createTime);
         zks.serverStats().updateLatency(request.createTime);
-        ((CnxnStats)cnxn.getStats())
-            .updateForResponse(request.cxid, request.zxid, lastOp,
+        cnxn.updateStatsForResponse(request.cxid, request.zxid, lastOp,
                     request.createTime, System.currentTimeMillis());
                     request.createTime, System.currentTimeMillis());
 
 
         try {
         try {

File diff suppressed because it is too large
+ 93 - 643
src/java/main/org/apache/zookeeper/server/NIOServerCnxn.java


+ 306 - 0
src/java/main/org/apache/zookeeper/server/NIOServerCnxnFactory.java

@@ -0,0 +1,306 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.zookeeper.server;
+
+import java.io.IOException;
+import java.net.InetAddress;
+import java.net.InetSocketAddress;
+import java.nio.ByteBuffer;
+import java.nio.channels.SelectionKey;
+import java.nio.channels.Selector;
+import java.nio.channels.ServerSocketChannel;
+import java.nio.channels.SocketChannel;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Set;
+
+import org.apache.log4j.Logger;
+
+public class NIOServerCnxnFactory extends ServerCnxnFactory implements Runnable {
+    private static final Logger LOG = Logger.getLogger(NIOServerCnxnFactory.class);
+
+    static {
+        Thread.setDefaultUncaughtExceptionHandler(new Thread.UncaughtExceptionHandler() {
+                public void uncaughtException(Thread t, Throwable e) {
+                    LOG.error("Thread " + t + " died", e);
+                }
+            });
+        /**
+         * this is to avoid the jvm bug:
+         * NullPointerException in Selector.open()
+         * http://bugs.sun.com/view_bug.do?bug_id=6427854
+         */
+        try {
+            Selector.open().close();
+        } catch(IOException ie) {
+            LOG.error("Selector failed to open", ie);
+        }
+    }
+
+    ServerSocketChannel ss;
+
+    final Selector selector = Selector.open();
+
+    /**
+     * We use this buffer to do efficient socket I/O. Since there is a single
+     * sender thread per NIOServerCnxn instance, we can use a member variable to
+     * only allocate it once.
+    */
+    final ByteBuffer directBuffer = ByteBuffer.allocateDirect(64 * 1024);
+
+    final HashSet<ServerCnxn> cnxns = new HashSet<ServerCnxn>();
+    final HashMap<InetAddress, Set<NIOServerCnxn>> ipMap =
+        new HashMap<InetAddress, Set<NIOServerCnxn>>( );
+
+    int maxClientCnxns = 10;
+
+
+    /**
+     * Construct a new server connection factory which will accept an unlimited number
+     * of concurrent connections from each client (up to the file descriptor
+     * limits of the operating system). startup(zks) must be called subsequently.
+     * @throws IOException
+     */
+    public NIOServerCnxnFactory() throws IOException {
+    }
+
+    Thread thread;
+    @Override
+    public void configure(InetSocketAddress addr, int maxcc) throws IOException {
+        thread = new Thread(this, "NIOServerCxn.Factory:" + addr);
+        thread.setDaemon(true);
+        maxClientCnxns = maxcc;
+        this.ss = ServerSocketChannel.open();
+        ss.socket().setReuseAddress(true);
+        LOG.info("binding to port " + addr);
+        ss.socket().bind(addr);
+        ss.configureBlocking(false);
+        ss.register(selector, SelectionKey.OP_ACCEPT);
+    }
+
+    /** {@inheritDoc} */
+    public int getMaxClientCnxnsPerHost() {
+        return maxClientCnxns;
+    }
+
+    /** {@inheritDoc} */
+    public void setMaxClientCnxnsPerHost(int max) {
+        maxClientCnxns = max;
+    }
+
+    @Override
+    public void start() {
+        // ensure thread is started once and only once
+        if (thread.getState() == Thread.State.NEW) {
+            thread.start();
+        }
+    }
+
+    @Override
+    public void startup(ZooKeeperServer zks) throws IOException,
+            InterruptedException {
+        start();
+        zks.startdata();
+        zks.startup();
+        setZooKeeperServer(zks);
+    }
+
+    @Override
+    public InetSocketAddress getLocalAddress(){
+        return (InetSocketAddress)ss.socket().getLocalSocketAddress();
+    }
+
+    @Override
+    public int getLocalPort(){
+        return ss.socket().getLocalPort();
+    }
+
+    private void addCnxn(NIOServerCnxn cnxn) {
+        synchronized (cnxns) {
+            cnxns.add(cnxn);
+            synchronized (ipMap){
+                InetAddress addr = cnxn.sock.socket().getInetAddress();
+                Set<NIOServerCnxn> s = ipMap.get(addr);
+                if (s == null) {
+                    // in general we will see 1 connection from each
+                    // host, setting the initial cap to 2 allows us
+                    // to minimize mem usage in the common case
+                    // of 1 entry --  we need to set the initial cap
+                    // to 2 to avoid rehash when the first entry is added
+                    s = new HashSet<NIOServerCnxn>(2);
+                    s.add(cnxn);
+                    ipMap.put(addr,s);
+                } else {
+                    s.add(cnxn);
+                }
+            }
+        }
+    }
+
+    protected NIOServerCnxn createConnection(SocketChannel sock,
+            SelectionKey sk) throws IOException {
+        return new NIOServerCnxn(zkServer, sock, sk, this);
+    }
+
+    private int getClientCnxnCount(InetAddress cl) {
+        // The ipMap lock covers both the map, and its contents
+        // (that is, the cnxn sets shouldn't be modified outside of
+        // this lock)
+        synchronized (ipMap) {
+            Set<NIOServerCnxn> s = ipMap.get(cl);
+            if (s == null) return 0;
+            return s.size();
+        }
+    }
+
+    public void run() {
+        while (!ss.socket().isClosed()) {
+            try {
+                selector.select(1000);
+                Set<SelectionKey> selected;
+                synchronized (this) {
+                    selected = selector.selectedKeys();
+                }
+                ArrayList<SelectionKey> selectedList = new ArrayList<SelectionKey>(
+                        selected);
+                Collections.shuffle(selectedList);
+                for (SelectionKey k : selectedList) {
+                    if ((k.readyOps() & SelectionKey.OP_ACCEPT) != 0) {
+                        SocketChannel sc = ((ServerSocketChannel) k
+                                .channel()).accept();
+                        InetAddress ia = sc.socket().getInetAddress();
+                        int cnxncount = getClientCnxnCount(ia);
+                        if (maxClientCnxns > 0 && cnxncount >= maxClientCnxns){
+                            LOG.warn("Too many connections from " + ia
+                                     + " - max is " + maxClientCnxns );
+                            sc.close();
+                        } else {
+                            LOG.info("Accepted socket connection from "
+                                     + sc.socket().getRemoteSocketAddress());
+                            sc.configureBlocking(false);
+                            SelectionKey sk = sc.register(selector,
+                                    SelectionKey.OP_READ);
+                            NIOServerCnxn cnxn = createConnection(sc, sk);
+                            sk.attach(cnxn);
+                            addCnxn(cnxn);
+                        }
+                    } else if ((k.readyOps() & (SelectionKey.OP_READ | SelectionKey.OP_WRITE)) != 0) {
+                        NIOServerCnxn c = (NIOServerCnxn) k.attachment();
+                        c.doIO(k);
+                    } else {
+                        if (LOG.isDebugEnabled()) {
+                            LOG.debug("Unexpected ops in select "
+                                      + k.readyOps());
+                        }
+                    }
+                }
+                selected.clear();
+            } catch (RuntimeException e) {
+                LOG.warn("Ignoring unexpected runtime exception", e);
+            } catch (Exception e) {
+                LOG.warn("Ignoring exception", e);
+            }
+        }
+        closeAll();
+        LOG.info("NIOServerCnxn factory exited run method");
+    }
+
+    /**
+     * clear all the connections in the selector
+     *
+     */
+    @Override
+    @SuppressWarnings("unchecked")
+    synchronized public void closeAll() {
+        selector.wakeup();
+        HashSet<NIOServerCnxn> cnxns;
+        synchronized (this.cnxns) {
+            cnxns = (HashSet<NIOServerCnxn>)this.cnxns.clone();
+        }
+        // got to clear all the connections that we have in the selector
+        for (NIOServerCnxn cnxn: cnxns) {
+            try {
+                // don't hold this.cnxns lock as deadlock may occur
+                cnxn.close();
+            } catch (Exception e) {
+                LOG.warn("Ignoring exception closing cnxn sessionid 0x"
+                         + Long.toHexString(cnxn.sessionId), e);
+            }
+        }
+    }
+
+    public void shutdown() {
+        try {
+            ss.close();
+            closeAll();
+            thread.interrupt();
+            thread.join();
+        } catch (InterruptedException e) {
+            LOG.warn("Ignoring interrupted exception during shutdown", e);
+        } catch (Exception e) {
+            LOG.warn("Ignoring unexpected exception during shutdown", e);
+        }
+        try {
+            selector.close();
+        } catch (IOException e) {
+            LOG.warn("Selector closing", e);
+        }
+        if (zkServer != null) {
+            zkServer.shutdown();
+        }
+    }
+
+    @Override
+    public synchronized void closeSession(long sessionId) {
+        selector.wakeup();
+        closeSessionWithoutWakeup(sessionId);
+    }
+
+    @SuppressWarnings("unchecked")
+    private void closeSessionWithoutWakeup(long sessionId) {
+        HashSet<NIOServerCnxn> cnxns;
+        synchronized (this.cnxns) {
+            cnxns = (HashSet<NIOServerCnxn>)this.cnxns.clone();
+        }
+
+        for (NIOServerCnxn cnxn : cnxns) {
+            if (cnxn.getSessionId() == sessionId) {
+                try {
+                    cnxn.close();
+                } catch (Exception e) {
+                    LOG.warn("exception during session close", e);
+                }
+                break;
+            }
+        }
+    }
+
+    @Override
+    public void join() throws InterruptedException {
+        thread.join();
+    }
+
+    @Override
+    public Iterable<ServerCnxn> getConnections() {
+        return cnxns;
+    }
+
+}

+ 820 - 0
src/java/main/org/apache/zookeeper/server/NettyServerCnxn.java

@@ -0,0 +1,820 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.zookeeper.server;
+
+import static org.jboss.netty.buffer.ChannelBuffers.wrappedBuffer;
+
+import java.io.BufferedWriter;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.io.StringWriter;
+import java.io.Writer;
+import java.lang.management.ManagementFactory;
+import java.lang.management.OperatingSystemMXBean;
+import java.net.InetSocketAddress;
+import java.net.SocketAddress;
+import java.nio.ByteBuffer;
+import java.util.AbstractSet;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicLong;
+
+import org.apache.jute.BinaryInputArchive;
+import org.apache.jute.BinaryOutputArchive;
+import org.apache.jute.Record;
+import org.apache.log4j.Logger;
+import org.apache.zookeeper.Environment;
+import org.apache.zookeeper.Version;
+import org.apache.zookeeper.WatchedEvent;
+import org.apache.zookeeper.proto.ReplyHeader;
+import org.apache.zookeeper.proto.WatcherEvent;
+import org.apache.zookeeper.server.quorum.Leader;
+import org.apache.zookeeper.server.quorum.LeaderZooKeeperServer;
+import org.jboss.netty.buffer.ChannelBuffer;
+import org.jboss.netty.buffer.ChannelBuffers;
+import org.jboss.netty.channel.Channel;
+import org.jboss.netty.channel.ChannelFuture;
+import org.jboss.netty.channel.MessageEvent;
+
+import com.sun.management.UnixOperatingSystemMXBean;
+
+public class NettyServerCnxn extends ServerCnxn {
+    Logger LOG = Logger.getLogger(NettyServerCnxn.class);
+    Channel channel;
+    ChannelBuffer queuedBuffer;
+    volatile boolean throttled;
+    ByteBuffer bb;
+    ByteBuffer bbLen = ByteBuffer.allocate(4);
+    long sessionId;
+    int sessionTimeout;
+    AtomicLong outstandingCount = new AtomicLong();
+
+    /** The ZooKeeperServer for this connection. May be null if the server
+     * is not currently serving requests (for example if the server is not
+     * an active quorum participant.
+     */
+    private volatile ZooKeeperServer zkServer;
+
+    NettyServerCnxnFactory factory;
+    boolean initialized;
+    
+    NettyServerCnxn(Channel channel, ZooKeeperServer zks, NettyServerCnxnFactory factory) {
+        this.channel = channel;
+        this.zkServer = zks;
+        this.factory = factory;
+    }
+    
+    @Override
+    public void close() {
+        if (LOG.isDebugEnabled()) {
+            LOG.debug("close called for sessionid:0x"
+                    + Long.toHexString(sessionId));
+        }
+        synchronized(factory.cnxns){
+            // if this is not in cnxns then it's already closed
+            if (!factory.cnxns.remove(this)) {
+                if (LOG.isDebugEnabled()) {
+                    LOG.debug("cnxns size:" + factory.cnxns.size());
+                }
+                return;
+            }
+            if (LOG.isDebugEnabled()) {
+                LOG.debug("close in progress for sessionid:0x"
+                        + Long.toHexString(sessionId));
+            }
+
+            synchronized (factory.ipMap) {
+                Set<NettyServerCnxn> s =
+                    factory.ipMap.get(((InetSocketAddress)channel
+                            .getRemoteAddress()).getAddress());
+                s.remove(this);
+            }
+    
+            if (channel.isOpen()) {
+                channel.close();
+            }
+            factory.unregisterConnection(this);
+        }
+    }
+
+    @Override
+    public long getSessionId() {
+        return sessionId;
+    }
+
+    @Override
+    public int getSessionTimeout() {
+        return sessionTimeout;
+    }
+
+    @Override
+    public void process(WatchedEvent event) {
+        ReplyHeader h = new ReplyHeader(-1, -1L, 0);
+        if (LOG.isTraceEnabled()) {
+            ZooTrace.logTraceMessage(LOG, ZooTrace.EVENT_DELIVERY_TRACE_MASK,
+                                     "Deliver event " + event + " to 0x"
+                                     + Long.toHexString(this.sessionId)
+                                     + " through " + this);
+        }
+
+        // Convert WatchedEvent to a type that can be sent over the wire
+        WatcherEvent e = event.getWrapper();
+
+        try {
+            sendResponse(h, e, "notification");
+        } catch (IOException e1) {
+            if (LOG.isDebugEnabled()) {
+                LOG.debug("Problem sending to " + getRemoteSocketAddress(), e1);
+            }
+            close();
+        }
+    }
+
+    private static final byte[] fourBytes = new byte[4];
+    static class ResumeMessageEvent implements MessageEvent {
+        Channel channel;
+        ResumeMessageEvent(Channel channel) {
+            this.channel = channel;
+        }
+        public Object getMessage() {return null;}
+        public SocketAddress getRemoteAddress() {return null;}
+        public Channel getChannel() {return channel;}
+        public ChannelFuture getFuture() {return null;}
+    };
+    
+    @Override
+    public void sendResponse(ReplyHeader h, Record r, String tag)
+            throws IOException {
+        if (!channel.isOpen()) {
+            return;
+        }
+        ByteArrayOutputStream baos = new ByteArrayOutputStream();
+        // Make space for length
+        BinaryOutputArchive bos = BinaryOutputArchive.getArchive(baos);
+        try {
+            baos.write(fourBytes);
+            bos.writeRecord(h, "header");
+            if (r != null) {
+                bos.writeRecord(r, tag);
+            }
+            baos.close();
+        } catch (IOException e) {
+            LOG.error("Error serializing response");
+        }
+        byte b[] = baos.toByteArray();
+        ByteBuffer bb = ByteBuffer.wrap(b);
+        bb.putInt(b.length - 4).rewind();
+        sendBuffer(bb);
+        if (h.getXid() > 0) {
+            // zks cannot be null otherwise we would not have gotten here!
+            if (!zkServer.shouldThrottle(outstandingCount.decrementAndGet())) {
+                enableRecv();
+            }
+        }
+    }
+
+    @Override
+    public void setSessionId(long sessionId) {
+        this.sessionId = sessionId;
+    }
+
+    @Override
+    public void enableRecv() {
+        if (throttled) {
+            throttled = false;
+            if (LOG.isDebugEnabled()) {
+                LOG.debug("Sending unthrottle event " + this);
+            }
+            channel.getPipeline().sendUpstream(new ResumeMessageEvent(channel));
+        }
+    }
+
+    @Override
+    public void sendBuffer(ByteBuffer sendBuffer) {
+        if (sendBuffer == ServerCnxnFactory.closeConn) {
+            channel.close();
+            return;
+        }
+        channel.write(wrappedBuffer(sendBuffer));
+        packetSent();
+    }
+
+    /**
+     * clean up the socket related to a command and also make sure we flush the
+     * data before we do that
+     * 
+     * @param pwriter
+     *            the pwriter for a command socket
+     */
+    private void cleanupWriterSocket(PrintWriter pwriter) {
+        try {
+            if (pwriter != null) {
+                pwriter.flush();
+                pwriter.close();
+            }
+        } catch (Exception e) {
+            LOG.info("Error closing PrintWriter ", e);
+        } finally {
+            try {
+                close();
+            } catch (Exception e) {
+                LOG.error("Error closing a command socket ", e);
+            }
+        }
+    }
+
+    /**
+     * This class wraps the sendBuffer method of NIOServerCnxn. It is
+     * responsible for chunking up the response to a client. Rather
+     * than cons'ing up a response fully in memory, which may be large
+     * for some commands, this class chunks up the result.
+     */
+    private class SendBufferWriter extends Writer {
+        private StringBuffer sb = new StringBuffer();
+        
+        /**
+         * Check if we are ready to send another chunk.
+         * @param force force sending, even if not a full chunk
+         */
+        private void checkFlush(boolean force) {
+            if ((force && sb.length() > 0) || sb.length() > 2048) {
+                sendBuffer(ByteBuffer.wrap(sb.toString().getBytes()));
+                // clear our internal buffer
+                sb.setLength(0);
+            }
+        }
+
+        @Override
+        public void close() throws IOException {
+            if (sb == null) return;
+            checkFlush(true);
+            sb = null; // clear out the ref to ensure no reuse
+        }
+
+        @Override
+        public void flush() throws IOException {
+            checkFlush(true);
+        }
+
+        @Override
+        public void write(char[] cbuf, int off, int len) throws IOException {
+            sb.append(cbuf, off, len);
+            checkFlush(false);
+        }
+    }
+
+    private static final String ZK_NOT_SERVING =
+        "This ZooKeeper instance is not currently serving requests";
+    
+    /**
+     * Set of threads for commmand ports. All the 4
+     * letter commands are run via a thread. Each class
+     * maps to a correspoding 4 letter command. CommandThread
+     * is the abstract class from which all the others inherit.
+     */
+    private abstract class CommandThread /*extends Thread*/ {
+        PrintWriter pw;
+        
+        CommandThread(PrintWriter pw) {
+            this.pw = pw;
+        }
+        
+        public void start() {
+            run();
+        }
+
+        public void run() {
+            try {
+                commandRun();
+            } catch (IOException ie) {
+                LOG.error("Error in running command ", ie);
+            } finally {
+                cleanupWriterSocket(pw);
+            }
+        }
+        
+        public abstract void commandRun() throws IOException;
+    }
+    
+    private class RuokCommand extends CommandThread {
+        public RuokCommand(PrintWriter pw) {
+            super(pw);
+        }
+        
+        @Override
+        public void commandRun() {
+            pw.print("imok");
+            
+        }
+    }
+    
+    private class TraceMaskCommand extends CommandThread {
+        TraceMaskCommand(PrintWriter pw) {
+            super(pw);
+        }
+        
+        @Override
+        public void commandRun() {
+            long traceMask = ZooTrace.getTextTraceLevel();
+            pw.print(traceMask);
+        }
+    }
+    
+    private class SetTraceMaskCommand extends CommandThread {
+        long trace = 0;
+        SetTraceMaskCommand(PrintWriter pw, long trace) {
+            super(pw);
+            this.trace = trace;
+        }
+        
+        @Override
+        public void commandRun() {
+            pw.print(trace);
+        }
+    }
+    
+    private class EnvCommand extends CommandThread {
+        EnvCommand(PrintWriter pw) {
+            super(pw);
+        }
+        
+        @Override
+        public void commandRun() {
+            List<Environment.Entry> env = Environment.list();
+
+            pw.println("Environment:");
+            for(Environment.Entry e : env) {
+                pw.print(e.getKey());
+                pw.print("=");
+                pw.println(e.getValue());
+            }
+            
+        } 
+    }
+    
+    private class ConfCommand extends CommandThread {
+        ConfCommand(PrintWriter pw) {
+            super(pw);
+        }
+            
+        @Override
+        public void commandRun() {
+            if (zkServer == null) {
+                pw.println(ZK_NOT_SERVING);
+            } else {
+                zkServer.dumpConf(pw);
+            }
+        }
+    }
+    
+    private class StatResetCommand extends CommandThread {
+        public StatResetCommand(PrintWriter pw) {
+            super(pw);
+        }
+        
+        @Override
+        public void commandRun() {
+            if (zkServer == null) {
+                pw.println(ZK_NOT_SERVING);
+            }
+            else { 
+                zkServer.serverStats().reset();
+                pw.println("Server stats reset.");
+            }
+        }
+    }
+    
+    private class CnxnStatResetCommand extends CommandThread {
+        public CnxnStatResetCommand(PrintWriter pw) {
+            super(pw);
+        }
+        
+        @Override
+        public void commandRun() {
+            if (zkServer == null) {
+                pw.println(ZK_NOT_SERVING);
+            } else {
+                synchronized(factory.cnxns){
+                    for(ServerCnxn c : factory.cnxns){
+                        c.resetStats();
+                    }
+                }
+                pw.println("Connection stats reset.");
+            }
+        }
+    }
+
+    private class DumpCommand extends CommandThread {
+        public DumpCommand(PrintWriter pw) {
+            super(pw);
+        }
+        
+        @Override
+        public void commandRun() {
+            if (zkServer == null) {
+                pw.println(ZK_NOT_SERVING);
+            }
+            else {
+                pw.println("SessionTracker dump:");
+                zkServer.sessionTracker.dumpSessions(pw);
+                pw.println("ephemeral nodes dump:");
+                zkServer.dumpEphemerals(pw);
+            }
+        }
+    }
+    
+    private class StatCommand extends CommandThread {
+        int len;
+        public StatCommand(PrintWriter pw, int len) {
+            super(pw);
+            this.len = len;
+        }
+        
+        @Override
+        public void commandRun() {
+            if (zkServer == null) {
+                pw.println(ZK_NOT_SERVING);
+            }
+            else {   
+                pw.print("Zookeeper version: ");
+                pw.println(Version.getFullVersion());
+                if (len == statCmd) {
+                    LOG.info("Stat command output");
+                    pw.println("Clients:");
+                    // clone should be faster than iteration
+                    // ie give up the cnxns lock faster
+                    HashSet<ServerCnxn> cnxns;
+                    synchronized(factory.cnxns){
+                        cnxns = new HashSet<ServerCnxn>(factory.cnxns);
+                    }
+                    for(ServerCnxn c : cnxns){
+                        c.dumpConnectionInfo(pw, true);
+                    }
+                    pw.println();
+                }
+                pw.print(zkServer.serverStats().toString());
+                pw.print("Node count: ");
+                pw.println(zkServer.getZKDatabase().getNodeCount());
+            }
+            
+        }
+    }
+    
+    private class ConsCommand extends CommandThread {
+        public ConsCommand(PrintWriter pw) {
+            super(pw);
+        }
+        
+        @Override
+        public void commandRun() {
+            if (zkServer == null) {
+                pw.println(ZK_NOT_SERVING);
+            } else {
+                // clone should be faster than iteration
+                // ie give up the cnxns lock faster
+                AbstractSet<ServerCnxn> cnxns;
+                synchronized (factory.cnxns) {
+                    cnxns = new HashSet<ServerCnxn>(factory.cnxns);
+                }
+                for (ServerCnxn c : cnxns) {
+                    c.dumpConnectionInfo(pw, false);
+                }
+                pw.println();
+            }
+        }
+    }
+    
+    private class WatchCommand extends CommandThread {
+        int len = 0;
+        public WatchCommand(PrintWriter pw, int len) {
+            super(pw);
+            this.len = len;
+        }
+
+        @Override
+        public void commandRun() {
+            if (zkServer == null) {
+                pw.println(ZK_NOT_SERVING);
+            } else {
+                DataTree dt = zkServer.getZKDatabase().getDataTree();
+                if (len == wchsCmd) {
+                    dt.dumpWatchesSummary(pw);
+                } else if (len == wchpCmd) {
+                    dt.dumpWatches(pw, true);
+                } else {
+                    dt.dumpWatches(pw, false);
+                }
+                pw.println();
+            }
+        }
+    }
+
+    private class MonitorCommand extends CommandThread {
+
+        MonitorCommand(PrintWriter pw) {
+            super(pw);
+        }
+
+        @Override
+        public void commandRun() {
+            if(zkServer == null) {
+                pw.println(ZK_NOT_SERVING);
+                return;
+            }
+            ZKDatabase zkdb = zkServer.getZKDatabase();
+            ServerStats stats = zkServer.serverStats();
+
+            print("version", Version.getFullVersion());
+
+            print("avg_latency", stats.getAvgLatency());
+            print("max_latency", stats.getMaxLatency());
+            print("min_latency", stats.getMinLatency());
+
+            print("packets_received", stats.getPacketsReceived());
+            print("packets_sent", stats.getPacketsSent());
+
+            print("outstanding_requests", stats.getOutstandingRequests());
+
+            print("server_state", stats.getServerState());
+            print("znode_count", zkdb.getNodeCount());
+
+            print("watch_count", zkdb.getDataTree().getWatchCount());
+            print("ephemerals_count", zkdb.getDataTree().getEphemeralsCount());
+            print("approximate_data_size", zkdb.getDataTree().approximateDataSize());
+
+            OperatingSystemMXBean osMbean = ManagementFactory.getOperatingSystemMXBean();
+            if(osMbean != null && osMbean instanceof UnixOperatingSystemMXBean) {
+                UnixOperatingSystemMXBean unixos = (UnixOperatingSystemMXBean)osMbean;
+
+                print("open_file_descriptor_count", unixos.getOpenFileDescriptorCount());
+                print("max_file_descriptor_count", unixos.getMaxFileDescriptorCount());
+            }
+
+            if(stats.getServerState() == "leader") {
+                Leader leader = ((LeaderZooKeeperServer)zkServer).getLeader();
+
+                print("followers", leader.learners.size());
+                print("synced_followers", leader.forwardingFollowers.size());
+                print("pending_syncs", leader.pendingSyncs.size());
+            }
+        }
+
+        private void print(String key, long number) {
+            print(key, "" + number);
+        }
+
+        private void print(String key, String value) {
+            pw.print("zk_");
+            pw.print(key);
+            pw.print("\t");
+            pw.println(value);
+        }
+
+    }
+
+
+    /** Return if four letter word found and responded to, otw false **/
+    private boolean checkFourLetterWord(final Channel channel,
+            ChannelBuffer message, final int len) throws IOException
+    {
+        // We take advantage of the limited size of the length to look
+        // for cmds. They are all 4-bytes which fits inside of an int
+        String cmd = cmd2String.get(len);
+        if (cmd == null) {
+            return false;
+        }
+        channel.setInterestOps(0).awaitUninterruptibly();
+        LOG.info("Processing " + cmd + " command from "
+                + channel.getRemoteAddress());
+        packetReceived();
+
+        final PrintWriter pwriter = new PrintWriter(
+                new BufferedWriter(new SendBufferWriter()));
+        if (len == ruokCmd) {
+            RuokCommand ruok = new RuokCommand(pwriter);
+            ruok.start();
+            return true;
+        } else if (len == getTraceMaskCmd) {
+            TraceMaskCommand tmask = new TraceMaskCommand(pwriter);
+            tmask.start();
+            return true;
+        } else if (len == setTraceMaskCmd) {
+            ByteBuffer mask = ByteBuffer.allocate(4);
+            message.readBytes(mask);
+
+            bb.flip();
+            long traceMask = mask.getLong();
+            ZooTrace.setTextTraceLevel(traceMask);
+            SetTraceMaskCommand setMask = new SetTraceMaskCommand(pwriter, traceMask);
+            setMask.start();
+            return true;
+        } else if (len == enviCmd) {
+            EnvCommand env = new EnvCommand(pwriter);
+            env.start();
+            return true;
+        } else if (len == confCmd) {
+            ConfCommand ccmd = new ConfCommand(pwriter);
+            ccmd.start();
+            return true;
+        } else if (len == srstCmd) {
+            StatResetCommand strst = new StatResetCommand(pwriter);
+            strst.start();
+            return true;
+        } else if (len == crstCmd) {
+            CnxnStatResetCommand crst = new CnxnStatResetCommand(pwriter);
+            crst.start();
+            return true;
+        } else if (len == dumpCmd) {
+            DumpCommand dump = new DumpCommand(pwriter);
+            dump.start();
+            return true;
+        } else if (len == statCmd || len == srvrCmd) {
+            StatCommand stat = new StatCommand(pwriter, len);
+            stat.start();
+            return true;
+        } else if (len == consCmd) {
+            ConsCommand cons = new ConsCommand(pwriter);
+            cons.start();
+            return true;
+        } else if (len == wchpCmd || len == wchcCmd || len == wchsCmd) {
+            WatchCommand wcmd = new WatchCommand(pwriter, len);
+            wcmd.start();
+            return true;
+        } else if (len == mntrCmd) {
+            MonitorCommand mntr = new MonitorCommand(pwriter);
+            mntr.start();
+            return true;
+        }
+        return false;
+    }
+
+    public void receiveMessage(ChannelBuffer message) {
+        try {
+            while(message.readable() && !throttled) {
+                if (bb != null) {
+                    if (LOG.isTraceEnabled()) {
+                        LOG.trace("message readable " + message.readableBytes()
+                                + " bb len " + bb.remaining() + " " + bb);
+                        ByteBuffer dat = bb.duplicate();
+                        dat.flip();
+                        LOG.trace(Long.toHexString(sessionId)
+                                + " bb 0x"
+                                + ChannelBuffers.hexDump(
+                                        ChannelBuffers.copiedBuffer(dat)));
+                    }
+
+                    if (bb.remaining() > message.readableBytes()) {
+                        int newLimit = bb.position() + message.readableBytes();
+                        bb.limit(newLimit);
+                    }
+                    message.readBytes(bb);
+                    bb.limit(bb.capacity());
+
+                    if (LOG.isTraceEnabled()) {
+                        LOG.trace("after readBytes message readable "
+                                + message.readableBytes()
+                                + " bb len " + bb.remaining() + " " + bb);
+                        ByteBuffer dat = bb.duplicate();
+                        dat.flip();
+                        LOG.trace("after readbytes "
+                                + Long.toHexString(sessionId)
+                                + " bb 0x"
+                                + ChannelBuffers.hexDump(
+                                        ChannelBuffers.copiedBuffer(dat)));
+                    }
+                    if (bb.remaining() == 0) {
+                        packetReceived();
+                        bb.flip();
+
+                        ZooKeeperServer zks = this.zkServer;
+                        if (zks == null) {
+                            throw new IOException("ZK down");
+                        }
+                        if (initialized) {
+                            zks.processPacket(this, bb);
+
+                            if (zks.shouldThrottle(outstandingCount.incrementAndGet())) {
+                                disableRecv();
+                            }
+                        } else {
+                            LOG.debug("got conn req request from "
+                                    + getRemoteSocketAddress());
+                            zks.processConnectRequest(this, bb);
+                            initialized = true;
+                        }
+                        bb = null;
+                    }
+                } else {
+                    if (LOG.isTraceEnabled()) {
+                        LOG.trace("message readable "
+                                + message.readableBytes()
+                                + " bblenrem " + bbLen.remaining());
+                        ByteBuffer dat = bbLen.duplicate();
+                        dat.flip();
+                        LOG.trace(Long.toHexString(sessionId)
+                                + " bbLen 0x"
+                                + ChannelBuffers.hexDump(
+                                        ChannelBuffers.copiedBuffer(dat)));
+                    }
+
+                    if (message.readableBytes() < bbLen.remaining()) {
+                        bbLen.limit(bbLen.position() + message.readableBytes());
+                    }
+                    message.readBytes(bbLen);
+                    bbLen.limit(bbLen.capacity());
+                    if (bbLen.remaining() == 0) {
+                        bbLen.flip();
+
+                        if (LOG.isTraceEnabled()) {
+                            LOG.trace(Long.toHexString(sessionId)
+                                    + " bbLen 0x"
+                                    + ChannelBuffers.hexDump(
+                                            ChannelBuffers.copiedBuffer(bbLen)));
+                        }
+                        int len = bbLen.getInt();
+                        if (LOG.isTraceEnabled()) {
+                            LOG.trace(Long.toHexString(sessionId)
+                                    + " bbLen len is " + len);
+                        }
+
+                        bbLen.clear();
+                        if (!initialized) {
+                            if (checkFourLetterWord(channel, message, len)) {
+                                return;
+                            }
+                        }
+                        if (len < 0 || len > BinaryInputArchive.maxBuffer) {
+                            throw new IOException("Len error " + len);
+                        }
+                        bb = ByteBuffer.allocate(len);
+                    }
+                }
+            }
+        } catch(IOException e) {
+            LOG.warn("Closing connection to " + getRemoteSocketAddress(), e);
+            close();
+        }
+    }
+
+    @Override
+    public void disableRecv() {
+        throttled = true;
+        if (LOG.isDebugEnabled()) {
+            LOG.debug("Throttling - disabling recv " + this);
+        }
+        channel.setReadable(false).awaitUninterruptibly();
+    }
+
+    @Override
+    public long getOutstandingRequests() {
+        return outstandingCount.longValue();
+    }
+
+    @Override
+    public void setSessionTimeout(int sessionTimeout) {
+        this.sessionTimeout = sessionTimeout;
+    }
+
+    @Override
+    public int getInterestOps() {
+        return channel.getInterestOps();
+    }
+
+    @Override
+    public InetSocketAddress getRemoteSocketAddress() {
+        return (InetSocketAddress)channel.getRemoteAddress();
+    }
+
+    /** Send close connection packet to the client.
+     */
+    public void sendCloseSession() {
+        sendBuffer(ServerCnxnFactory.closeConn);
+    }
+
+    @Override
+    protected ServerStats serverStats() {
+        if (zkServer == null) {
+            return null;
+        }
+        return zkServer.serverStats();
+    }
+
+}

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

@@ -0,0 +1,396 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.zookeeper.server;
+
+import static org.jboss.netty.buffer.ChannelBuffers.dynamicBuffer;
+
+import java.io.IOException;
+import java.net.InetAddress;
+import java.net.InetSocketAddress;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.concurrent.Executors;
+
+import org.apache.log4j.Logger;
+import org.jboss.netty.bootstrap.ServerBootstrap;
+import org.jboss.netty.buffer.ChannelBuffer;
+import org.jboss.netty.buffer.ChannelBuffers;
+import org.jboss.netty.channel.Channel;
+import org.jboss.netty.channel.ChannelHandlerContext;
+import org.jboss.netty.channel.ChannelPipelineCoverage;
+import org.jboss.netty.channel.ChannelStateEvent;
+import org.jboss.netty.channel.ExceptionEvent;
+import org.jboss.netty.channel.MessageEvent;
+import org.jboss.netty.channel.SimpleChannelHandler;
+import org.jboss.netty.channel.WriteCompletionEvent;
+import org.jboss.netty.channel.group.ChannelGroup;
+import org.jboss.netty.channel.group.DefaultChannelGroup;
+import org.jboss.netty.channel.socket.nio.NioServerSocketChannelFactory;
+
+public class NettyServerCnxnFactory extends ServerCnxnFactory {
+    Logger LOG = Logger.getLogger(NettyServerCnxnFactory.class);
+
+    ServerBootstrap bootstrap;
+    Channel parentChannel;
+    ChannelGroup allChannels = new DefaultChannelGroup("zkServerCnxns");
+    HashSet<ServerCnxn> cnxns = new HashSet<ServerCnxn>();
+    HashMap<InetAddress, Set<NettyServerCnxn>> ipMap =
+        new HashMap<InetAddress, Set<NettyServerCnxn>>( );
+    InetSocketAddress localAddress;
+    int maxClientCnxns = 10;
+    
+    /**
+     * This is an inner class since we need to extend SimpleChannelHandler, but
+     * NettyServerCnxnFactory already extends ServerCnxnFactory. By making it inner
+     * this class gets access to the member variables and methods.
+     */
+    @ChannelPipelineCoverage("all")
+    class CnxnChannelHandler extends SimpleChannelHandler {
+
+        @Override
+        public void channelClosed(ChannelHandlerContext ctx, ChannelStateEvent e)
+            throws Exception
+        {
+            if (LOG.isTraceEnabled()) {
+                LOG.trace("Channel closed " + e);
+            }
+            allChannels.remove(ctx.getChannel());
+        }
+
+        @Override
+        public void channelConnected(ChannelHandlerContext ctx,
+                ChannelStateEvent e) throws Exception
+        {
+            if (LOG.isTraceEnabled()) {
+                LOG.trace("Channel connected " + e);
+            }
+            allChannels.add(ctx.getChannel());
+            NettyServerCnxn cnxn = new NettyServerCnxn(ctx.getChannel(),
+                    zkServer, NettyServerCnxnFactory.this);
+            ctx.setAttachment(cnxn);
+            addCnxn(cnxn);
+        }
+
+        @Override
+        public void channelDisconnected(ChannelHandlerContext ctx,
+                ChannelStateEvent e) throws Exception
+        {
+            if (LOG.isTraceEnabled()) {
+                LOG.trace("Channel disconnected " + e);
+            }
+            NettyServerCnxn cnxn = (NettyServerCnxn) ctx.getAttachment();
+            if (cnxn != null) {
+                if (LOG.isTraceEnabled()) {
+                    LOG.trace("Channel disconnect caused close " + e);
+                }
+                cnxn.close();
+            }
+        }
+
+        @Override
+        public void exceptionCaught(ChannelHandlerContext ctx, ExceptionEvent e)
+            throws Exception
+        {
+            LOG.warn("Exception caught " + e, e.getCause());
+            NettyServerCnxn cnxn = (NettyServerCnxn) ctx.getAttachment();
+            if (cnxn != null) {
+                if (LOG.isDebugEnabled()) {
+                    LOG.debug("Closing " + cnxn);
+                    cnxn.close();
+                }
+            }
+        }
+
+        @Override
+        public void messageReceived(ChannelHandlerContext ctx, MessageEvent e)
+            throws Exception
+        {
+            if (LOG.isTraceEnabled()) {
+                LOG.trace("message received called " + e.getMessage());
+            }
+            try {
+                if (LOG.isDebugEnabled()) {
+                    LOG.debug("New message " + e.toString()
+                            + " from " + ctx.getChannel());
+                }
+                NettyServerCnxn cnxn = (NettyServerCnxn)ctx.getAttachment();
+                synchronized(cnxn) {
+                    processMessage(e, cnxn);
+                }
+            } catch(Exception ex) {
+                LOG.error("Unexpected exception in receive", ex);
+                throw ex;
+            }
+        }
+
+        private void processMessage(MessageEvent e, NettyServerCnxn cnxn) {
+            if (LOG.isDebugEnabled()) {
+                LOG.debug(Long.toHexString(cnxn.sessionId) + " queuedBuffer: "
+                        + cnxn.queuedBuffer);
+            }
+
+            if (e instanceof NettyServerCnxn.ResumeMessageEvent) {
+                LOG.debug("Received ResumeMessageEvent");
+                if (cnxn.queuedBuffer != null) {
+                    if (LOG.isTraceEnabled()) {
+                        LOG.trace("processing queue "
+                                + Long.toHexString(cnxn.sessionId)
+                                + " queuedBuffer 0x"
+                                + ChannelBuffers.hexDump(cnxn.queuedBuffer));
+                    }
+                    cnxn.receiveMessage(cnxn.queuedBuffer);
+                    if (!cnxn.queuedBuffer.readable()) {
+                        LOG.debug("Processed queue - no bytes remaining");
+                        cnxn.queuedBuffer = null;
+                    } else {
+                        LOG.debug("Processed queue - bytes remaining");
+                    }
+                } else {
+                    LOG.debug("queue empty");
+                }
+                cnxn.channel.setReadable(true);
+            } else {
+                ChannelBuffer buf = (ChannelBuffer)e.getMessage();
+                if (LOG.isTraceEnabled()) {
+                    LOG.trace(Long.toHexString(cnxn.sessionId)
+                            + " buf 0x"
+                            + ChannelBuffers.hexDump(buf));
+                }
+                
+                if (cnxn.throttled) {
+                    LOG.debug("Received message while throttled");
+                    // we are throttled, so we need to queue
+                    if (cnxn.queuedBuffer == null) {
+                        LOG.debug("allocating queue");
+                        cnxn.queuedBuffer = dynamicBuffer(buf.readableBytes());
+                    }
+                    cnxn.queuedBuffer.writeBytes(buf);
+                    LOG.debug(Long.toHexString(cnxn.sessionId)
+                            + " queuedBuffer 0x"
+                            + ChannelBuffers.hexDump(cnxn.queuedBuffer));
+                } else {
+                    LOG.debug("not throttled");
+                    if (cnxn.queuedBuffer != null) {
+                        if (LOG.isTraceEnabled()) {
+                            LOG.trace(Long.toHexString(cnxn.sessionId)
+                                    + " queuedBuffer 0x"
+                                    + ChannelBuffers.hexDump(cnxn.queuedBuffer));
+                        }
+                        cnxn.queuedBuffer.writeBytes(buf);
+                        if (LOG.isTraceEnabled()) {
+                            LOG.trace(Long.toHexString(cnxn.sessionId)
+                                    + " queuedBuffer 0x"
+                                    + ChannelBuffers.hexDump(cnxn.queuedBuffer));
+                        }
+
+                        cnxn.receiveMessage(cnxn.queuedBuffer);
+                        if (!cnxn.queuedBuffer.readable()) {
+                            LOG.debug("Processed queue - no bytes remaining");
+                            cnxn.queuedBuffer = null;
+                        } else {
+                            LOG.debug("Processed queue - bytes remaining");
+                        }
+                    } else {
+                        cnxn.receiveMessage(buf);
+                        if (buf.readable()) {
+                            if (LOG.isTraceEnabled()) {
+                                LOG.trace("Before copy " + buf);
+                            }
+                            cnxn.queuedBuffer = dynamicBuffer(buf.readableBytes()); 
+                            cnxn.queuedBuffer.writeBytes(buf);
+                            if (LOG.isTraceEnabled()) {
+                                LOG.trace("Copy is " + cnxn.queuedBuffer);
+                                LOG.trace(Long.toHexString(cnxn.sessionId)
+                                        + " queuedBuffer 0x"
+                                        + ChannelBuffers.hexDump(cnxn.queuedBuffer));
+                            }
+                        }
+                    }
+                }
+            }
+        }
+
+        @Override
+        public void writeComplete(ChannelHandlerContext ctx,
+                WriteCompletionEvent e) throws Exception
+        {
+            if (LOG.isTraceEnabled()) {
+                LOG.trace("write complete " + e);
+            }
+        }
+        
+    }
+    
+    CnxnChannelHandler channelHandler = new CnxnChannelHandler();
+    
+    NettyServerCnxnFactory() {
+        bootstrap = new ServerBootstrap(
+                new NioServerSocketChannelFactory(
+                        Executors.newCachedThreadPool(),
+                        Executors.newCachedThreadPool()));
+        // parent channel
+        bootstrap.setOption("reuseAddress", true);
+        // child channels
+        bootstrap.setOption("child.tcpNoDelay", true);
+        bootstrap.setOption("child.soLinger", 2);
+
+        bootstrap.getPipeline().addLast("servercnxnfactory", channelHandler);
+    }
+    
+    @Override
+    public void closeAll() {
+        if (LOG.isDebugEnabled()) {
+            LOG.debug("closeAll()");
+        }
+
+        synchronized (cnxns) {
+            // got to clear all the connections that we have in the selector
+            for (NettyServerCnxn cnxn : cnxns.toArray(new NettyServerCnxn[cnxns.size()])) {
+                try {
+                    cnxn.close();
+                } catch (Exception e) {
+                    LOG.warn("Ignoring exception closing cnxn sessionid 0x"
+                            + Long.toHexString(cnxn.getSessionId()), e);
+                }
+            }
+        }
+        if (LOG.isDebugEnabled()) {
+            LOG.debug("allChannels size:" + allChannels.size()
+                    + " cnxns size:" + cnxns.size());
+        }
+    }
+
+    @Override
+    public void closeSession(long sessionId) {
+        if (LOG.isDebugEnabled()) {
+            LOG.debug("closeSession sessionid:0x" + sessionId);
+        }
+
+        synchronized (cnxns) {
+            for (NettyServerCnxn cnxn : cnxns.toArray(new NettyServerCnxn[cnxns.size()])) {
+                if (cnxn.getSessionId() == sessionId) {
+                    try {
+                        cnxn.close();
+                    } catch (Exception e) {
+                        LOG.warn("exception during session close", e);
+                    }
+                    break;
+                }
+            }
+        }
+    }
+
+    @Override
+    public void configure(InetSocketAddress addr, int maxClientCnxns)
+            throws IOException
+    {
+        localAddress = addr;
+        this.maxClientCnxns = maxClientCnxns;
+    }
+
+    /** {@inheritDoc} */
+    public int getMaxClientCnxnsPerHost() {
+        return maxClientCnxns;
+    }
+
+    /** {@inheritDoc} */
+    public void setMaxClientCnxnsPerHost(int max) {
+        maxClientCnxns = max;
+    }
+
+    @Override
+    public int getLocalPort() {
+        return localAddress.getPort();
+    }
+
+    boolean killed;
+    @Override
+    public void join() throws InterruptedException {
+        synchronized(this) {
+            while(!killed) {
+                wait();
+            }
+        }
+    }
+
+    @Override
+    public void shutdown() {
+        LOG.info("shutdown called " + localAddress);
+        
+        // null if factory never started
+        if (parentChannel != null) {
+            parentChannel.close().awaitUninterruptibly();
+            closeAll();
+            allChannels.close().awaitUninterruptibly();
+            bootstrap.releaseExternalResources();
+        }
+
+        if (zkServer != null) {
+            zkServer.shutdown();
+        }
+        synchronized(this) {
+            killed = true;
+            notifyAll();
+        }
+    }
+    
+    @Override
+    public void start() {
+        LOG.info("binding to port " + localAddress);
+        parentChannel = bootstrap.bind(localAddress);
+    }
+
+    @Override
+    public void startup(ZooKeeperServer zks) throws IOException,
+            InterruptedException {
+        start();
+        zks.startdata();
+        zks.startup();
+        setZooKeeperServer(zks);
+    }
+
+    @Override
+    public Iterable<ServerCnxn> getConnections() {
+        return cnxns;
+    }
+
+    @Override
+    public InetSocketAddress getLocalAddress() {
+        return localAddress;
+    }
+
+    private void addCnxn(NettyServerCnxn cnxn) {
+        synchronized (cnxns) {
+            cnxns.add(cnxn);
+            synchronized (ipMap){
+                InetAddress addr =
+                    ((InetSocketAddress)cnxn.channel.getRemoteAddress())
+                        .getAddress();
+                Set<NettyServerCnxn> s = ipMap.get(addr);
+                if (s == null) {
+                    s = new HashSet<NettyServerCnxn>();
+                }
+                s.add(cnxn);
+                ipMap.put(addr,s);
+            }
+        }
+    }
+
+}

+ 3 - 1
src/java/main/org/apache/zookeeper/server/PrepRequestProcessor.java

@@ -84,7 +84,8 @@ public class PrepRequestProcessor extends Thread implements RequestProcessor {
 
 
     public PrepRequestProcessor(ZooKeeperServer zks,
     public PrepRequestProcessor(ZooKeeperServer zks,
             RequestProcessor nextProcessor) {
             RequestProcessor nextProcessor) {
-        super("ProcessThread:" + zks.getClientPort());
+        super("ProcessThread(sid:" + zks.getServerId()
+                + " cport:" + zks.getClientPort() + "):");
         this.nextProcessor = nextProcessor;
         this.nextProcessor = nextProcessor;
         this.zks = zks;
         this.zks = zks;
     }
     }
@@ -505,6 +506,7 @@ public class PrepRequestProcessor extends Thread implements RequestProcessor {
     }
     }
 
 
     public void shutdown() {
     public void shutdown() {
+        LOG.info("Shutting down");
         submittedRequests.clear();
         submittedRequests.clear();
         submittedRequests.add(Request.requestOfDeath);
         submittedRequests.add(Request.requestOfDeath);
         nextProcessor.shutdown();
         nextProcessor.shutdown();

+ 381 - 57
src/java/main/org/apache/zookeeper/server/ServerCnxn.java

@@ -19,90 +19,414 @@
 package org.apache.zookeeper.server;
 package org.apache.zookeeper.server;
 
 
 import java.io.IOException;
 import java.io.IOException;
+import java.io.PrintWriter;
+import java.io.StringWriter;
 import java.net.InetSocketAddress;
 import java.net.InetSocketAddress;
+import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.ArrayList;
+import java.util.Collections;
 import java.util.Date;
 import java.util.Date;
+import java.util.HashMap;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicLong;
 
 
 import org.apache.jute.Record;
 import org.apache.jute.Record;
 import org.apache.zookeeper.WatchedEvent;
 import org.apache.zookeeper.WatchedEvent;
 import org.apache.zookeeper.Watcher;
 import org.apache.zookeeper.Watcher;
 import org.apache.zookeeper.data.Id;
 import org.apache.zookeeper.data.Id;
 import org.apache.zookeeper.proto.ReplyHeader;
 import org.apache.zookeeper.proto.ReplyHeader;
+import org.apache.zookeeper.proto.RequestHeader;
 
 
 /**
 /**
  * Interface to a Server connection - represents a connection from a client
  * Interface to a Server connection - represents a connection from a client
  * to the server.
  * to the server.
  */
  */
-public interface ServerCnxn extends Watcher {
+public abstract class ServerCnxn implements Stats, Watcher {
     // This is just an arbitrary object to represent requests issued by
     // This is just an arbitrary object to represent requests issued by
     // (aka owned by) this class
     // (aka owned by) this class
     final public static Object me = new Object();
     final public static Object me = new Object();
+    
+    protected ArrayList<Id> authInfo = new ArrayList<Id>();
 
 
-    int getSessionTimeout();
+    abstract int getSessionTimeout();
 
 
-    void sendResponse(ReplyHeader h, Record r, String tag) throws IOException;
+    abstract void close();
+
+    abstract void sendResponse(ReplyHeader h, Record r, String tag)
+        throws IOException;
 
 
     /* notify the client the session is closing and close/cleanup socket */
     /* notify the client the session is closing and close/cleanup socket */
-    void sendCloseSession();
+    abstract void sendCloseSession();
+
+    public abstract void process(WatchedEvent event);
+
+    abstract long getSessionId();
+
+    abstract void setSessionId(long sessionId);
+
+    /** auth info for the cnxn, returns an unmodifyable list */
+    public List<Id> getAuthInfo() {
+        return Collections.unmodifiableList(authInfo);
+    }
+
+    public void addAuthInfo(Id id) {
+        authInfo.add(id);
+    }
+
+    public boolean removeAuthInfo(Id id) {
+        return authInfo.remove(id);
+    }
+
+    abstract void sendBuffer(ByteBuffer closeConn);
+
+    abstract void enableRecv();
+
+    abstract void disableRecv();
+
+    abstract void setSessionTimeout(int sessionTimeout);
+
+    protected static class CloseRequestException extends IOException {
+        private static final long serialVersionUID = -7854505709816442681L;
+
+        public CloseRequestException(String msg) {
+            super(msg);
+        }
+    }
+
+    protected static class EndOfStreamException extends IOException {
+        private static final long serialVersionUID = -8255690282104294178L;
+
+        public EndOfStreamException(String msg) {
+            super(msg);
+        }
+
+        public String toString() {
+            return "EndOfStreamException: " + getMessage();
+        }
+    }
+
+    /*
+     * See <a href="{@docRoot}/../../../docs/zookeeperAdmin.html#sc_zkCommands">
+     * Zk Admin</a>. this link is for all the commands.
+     */
+    protected final static int confCmd =
+        ByteBuffer.wrap("conf".getBytes()).getInt();
+
+    /*
+     * See <a href="{@docRoot}/../../../docs/zookeeperAdmin.html#sc_zkCommands">
+     * Zk Admin</a>. this link is for all the commands.
+     */
+    protected final static int consCmd =
+        ByteBuffer.wrap("cons".getBytes()).getInt();
+
+    /*
+     * See <a href="{@docRoot}/../../../docs/zookeeperAdmin.html#sc_zkCommands">
+     * Zk Admin</a>. this link is for all the commands.
+     */
+    protected final static int crstCmd =
+        ByteBuffer.wrap("crst".getBytes()).getInt();
+
+    /*
+     * See <a href="{@docRoot}/../../../docs/zookeeperAdmin.html#sc_zkCommands">
+     * Zk Admin</a>. this link is for all the commands.
+     */
+    protected final static int dumpCmd =
+        ByteBuffer.wrap("dump".getBytes()).getInt();
+
+    /*
+     * See <a href="{@docRoot}/../../../docs/zookeeperAdmin.html#sc_zkCommands">
+     * Zk Admin</a>. this link is for all the commands.
+     */
+    protected final static int enviCmd =
+        ByteBuffer.wrap("envi".getBytes()).getInt();
+
+    /*
+     * See <a href="{@docRoot}/../../../docs/zookeeperAdmin.html#sc_zkCommands">
+     * Zk Admin</a>. this link is for all the commands.
+     */
+    protected final static int getTraceMaskCmd =
+        ByteBuffer.wrap("gtmk".getBytes()).getInt();
+
+    /*
+     * See <a href="{@docRoot}/../../../docs/zookeeperAdmin.html#sc_zkCommands">
+     * Zk Admin</a>. this link is for all the commands.
+     */
+    protected final static int ruokCmd =
+        ByteBuffer.wrap("ruok".getBytes()).getInt();
+    /*
+     * See <a href="{@docRoot}/../../../docs/zookeeperAdmin.html#sc_zkCommands">
+     * Zk Admin</a>. this link is for all the commands.
+     */
+    protected final static int setTraceMaskCmd =
+        ByteBuffer.wrap("stmk".getBytes()).getInt();
+
+    /*
+     * See <a href="{@docRoot}/../../../docs/zookeeperAdmin.html#sc_zkCommands">
+     * Zk Admin</a>. this link is for all the commands.
+     */
+    protected final static int srvrCmd =
+        ByteBuffer.wrap("srvr".getBytes()).getInt();
+
+    /*
+     * See <a href="{@docRoot}/../../../docs/zookeeperAdmin.html#sc_zkCommands">
+     * Zk Admin</a>. this link is for all the commands.
+     */
+    protected final static int srstCmd =
+        ByteBuffer.wrap("srst".getBytes()).getInt();
+
+    /*
+     * See <a href="{@docRoot}/../../../docs/zookeeperAdmin.html#sc_zkCommands">
+     * Zk Admin</a>. this link is for all the commands.
+     */
+    protected final static int statCmd =
+        ByteBuffer.wrap("stat".getBytes()).getInt();
+
+    /*
+     * See <a href="{@docRoot}/../../../docs/zookeeperAdmin.html#sc_zkCommands">
+     * Zk Admin</a>. this link is for all the commands.
+     */
+    protected final static int wchcCmd =
+        ByteBuffer.wrap("wchc".getBytes()).getInt();
+
+    /*
+     * See <a href="{@docRoot}/../../../docs/zookeeperAdmin.html#sc_zkCommands">
+     * Zk Admin</a>. this link is for all the commands.
+     */
+    protected final static int wchpCmd =
+        ByteBuffer.wrap("wchp".getBytes()).getInt();
+
+    /*
+     * See <a href="{@docRoot}/../../../docs/zookeeperAdmin.html#sc_zkCommands">
+     * Zk Admin</a>. this link is for all the commands.
+     */
+    protected final static int wchsCmd =
+        ByteBuffer.wrap("wchs".getBytes()).getInt();
 
 
-    void finishSessionInit(boolean valid);
+    /*
+     * See <a href="{@docRoot}/../../../docs/zookeeperAdmin.html#sc_zkCommands">
+     * Zk Admin</a>. this link is for all the commands.
+     */
+    protected final static int mntrCmd = ByteBuffer.wrap("mntr".getBytes())
+            .getInt();
+
+    protected final static HashMap<Integer, String> cmd2String =
+        new HashMap<Integer, String>();
+
+    // specify all of the commands that are available
+    static {
+        cmd2String.put(confCmd, "conf");
+        cmd2String.put(consCmd, "cons");
+        cmd2String.put(crstCmd, "crst");
+        cmd2String.put(dumpCmd, "dump");
+        cmd2String.put(enviCmd, "envi");
+        cmd2String.put(getTraceMaskCmd, "gtmk");
+        cmd2String.put(ruokCmd, "ruok");
+        cmd2String.put(setTraceMaskCmd, "stmk");
+        cmd2String.put(srstCmd, "srst");
+        cmd2String.put(srvrCmd, "srvr");
+        cmd2String.put(statCmd, "stat");
+        cmd2String.put(wchcCmd, "wchc");
+        cmd2String.put(wchpCmd, "wchp");
+        cmd2String.put(wchsCmd, "wchs");
+        cmd2String.put(mntrCmd, "mntr");
+    }
+
+    protected void packetReceived() {
+        incrPacketsReceived();
+        ServerStats serverStats = serverStats();
+        if (serverStats != null) {
+            serverStats().incrementPacketsReceived();
+        }
+    }
+
+    protected void packetSent() {
+        incrPacketsSent();
+        ServerStats serverStats = serverStats();
+        if (serverStats != null) {
+            serverStats().incrementPacketsSent();
+        }
+    }
+
+    protected abstract ServerStats serverStats();
+    
+    protected final Date established = new Date();
+
+    protected final AtomicLong packetsReceived = new AtomicLong();
+    protected final AtomicLong packetsSent = new AtomicLong();
+
+    protected long minLatency;
+    protected long maxLatency;
+    protected String lastOp;
+    protected long lastCxid;
+    protected long lastZxid;
+    protected long lastResponseTime;
+    protected long lastLatency;
+
+    protected long count;
+    protected long totalLatency;
+
+    public synchronized void resetStats() {
+        packetsReceived.set(0);
+        packetsSent.set(0);
+        minLatency = Long.MAX_VALUE;
+        maxLatency = 0;
+        lastOp = "NA";
+        lastCxid = -1;
+        lastZxid = -1;
+        lastResponseTime = 0;
+        lastLatency = 0;
+
+        count = 0;
+        totalLatency = 0;
+    }
+
+    protected long incrPacketsReceived() {
+        return packetsReceived.incrementAndGet();
+    }
+    
+    protected void incrOutstandingRequests(RequestHeader h) {
+    }
+
+    protected long incrPacketsSent() {
+        return packetsSent.incrementAndGet();
+    }
+
+    protected synchronized void updateStatsForResponse(long cxid, long zxid,
+            String op, long start, long end)
+    {
+        // don't overwrite with "special" xids - we're interested
+        // in the clients last real operation
+        if (cxid >= 0) {
+            lastCxid = cxid;
+        }
+        lastZxid = zxid;
+        lastOp = op;
+        lastResponseTime = end;
+        long elapsed = end - start;
+        lastLatency = elapsed;
+        if (elapsed < minLatency) {
+            minLatency = elapsed;
+        }
+        if (elapsed > maxLatency) {
+            maxLatency = elapsed;
+        }
+        count++;
+        totalLatency += elapsed;
+    }
+
+    public Date getEstablished() {
+        return (Date)established.clone();
+    }
 
 
-    void process(WatchedEvent event);
+    public abstract long getOutstandingRequests();
 
 
-    long getSessionId();
+    public long getPacketsReceived() {
+        return packetsReceived.longValue();
+    }
+
+    public long getPacketsSent() {
+        return packetsSent.longValue();
+    }
+
+    public synchronized long getMinLatency() {
+        return minLatency == Long.MAX_VALUE ? 0 : minLatency;
+    }
+
+    public synchronized long getAvgLatency() {
+        return count == 0 ? 0 : totalLatency / count;
+    }
+
+    public synchronized long getMaxLatency() {
+        return maxLatency;
+    }
+
+    public synchronized String getLastOperation() {
+        return lastOp;
+    }
+
+    public synchronized long getLastCxid() {
+        return lastCxid;
+    }
+
+    public synchronized long getLastZxid() {
+        return lastZxid;
+    }
 
 
-    void setSessionId(long sessionId);
+    public synchronized long getLastResponseTime() {
+        return lastResponseTime;
+    }
 
 
-    ArrayList<Id> getAuthInfo();
+    public synchronized long getLastLatency() {
+        return lastLatency;
+    }
 
 
-    InetSocketAddress getRemoteAddress();
+    /**
+     * Prints detailed stats information for the connection.
+     *
+     * @see dumpConnectionInfo(PrintWriter, boolean) for brief stats
+     */
+    @Override
+    public String toString() {
+        StringWriter sw = new StringWriter();
+        PrintWriter pwriter = new PrintWriter(sw);
+        dumpConnectionInfo(pwriter, false);
+        pwriter.flush();
+        pwriter.close();
+        return sw.toString();
+    }
 
 
+    public abstract InetSocketAddress getRemoteSocketAddress();
+    public abstract int getInterestOps();
+    
     /**
     /**
-     * Statistics on the ServerCnxn
-     */
-    interface Stats {
-        /** Date/time the connection was established
-         * @since 3.3.0 */
-        Date getEstablished();
-
-        /**
-         * The number of requests that have been submitted but not yet
-         * responded to.
-         */
-        long getOutstandingRequests();
-        /** Number of packets received */
-        long getPacketsReceived();
-        /** Number of packets sent (incl notifications) */
-        long getPacketsSent();
-        /** Min latency in ms
-         * @since 3.3.0 */
-        long getMinLatency();
-        /** Average latency in ms
-         * @since 3.3.0 */
-        long getAvgLatency();
-        /** Max latency in ms
-         * @since 3.3.0 */
-        long getMaxLatency();
-        /** Last operation performed by this connection
-         * @since 3.3.0 */
-        String getLastOperation();
-        /** Last cxid of this connection
-         * @since 3.3.0 */
-        long getLastCxid();
-        /** Last zxid of this connection
-         * @since 3.3.0 */
-        long getLastZxid();
-        /** Last time server sent a response to client on this connection
-         * @since 3.3.0 */
-        long getLastResponseTime();
-        /** Latency of last response to client on this connection in ms
-         * @since 3.3.0 */
-        long getLastLatency();
-
-        /** Reset counters
-         * @since 3.3.0 */
-        void reset();
-    }
-
-    Stats getStats();
+     * Print information about the connection.
+     * @param brief iff true prints brief details, otw full detail
+     * @return information about this connection
+     */
+    protected synchronized void
+    dumpConnectionInfo(PrintWriter pwriter, boolean brief) {
+        pwriter.print(" ");
+        pwriter.print(getRemoteSocketAddress());
+        pwriter.print("[");
+        int interestOps = getInterestOps();
+        pwriter.print(interestOps == 0 ? "0" : Integer.toHexString(interestOps));
+        pwriter.print("](queued=");
+        pwriter.print(getOutstandingRequests());
+        pwriter.print(",recved=");
+        pwriter.print(getPacketsReceived());
+        pwriter.print(",sent=");
+        pwriter.print(getPacketsSent());
+
+        if (!brief) {
+            long sessionId = getSessionId();
+            if (sessionId != 0) {
+                pwriter.print(",sid=0x");
+                pwriter.print(Long.toHexString(sessionId));
+                pwriter.print(",lop=");
+                pwriter.print(getLastOperation());
+                pwriter.print(",est=");
+                pwriter.print(getEstablished().getTime());
+                pwriter.print(",to=");
+                pwriter.print(getSessionTimeout());
+                long lastCxid = getLastCxid();
+                if (lastCxid >= 0) {
+                    pwriter.print(",lcxid=0x");
+                    pwriter.print(Long.toHexString(lastCxid));
+                }
+                pwriter.print(",lzxid=0x");
+                pwriter.print(Long.toHexString(getLastZxid()));
+                pwriter.print(",lresp=");
+                pwriter.print(getLastResponseTime());
+                pwriter.print(",llat=");
+                pwriter.print(getLastLatency());
+                pwriter.print(",minlat=");
+                pwriter.print(getMinLatency());
+                pwriter.print(",avglat=");
+                pwriter.print(getAvgLatency());
+                pwriter.print(",maxlat=");
+                pwriter.print(getMaxLatency());
+            }
+        }
+    }
+
 }
 }

+ 134 - 0
src/java/main/org/apache/zookeeper/server/ServerCnxnFactory.java

@@ -0,0 +1,134 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.zookeeper.server;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.nio.ByteBuffer;
+import java.util.HashMap;
+
+import javax.management.JMException;
+
+import org.apache.log4j.Logger;
+import org.apache.zookeeper.jmx.MBeanRegistry;
+
+public abstract class ServerCnxnFactory {
+    
+    public static final String ZOOKEEPER_SERVER_CNXN_FACTORY = "zookeeper.serverCnxnFactory";
+
+    public interface PacketProcessor {
+        public void processPacket(ByteBuffer packet, ServerCnxn src);
+    }
+    
+    Logger LOG = Logger.getLogger(ServerCnxnFactory.class);
+
+    /**
+     * The buffer will cause the connection to be close when we do a send.
+     */
+    static final ByteBuffer closeConn = ByteBuffer.allocate(0);
+
+    public abstract int getLocalPort();
+    
+    public abstract Iterable<ServerCnxn> getConnections();
+
+    public abstract void closeSession(long sessionId);
+
+    public abstract void configure(InetSocketAddress addr,
+            int maxClientCnxns) throws IOException;
+
+    /** Maximum number of connections allowed from particular host (ip) */
+    public abstract int getMaxClientCnxnsPerHost();
+
+    /** Maximum number of connections allowed from particular host (ip) */
+    public abstract void setMaxClientCnxnsPerHost(int max);
+
+    public abstract void startup(ZooKeeperServer zkServer)
+        throws IOException, InterruptedException;
+
+    public abstract void join() throws InterruptedException;
+
+    public abstract void shutdown();
+
+    public abstract void start();
+
+    protected ZooKeeperServer zkServer;
+    final public void setZooKeeperServer(ZooKeeperServer zk) {
+        this.zkServer = zk;
+        if (zk != null) {
+            zk.setServerCnxnFactory(this);
+        }
+    }
+
+    public abstract void closeAll();
+    
+    static public ServerCnxnFactory createFactory() throws IOException {
+        String serverCnxnFactoryName =
+            System.getProperty(ZOOKEEPER_SERVER_CNXN_FACTORY);
+        if (serverCnxnFactoryName == null) {
+            serverCnxnFactoryName = NIOServerCnxnFactory.class.getName();
+        }
+        try {
+            return (ServerCnxnFactory) Class.forName(serverCnxnFactoryName)
+                                                .newInstance();
+        } catch (Exception e) {
+            IOException ioe = new IOException("Couldn't instantiate "
+                    + serverCnxnFactoryName);
+            ioe.initCause(e);
+            throw ioe;
+        }
+    }
+    
+    static public ServerCnxnFactory createFactory(int clientPort,
+            int maxClientCnxns) throws IOException
+    {
+        return createFactory(new InetSocketAddress(clientPort), maxClientCnxns);
+    }
+
+    static public ServerCnxnFactory createFactory(InetSocketAddress addr,
+            int maxClientCnxns) throws IOException
+    {
+        ServerCnxnFactory factory = createFactory();
+        factory.configure(addr, maxClientCnxns);
+        return factory;
+    }
+
+    public abstract InetSocketAddress getLocalAddress();
+
+    private HashMap<ServerCnxn, ConnectionBean> connectionBeans = new HashMap<ServerCnxn, ConnectionBean>();
+    public void unregisterConnection(ServerCnxn serverCnxn) {
+        ConnectionBean jmxConnectionBean = connectionBeans.remove(serverCnxn);
+        if (jmxConnectionBean != null){
+            MBeanRegistry.getInstance().unregister(jmxConnectionBean);
+        }
+    }
+    
+    public void registerConnection(ServerCnxn serverCnxn) {
+        if (zkServer != null) {
+            ConnectionBean jmxConnectionBean = new ConnectionBean(serverCnxn, zkServer);
+            try {
+                MBeanRegistry.getInstance().register(jmxConnectionBean, zkServer.jmxServerBean);
+                connectionBeans.put(serverCnxn, jmxConnectionBean);
+            } catch (JMException e) {
+                LOG.warn("Could not register connection", e);
+            }
+        }
+
+    }
+
+}

+ 2 - 0
src/java/main/org/apache/zookeeper/server/SessionTrackerImpl.java

@@ -202,6 +202,8 @@ public class SessionTrackerImpl extends Thread implements SessionTracker {
     }
     }
 
 
     public void shutdown() {
     public void shutdown() {
+        LOG.info("Shutting down");
+
         running = false;
         running = false;
         if (LOG.isTraceEnabled()) {
         if (LOG.isTraceEnabled()) {
             ZooTrace.logTraceMessage(LOG, ZooTrace.getTextTraceLevel(),
             ZooTrace.logTraceMessage(LOG, ZooTrace.getTextTraceLevel(),

+ 68 - 0
src/java/main/org/apache/zookeeper/server/Stats.java

@@ -0,0 +1,68 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.zookeeper.server;
+
+import java.util.Date;
+
+/**
+ * Statistics on the ServerCnxn
+ */
+interface Stats {
+    /** Date/time the connection was established
+     * @since 3.3.0 */
+    Date getEstablished();
+
+    /**
+     * The number of requests that have been submitted but not yet
+     * responded to.
+     */
+    long getOutstandingRequests();
+    /** Number of packets received */
+    long getPacketsReceived();
+    /** Number of packets sent (incl notifications) */
+    long getPacketsSent();
+    /** Min latency in ms
+     * @since 3.3.0 */
+    long getMinLatency();
+    /** Average latency in ms
+     * @since 3.3.0 */
+    long getAvgLatency();
+    /** Max latency in ms
+     * @since 3.3.0 */
+    long getMaxLatency();
+    /** Last operation performed by this connection
+     * @since 3.3.0 */
+    String getLastOperation();
+    /** Last cxid of this connection
+     * @since 3.3.0 */
+    long getLastCxid();
+    /** Last zxid of this connection
+     * @since 3.3.0 */
+    long getLastZxid();
+    /** Last time server sent a response to client on this connection
+     * @since 3.3.0 */
+    long getLastResponseTime();
+    /** Latency of last response to client on this connection in ms
+     * @since 3.3.0 */
+    long getLastLatency();
+
+    /** Reset counters
+     * @since 3.3.0 */
+    void resetStats();
+}

+ 1 - 0
src/java/main/org/apache/zookeeper/server/SyncRequestProcessor.java

@@ -166,6 +166,7 @@ public class SyncRequestProcessor extends Thread implements RequestProcessor {
     }
     }
 
 
     public void shutdown() {
     public void shutdown() {
+        LOG.info("Shutting down");
         queuedRequests.add(requestOfDeath);
         queuedRequests.add(requestOfDeath);
         try {
         try {
             this.join();
             this.join();

+ 182 - 22
src/java/main/org/apache/zookeeper/server/ZooKeeperServer.java

@@ -18,8 +18,10 @@
 
 
 package org.apache.zookeeper.server;
 package org.apache.zookeeper.server;
 
 
+import java.io.ByteArrayOutputStream;
 import java.io.File;
 import java.io.File;
 import java.io.IOException;
 import java.io.IOException;
+import java.io.InputStream;
 import java.io.PrintWriter;
 import java.io.PrintWriter;
 import java.nio.ByteBuffer;
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.ArrayList;
@@ -31,18 +33,27 @@ import java.util.Random;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentHashMap;
 
 
 import org.apache.jute.BinaryInputArchive;
 import org.apache.jute.BinaryInputArchive;
+import org.apache.jute.BinaryOutputArchive;
 import org.apache.jute.Record;
 import org.apache.jute.Record;
 import org.apache.log4j.Logger;
 import org.apache.log4j.Logger;
 import org.apache.zookeeper.Environment;
 import org.apache.zookeeper.Environment;
+import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.KeeperException.SessionExpiredException;
 import org.apache.zookeeper.KeeperException.SessionExpiredException;
 import org.apache.zookeeper.ZooDefs.OpCode;
 import org.apache.zookeeper.ZooDefs.OpCode;
 import org.apache.zookeeper.data.ACL;
 import org.apache.zookeeper.data.ACL;
 import org.apache.zookeeper.data.Id;
 import org.apache.zookeeper.data.Id;
 import org.apache.zookeeper.data.StatPersisted;
 import org.apache.zookeeper.data.StatPersisted;
 import org.apache.zookeeper.jmx.MBeanRegistry;
 import org.apache.zookeeper.jmx.MBeanRegistry;
+import org.apache.zookeeper.proto.AuthPacket;
+import org.apache.zookeeper.proto.ConnectRequest;
+import org.apache.zookeeper.proto.ConnectResponse;
+import org.apache.zookeeper.proto.ReplyHeader;
 import org.apache.zookeeper.proto.RequestHeader;
 import org.apache.zookeeper.proto.RequestHeader;
+import org.apache.zookeeper.server.ServerCnxn.CloseRequestException;
 import org.apache.zookeeper.server.SessionTracker.Session;
 import org.apache.zookeeper.server.SessionTracker.Session;
 import org.apache.zookeeper.server.SessionTracker.SessionExpirer;
 import org.apache.zookeeper.server.SessionTracker.SessionExpirer;
+import org.apache.zookeeper.server.auth.AuthenticationProvider;
+import org.apache.zookeeper.server.auth.ProviderRegistry;
 import org.apache.zookeeper.server.persistence.FileTxnSnapLog;
 import org.apache.zookeeper.server.persistence.FileTxnSnapLog;
 
 
 /**
 /**
@@ -62,16 +73,7 @@ public class ZooKeeperServer implements SessionExpirer, ServerStats.Provider {
     protected ZooKeeperServerBean jmxServerBean;
     protected ZooKeeperServerBean jmxServerBean;
     protected DataTreeBean jmxDataTreeBean;
     protected DataTreeBean jmxDataTreeBean;
 
 
-    /**
-     * Create an instance of ZooKeeper server
-     */
-    static public interface Factory {
-        public ZooKeeperServer createServer() throws IOException;
-
-        public NIOServerCnxn.Factory createConnectionFactory()
-                throws IOException;
-    }
-
+ 
     /**
     /**
      * The server delegates loading of the tree to an instance of the interface
      * The server delegates loading of the tree to an instance of the interface
      */
      */
@@ -112,7 +114,7 @@ public class ZooKeeperServer implements SessionExpirer, ServerStats.Provider {
     final HashMap<String, ChangeRecord> outstandingChangesForPath =
     final HashMap<String, ChangeRecord> outstandingChangesForPath =
         new HashMap<String, ChangeRecord>();
         new HashMap<String, ChangeRecord>();
     
     
-    private NIOServerCnxn.Factory serverCnxnFactory;
+    private ServerCnxnFactory serverCnxnFactory;
 
 
     private final ServerStats serverStats;
     private final ServerStats serverStats;
 
 
@@ -182,7 +184,7 @@ public class ZooKeeperServer implements SessionExpirer, ServerStats.Provider {
         pwriter.print("tickTime=");
         pwriter.print("tickTime=");
         pwriter.println(getTickTime());
         pwriter.println(getTickTime());
         pwriter.print("maxClientCnxns=");
         pwriter.print("maxClientCnxns=");
-        pwriter.println(serverCnxnFactory.getMaxClientCnxns());
+        pwriter.println(serverCnxnFactory.getMaxClientCnxnsPerHost());
         pwriter.print("minSessionTimeout=");
         pwriter.print("minSessionTimeout=");
         pwriter.println(getMinSessionTimeout());
         pwriter.println(getMinSessionTimeout());
         pwriter.print("maxSessionTimeout=");
         pwriter.print("maxSessionTimeout=");
@@ -240,7 +242,7 @@ public class ZooKeeperServer implements SessionExpirer, ServerStats.Provider {
         setZxid(zkDb.loadDataBase());
         setZxid(zkDb.loadDataBase());
         // Clean up dead sessions
         // Clean up dead sessions
         LinkedList<Long> deadSessions = new LinkedList<Long>();
         LinkedList<Long> deadSessions = new LinkedList<Long>();
-        for (long session : zkDb.getSessions()) {
+        for (Long session : zkDb.getSessions()) {
             sessionsWithTimeouts = zkDb.getSessionWithTimeOuts();
             sessionsWithTimeouts = zkDb.getSessionWithTimeOuts();
             if (sessionsWithTimeouts.get(session) == null) {
             if (sessionsWithTimeouts.get(session) == null) {
                 deadSessions.add(session);
                 deadSessions.add(session);
@@ -400,6 +402,8 @@ public class ZooKeeperServer implements SessionExpirer, ServerStats.Provider {
     }
     }
 
 
     public void shutdown() {
     public void shutdown() {
+        LOG.info("shutting down");
+
         // new RuntimeException("Calling shutdown").printStackTrace();
         // new RuntimeException("Calling shutdown").printStackTrace();
         this.running = false;
         this.running = false;
         // Since sessionTracker and syncThreads poll we just have to
         // Since sessionTracker and syncThreads poll we just have to
@@ -497,8 +501,7 @@ public class ZooKeeperServer implements SessionExpirer, ServerStats.Provider {
                 && Arrays.equals(passwd, generatePasswd(sessionId));
                 && Arrays.equals(passwd, generatePasswd(sessionId));
     }
     }
 
 
-    long createSession(ServerCnxn cnxn, byte passwd[], int timeout)
-            throws InterruptedException {
+    long createSession(ServerCnxn cnxn, byte passwd[], int timeout) {
         long sessionId = sessionTracker.createSession(timeout);
         long sessionId = sessionTracker.createSession(timeout);
         Random r = new Random(sessionId ^ superSecret);
         Random r = new Random(sessionId ^ superSecret);
         r.nextBytes(passwd);
         r.nextBytes(passwd);
@@ -520,25 +523,71 @@ public class ZooKeeperServer implements SessionExpirer, ServerStats.Provider {
     }
     }
 
 
     protected void revalidateSession(ServerCnxn cnxn, long sessionId,
     protected void revalidateSession(ServerCnxn cnxn, long sessionId,
-            int sessionTimeout) throws IOException, InterruptedException {
+            int sessionTimeout) throws IOException {
         boolean rc = sessionTracker.touchSession(sessionId, sessionTimeout);
         boolean rc = sessionTracker.touchSession(sessionId, sessionTimeout);
         if (LOG.isTraceEnabled()) {
         if (LOG.isTraceEnabled()) {
             ZooTrace.logTraceMessage(LOG,ZooTrace.SESSION_TRACE_MASK,
             ZooTrace.logTraceMessage(LOG,ZooTrace.SESSION_TRACE_MASK,
                                      "Session 0x" + Long.toHexString(sessionId) +
                                      "Session 0x" + Long.toHexString(sessionId) +
                     " is valid: " + rc);
                     " is valid: " + rc);
         }
         }
-        cnxn.finishSessionInit(rc);
+        finishSessionInit(cnxn, rc);
     }
     }
 
 
     public void reopenSession(ServerCnxn cnxn, long sessionId, byte[] passwd,
     public void reopenSession(ServerCnxn cnxn, long sessionId, byte[] passwd,
-            int sessionTimeout) throws IOException, InterruptedException {
+            int sessionTimeout) throws IOException {
         if (!checkPasswd(sessionId, passwd)) {
         if (!checkPasswd(sessionId, passwd)) {
-            cnxn.finishSessionInit(false);
+            finishSessionInit(cnxn, false);
         } else {
         } else {
             revalidateSession(cnxn, sessionId, sessionTimeout);
             revalidateSession(cnxn, sessionId, sessionTimeout);
         }
         }
     }
     }
 
 
+    public void finishSessionInit(ServerCnxn cnxn, boolean valid) {
+        // register with JMX
+        try {
+            if (valid) {
+                serverCnxnFactory.registerConnection(cnxn);
+            }
+        } catch (Exception e) {
+                LOG.warn("Failed to register with JMX", e);
+        }
+
+        try {
+            ConnectResponse rsp = new ConnectResponse(0, valid ? cnxn.getSessionTimeout()
+                    : 0, valid ? cnxn.getSessionId() : 0, // send 0 if session is no
+                            // longer valid
+                            valid ? generatePasswd(cnxn.getSessionId()) : new byte[16]);
+            ByteArrayOutputStream baos = new ByteArrayOutputStream();
+            BinaryOutputArchive bos = BinaryOutputArchive.getArchive(baos);
+            bos.writeInt(-1, "len");
+            rsp.serialize(bos, "connect");
+            baos.close();
+            ByteBuffer bb = ByteBuffer.wrap(baos.toByteArray());
+            bb.putInt(bb.remaining() - 4).rewind();
+            cnxn.sendBuffer(bb);    
+
+            if (!valid) {
+                LOG.info("Invalid session 0x"
+                        + Long.toHexString(cnxn.getSessionId())
+                        + " for client "
+                        + cnxn.getRemoteSocketAddress()
+                        + ", probably expired");
+                cnxn.sendBuffer(ServerCnxnFactory.closeConn);
+            } else {
+                LOG.info("Established session 0x"
+                        + Long.toHexString(cnxn.getSessionId())
+                        + " with negotiated timeout " + cnxn.getSessionTimeout()
+                        + " for client "
+                        + cnxn.getRemoteSocketAddress());
+            }
+                
+            cnxn.enableRecv();
+        } catch (Exception e) {
+            LOG.warn("Exception while establishing session, closing", e);
+            cnxn.close();
+        }
+    }
+    
     public void closeSession(ServerCnxn cnxn, RequestHeader requestHeader) {
     public void closeSession(ServerCnxn cnxn, RequestHeader requestHeader) {
         closeSession(cnxn.getSessionId());
         closeSession(cnxn.getSessionId());
     }
     }
@@ -620,11 +669,11 @@ public class ZooKeeperServer implements SessionExpirer, ServerStats.Provider {
         return limit;
         return limit;
     }
     }
 
 
-    public void setServerCnxnFactory(NIOServerCnxn.Factory factory) {
+    public void setServerCnxnFactory(ServerCnxnFactory factory) {
         serverCnxnFactory = factory;
         serverCnxnFactory = factory;
     }
     }
 
 
-    public NIOServerCnxn.Factory getServerCnxnFactory() {
+    public ServerCnxnFactory getServerCnxnFactory() {
         return serverCnxnFactory;
         return serverCnxnFactory;
     }
     }
 
 
@@ -684,7 +733,7 @@ public class ZooKeeperServer implements SessionExpirer, ServerStats.Provider {
     }
     }
 
 
     public int getClientPort() {
     public int getClientPort() {
-        return serverCnxnFactory != null ? serverCnxnFactory.ss.socket().getLocalPort() : -1;
+        return serverCnxnFactory != null ? serverCnxnFactory.getLocalPort() : -1;
     }
     }
 
 
     public void setTxnLogFactory(FileTxnSnapLog txnLog) {
     public void setTxnLogFactory(FileTxnSnapLog txnLog) {
@@ -703,4 +752,115 @@ public class ZooKeeperServer implements SessionExpirer, ServerStats.Provider {
     	zkDb.dumpEphemerals(pwriter);
     	zkDb.dumpEphemerals(pwriter);
     }
     }
     
     
+    public void processConnectRequest(ServerCnxn cnxn, ByteBuffer incomingBuffer) throws IOException {
+        BinaryInputArchive bia = BinaryInputArchive.getArchive(new ByteBufferInputStream(incomingBuffer));
+        ConnectRequest connReq = new ConnectRequest();
+        connReq.deserialize(bia, "connect");
+        if (LOG.isDebugEnabled()) {
+            LOG.debug("Session establishment request from client "
+                    + cnxn.getRemoteSocketAddress()
+                    + " client's lastZxid is 0x"
+                    + Long.toHexString(connReq.getLastZxidSeen()));
+        }
+        if (connReq.getLastZxidSeen() > zkDb.dataTree.lastProcessedZxid) {
+            String msg = "Refusing session request for client "
+                + cnxn.getRemoteSocketAddress()
+                + " as it has seen zxid 0x"
+                + Long.toHexString(connReq.getLastZxidSeen())
+                + " our last zxid is 0x"
+                + Long.toHexString(getZKDatabase().getDataTreeLastProcessedZxid())
+                + " client must try another server";
+
+            LOG.info(msg);
+            throw new CloseRequestException(msg);
+        }
+        int sessionTimeout = connReq.getTimeOut();
+        byte passwd[] = connReq.getPasswd();
+        int minSessionTimeout = getMinSessionTimeout();
+        if (sessionTimeout < minSessionTimeout) {
+            sessionTimeout = minSessionTimeout;
+        }
+        int maxSessionTimeout = getMaxSessionTimeout();
+        if (sessionTimeout > maxSessionTimeout) {
+            sessionTimeout = maxSessionTimeout;
+        }
+        cnxn.setSessionTimeout(sessionTimeout);
+        // We don't want to receive any packets until we are sure that the
+        // session is setup
+        cnxn.disableRecv();
+        long sessionId = connReq.getSessionId();
+        if (sessionId != 0) {
+            long clientSessionId = connReq.getSessionId();
+            LOG.info("Client attempting to renew session 0x"
+                    + Long.toHexString(clientSessionId)
+                    + " at " + cnxn.getRemoteSocketAddress());
+            serverCnxnFactory.closeSession(sessionId);
+            cnxn.setSessionId(sessionId);
+            reopenSession(cnxn, sessionId, passwd, sessionTimeout);
+        } else {
+            LOG.info("Client attempting to establish new session at "
+                    + cnxn.getRemoteSocketAddress());
+            createSession(cnxn, passwd, sessionTimeout);
+        }
+    }
+
+    public boolean shouldThrottle(long outStandingCount) {
+        if (getGlobalOutstandingLimit() < getInProcess()) {
+            return outStandingCount > 0;
+        }
+        return false; 
+    }
+
+    public void processPacket(ServerCnxn cnxn, ByteBuffer incomingBuffer) throws IOException {
+        // We have the request, now process and setup for next
+        InputStream bais = new ByteBufferInputStream(incomingBuffer);
+        BinaryInputArchive bia = BinaryInputArchive.getArchive(bais);
+        RequestHeader h = new RequestHeader();
+        h.deserialize(bia, "header");
+        // Through the magic of byte buffers, txn will not be
+        // pointing
+        // to the start of the txn
+        incomingBuffer = incomingBuffer.slice();
+        if (h.getType() == OpCode.auth) {
+            LOG.info("got auth packet " + cnxn.getRemoteSocketAddress());
+            AuthPacket authPacket = new AuthPacket();
+            ZooKeeperServer.byteBuffer2Record(incomingBuffer, authPacket);
+            String scheme = authPacket.getScheme();
+            AuthenticationProvider ap = ProviderRegistry.getProvider(scheme);
+            if (ap == null
+                    || (ap.handleAuthentication(cnxn, authPacket.getAuth())
+                            != KeeperException.Code.OK)) {
+                if (ap == null) {
+                    LOG.warn("No authentication provider for scheme: "
+                            + scheme + " has "
+                            + ProviderRegistry.listProviders());
+                } else {
+                    LOG.warn("Authentication failed for scheme: " + scheme);
+                }
+                // send a response...
+                ReplyHeader rh = new ReplyHeader(h.getXid(), 0,
+                        KeeperException.Code.AUTHFAILED.intValue());
+                cnxn.sendResponse(rh, null, null);
+                // ... and close connection
+                cnxn.sendBuffer(ServerCnxnFactory.closeConn);
+                cnxn.disableRecv();
+            } else {
+                if (LOG.isDebugEnabled()) {
+                    LOG.debug("Authentication succeeded for scheme: "
+                              + scheme);
+                }
+                LOG.info("auth success " + cnxn.getRemoteSocketAddress());
+                ReplyHeader rh = new ReplyHeader(h.getXid(), 0,
+                        KeeperException.Code.OK.intValue());
+                cnxn.sendResponse(rh, null, null);
+            }
+            return;
+        } else {
+            Request si = new Request(cnxn, cnxn.getSessionId(), h.getXid(),
+                    h.getType(), incomingBuffer, cnxn.getAuthInfo());
+            si.setOwner(ServerCnxn.me);
+            submitRequest(si);
+        }
+        cnxn.incrOutstandingRequests(h);
+    }
 }
 }

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

@@ -90,16 +90,16 @@ public class ZooKeeperServerBean implements ZooKeeperServerMXBean, ZKMBeanInfo {
     }
     }
 
 
     public int getMaxClientCnxnsPerHost() {
     public int getMaxClientCnxnsPerHost() {
-        NIOServerCnxn.Factory fac = zks.getServerCnxnFactory();
+        ServerCnxnFactory fac = zks.getServerCnxnFactory();
         if (fac == null) {
         if (fac == null) {
             return -1;
             return -1;
         }
         }
-        return fac.getMaxClientCnxns();
+        return fac.getMaxClientCnxnsPerHost();
     }
     }
 
 
     public void setMaxClientCnxnsPerHost(int max) {
     public void setMaxClientCnxnsPerHost(int max) {
         // if fac is null the exception will be propagated to the client
         // if fac is null the exception will be propagated to the client
-        zks.getServerCnxnFactory().maxClientCnxns = max;
+        zks.getServerCnxnFactory().setMaxClientCnxnsPerHost(max);
     }
     }
 
 
     public int getMinSessionTimeout() {
     public int getMinSessionTimeout() {

+ 3 - 2
src/java/main/org/apache/zookeeper/server/ZooKeeperServerMain.java

@@ -38,7 +38,7 @@ public class ZooKeeperServerMain {
     private static final String USAGE =
     private static final String USAGE =
         "Usage: ZooKeeperServerMain configfile | port datadir [ticktime] [maxcnxns]";
         "Usage: ZooKeeperServerMain configfile | port datadir [ticktime] [maxcnxns]";
 
 
-    private NIOServerCnxn.Factory cnxnFactory;
+    private ServerCnxnFactory cnxnFactory;
 
 
     /*
     /*
      * Start up the ZooKeeper server.
      * Start up the ZooKeeper server.
@@ -105,7 +105,8 @@ public class ZooKeeperServerMain {
             zkServer.setTickTime(config.tickTime);
             zkServer.setTickTime(config.tickTime);
             zkServer.setMinSessionTimeout(config.minSessionTimeout);
             zkServer.setMinSessionTimeout(config.minSessionTimeout);
             zkServer.setMaxSessionTimeout(config.maxSessionTimeout);
             zkServer.setMaxSessionTimeout(config.maxSessionTimeout);
-            cnxnFactory = new NIOServerCnxn.Factory(config.getClientPortAddress(),
+            cnxnFactory = ServerCnxnFactory.createFactory();
+            cnxnFactory.configure(config.getClientPortAddress(),
                     config.getMaxClientCnxns());
                     config.getMaxClientCnxns());
             cnxnFactory.startup(zkServer);
             cnxnFactory.startup(zkServer);
             cnxnFactory.join();
             cnxnFactory.join();

+ 2 - 2
src/java/main/org/apache/zookeeper/server/auth/DigestAuthenticationProvider.java

@@ -102,9 +102,9 @@ public class DigestAuthenticationProvider implements AuthenticationProvider {
         try {
         try {
             String digest = generateDigest(id);
             String digest = generateDigest(id);
             if (digest.equals(superDigest)) {
             if (digest.equals(superDigest)) {
-                cnxn.getAuthInfo().add(new Id("super", ""));
+                cnxn.addAuthInfo(new Id("super", ""));
             }
             }
-            cnxn.getAuthInfo().add(new Id(getScheme(), digest));
+            cnxn.addAuthInfo(new Id(getScheme(), digest));
             return KeeperException.Code.OK;
             return KeeperException.Code.OK;
         } catch (NoSuchAlgorithmException e) {
         } catch (NoSuchAlgorithmException e) {
             LOG.error("Missing algorithm",e);
             LOG.error("Missing algorithm",e);

+ 3 - 3
src/java/main/org/apache/zookeeper/server/auth/IPAuthenticationProvider.java

@@ -18,9 +18,9 @@
 
 
 package org.apache.zookeeper.server.auth;
 package org.apache.zookeeper.server.auth;
 
 
+import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.data.Id;
 import org.apache.zookeeper.data.Id;
 import org.apache.zookeeper.server.ServerCnxn;
 import org.apache.zookeeper.server.ServerCnxn;
-import org.apache.zookeeper.KeeperException;
 
 
 public class IPAuthenticationProvider implements AuthenticationProvider {
 public class IPAuthenticationProvider implements AuthenticationProvider {
 
 
@@ -31,8 +31,8 @@ public class IPAuthenticationProvider implements AuthenticationProvider {
     public KeeperException.Code
     public KeeperException.Code
         handleAuthentication(ServerCnxn cnxn, byte[] authData)
         handleAuthentication(ServerCnxn cnxn, byte[] authData)
     {
     {
-        String id = cnxn.getRemoteAddress().getAddress().getHostAddress();
-        cnxn.getAuthInfo().add(new Id(getScheme(), id));
+        String id = cnxn.getRemoteSocketAddress().getAddress().getHostAddress();
+        cnxn.addAuthInfo(new Id(getScheme(), id));
         return KeeperException.Code.OK;
         return KeeperException.Code.OK;
     }
     }
 
 

+ 1 - 0
src/java/main/org/apache/zookeeper/server/quorum/CommitProcessor.java

@@ -176,6 +176,7 @@ public class CommitProcessor extends Thread implements RequestProcessor {
     }
     }
 
 
     public void shutdown() {
     public void shutdown() {
+        LOG.info("Shutting down");
         synchronized (this) {
         synchronized (this) {
             finished = true;
             finished = true;
             queuedRequests.clear();
             queuedRequests.clear();

+ 2 - 2
src/java/main/org/apache/zookeeper/server/quorum/FastLeaderElection.java

@@ -383,14 +383,14 @@ public class FastLeaderElection implements Election {
             this.ws = new WorkerSender(manager);
             this.ws = new WorkerSender(manager);
 
 
             Thread t = new Thread(this.ws,
             Thread t = new Thread(this.ws,
-                    "WorkerSender Thread");
+                    "WorkerSender(" + Thread.currentThread().getName() + ")");
             t.setDaemon(true);
             t.setDaemon(true);
             t.start();
             t.start();
 
 
             this.wr = new WorkerReceiver(manager);
             this.wr = new WorkerReceiver(manager);
 
 
             t = new Thread(this.wr,
             t = new Thread(this.wr,
-                                    "WorkerReceiver Thread");
+                    "WorkerReceiver(" + Thread.currentThread().getName() + ")");
             t.setDaemon(true);
             t.setDaemon(true);
             t.start();
             t.start();
         }
         }

+ 4 - 5
src/java/main/org/apache/zookeeper/server/quorum/Follower.java

@@ -59,8 +59,8 @@ public class Follower extends Learner{
      */
      */
     void followLeader() throws InterruptedException {
     void followLeader() throws InterruptedException {
         fzk.registerJMX(new FollowerBean(this, zk), self.jmxLocalPeerBean);
         fzk.registerJMX(new FollowerBean(this, zk), self.jmxLocalPeerBean);
-        try {            
-            InetSocketAddress addr = findLeader();            
+        try {
+            InetSocketAddress addr = findLeader();
             try {
             try {
                 connectToLeader(addr);
                 connectToLeader(addr);
                 long newLeaderZxid = registerWithLeader(Leader.FOLLOWERINFO);
                 long newLeaderZxid = registerWithLeader(Leader.FOLLOWERINFO);
@@ -76,8 +76,8 @@ public class Follower extends Learner{
                 QuorumPacket qp = new QuorumPacket();
                 QuorumPacket qp = new QuorumPacket();
                 while (self.isRunning()) {
                 while (self.isRunning()) {
                     readPacket(qp);
                     readPacket(qp);
-                    processPacket(qp);                   
-                }                              
+                    processPacket(qp);
+                }
             } catch (IOException e) {
             } catch (IOException e) {
                 LOG.warn("Exception when following the leader", e);
                 LOG.warn("Exception when following the leader", e);
                 try {
                 try {
@@ -137,7 +137,6 @@ public class Follower extends Learner{
         }
         }
     }
     }
 
 
-
     /**
     /**
      * The zxid of the last operation seen
      * The zxid of the last operation seen
      * @return zxid
      * @return zxid

+ 1 - 0
src/java/main/org/apache/zookeeper/server/quorum/FollowerRequestProcessor.java

@@ -100,6 +100,7 @@ public class FollowerRequestProcessor extends Thread implements
     }
     }
 
 
     public void shutdown() {
     public void shutdown() {
+        LOG.info("Shutting down");
         finished = true;
         finished = true;
         queuedRequests.clear();
         queuedRequests.clear();
         queuedRequests.add(Request.requestOfDeath);
         queuedRequests.add(Request.requestOfDeath);

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

@@ -80,7 +80,7 @@ public class FollowerZooKeeperServer extends LearnerZooKeeperServer {
                 new SendAckRequestProcessor((Learner)getFollower()));
                 new SendAckRequestProcessor((Learner)getFollower()));
         syncProcessor.start();
         syncProcessor.start();
     }
     }
-    
+
     LinkedBlockingQueue<Request> pendingTxns = new LinkedBlockingQueue<Request>();
     LinkedBlockingQueue<Request> pendingTxns = new LinkedBlockingQueue<Request>();
 
 
     public void logRequest(TxnHeader hdr, Record txn) {
     public void logRequest(TxnHeader hdr, Record txn) {
@@ -135,6 +135,7 @@ public class FollowerZooKeeperServer extends LearnerZooKeeperServer {
     
     
     @Override
     @Override
     public void shutdown() {
     public void shutdown() {
+        LOG.info("Shutting down");
         try {
         try {
             super.shutdown();
             super.shutdown();
         } catch (Exception e) {
         } catch (Exception e) {

+ 4 - 1
src/java/main/org/apache/zookeeper/server/quorum/Leader.java

@@ -383,6 +383,8 @@ public class Leader {
      * Close down all the LearnerHandlers
      * Close down all the LearnerHandlers
      */
      */
     void shutdown(String reason) {
     void shutdown(String reason) {
+        LOG.info("Shutting down");
+
         if (isShutdown) {
         if (isShutdown) {
             return;
             return;
         }
         }
@@ -402,7 +404,7 @@ public class Leader {
             LOG.warn("Ignoring unexpected exception during close",e);
             LOG.warn("Ignoring unexpected exception during close",e);
         }
         }
         // clear all the connections
         // clear all the connections
-        self.cnxnFactory.clear();
+        self.cnxnFactory.closeAll();
         // shutdown the previous zk
         // shutdown the previous zk
         if (zk != null) {
         if (zk != null) {
             zk.shutdown();
             zk.shutdown();
@@ -551,6 +553,7 @@ public class Leader {
          * @see org.apache.zookeeper.server.RequestProcessor#shutdown()
          * @see org.apache.zookeeper.server.RequestProcessor#shutdown()
          */
          */
         public void shutdown() {
         public void shutdown() {
+            LOG.info("Shutting down");
             next.shutdown();
             next.shutdown();
         }
         }
     }
     }

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

@@ -163,7 +163,7 @@ public class LeaderZooKeeperServer extends QuorumZooKeeperServer {
     
     
     @Override
     @Override
     protected void revalidateSession(ServerCnxn cnxn, long sessionId,
     protected void revalidateSession(ServerCnxn cnxn, long sessionId,
-        int sessionTimeout) throws IOException, InterruptedException {
+        int sessionTimeout) throws IOException {
         super.revalidateSession(cnxn, sessionId, sessionTimeout);
         super.revalidateSession(cnxn, sessionId, sessionTimeout);
         try {
         try {
             // setowner as the leader itself, unless updated
             // setowner as the leader itself, unless updated

+ 3 - 4
src/java/main/org/apache/zookeeper/server/quorum/Learner.java

@@ -88,10 +88,9 @@ public class Learner {
      *                the timeout for which the session is valid
      *                the timeout for which the session is valid
      * @return
      * @return
      * @throws IOException
      * @throws IOException
-     * @throws InterruptedException
      */
      */
     void validateSession(ServerCnxn cnxn, long clientId, int timeout)
     void validateSession(ServerCnxn cnxn, long clientId, int timeout)
-            throws IOException, InterruptedException {
+            throws IOException {
         LOG.info("Revalidating client: 0x" + Long.toHexString(clientId));
         LOG.info("Revalidating client: 0x" + Long.toHexString(clientId));
         ByteArrayOutputStream baos = new ByteArrayOutputStream();
         ByteArrayOutputStream baos = new ByteArrayOutputStream();
         DataOutputStream dos = new DataOutputStream(baos);
         DataOutputStream dos = new DataOutputStream(baos);
@@ -336,7 +335,7 @@ public class Learner {
                         + Long.toHexString(sessionId)
                         + Long.toHexString(sessionId)
                         + " for validation");
                         + " for validation");
             } else {
             } else {
-                cnxn.finishSessionInit(valid);
+                zk.finishSessionInit(cnxn, valid);
             }
             }
         }
         }
         if (LOG.isTraceEnabled()) {
         if (LOG.isTraceEnabled()) {
@@ -369,7 +368,7 @@ public class Learner {
         // set the zookeeper server to null
         // set the zookeeper server to null
         self.cnxnFactory.setZooKeeperServer(null);
         self.cnxnFactory.setZooKeeperServer(null);
         // clear all the connections
         // clear all the connections
-        self.cnxnFactory.clear();
+        self.cnxnFactory.closeAll();
         // shutdown previous zookeeper
         // shutdown previous zookeeper
         if (zk != null) {
         if (zk != null) {
             zk.shutdown();
             zk.shutdown();

+ 1 - 1
src/java/main/org/apache/zookeeper/server/quorum/LearnerZooKeeperServer.java

@@ -77,7 +77,7 @@ public abstract class LearnerZooKeeperServer extends QuorumZooKeeperServer {
     
     
     @Override
     @Override
     protected void revalidateSession(ServerCnxn cnxn, long sessionId,
     protected void revalidateSession(ServerCnxn cnxn, long sessionId,
-            int sessionTimeout) throws IOException, InterruptedException {
+            int sessionTimeout) throws IOException {
         getLearner().validateSession(cnxn, sessionId, sessionTimeout);
         getLearner().validateSession(cnxn, sessionId, sessionTimeout);
     }
     }
     
     

+ 1 - 7
src/java/main/org/apache/zookeeper/server/quorum/LocalPeerBean.java

@@ -18,8 +18,6 @@
 
 
 package org.apache.zookeeper.server.quorum;
 package org.apache.zookeeper.server.quorum;
 
 
-import org.apache.zookeeper.server.NIOServerCnxn;
-import org.apache.zookeeper.server.quorum.QuorumPeer;
 
 
 /**
 /**
  * Implementation of the local peer MBean interface.
  * Implementation of the local peer MBean interface.
@@ -44,11 +42,7 @@ public class LocalPeerBean extends ServerBean implements LocalPeerMXBean {
     }
     }
     
     
     public int getMaxClientCnxnsPerHost() {
     public int getMaxClientCnxnsPerHost() {
-        NIOServerCnxn.Factory fac = peer.getCnxnFactory();
-        if (fac == null) {
-            return -1;
-        }
-        return fac.getMaxClientCnxns();
+        return peer.getMaxClientCnxnsPerHost();
     }
     }
 
 
     public int getMinSessionTimeout() {
     public int getMinSessionTimeout() {

+ 1 - 0
src/java/main/org/apache/zookeeper/server/quorum/ObserverRequestProcessor.java

@@ -114,6 +114,7 @@ public class ObserverRequestProcessor extends Thread implements
      * Shutdown the processor.
      * Shutdown the processor.
      */
      */
     public void shutdown() {
     public void shutdown() {
+        LOG.info("Shutting down");
         finished = true;
         finished = true;
         queuedRequests.clear();
         queuedRequests.clear();
         queuedRequests.add(Request.requestOfDeath);
         queuedRequests.add(Request.requestOfDeath);

+ 6 - 0
src/java/main/org/apache/zookeeper/server/quorum/ProposalRequestProcessor.java

@@ -18,15 +18,20 @@
 
 
 package org.apache.zookeeper.server.quorum;
 package org.apache.zookeeper.server.quorum;
 
 
+import org.apache.log4j.Logger;
 import org.apache.zookeeper.server.Request;
 import org.apache.zookeeper.server.Request;
 import org.apache.zookeeper.server.RequestProcessor;
 import org.apache.zookeeper.server.RequestProcessor;
 import org.apache.zookeeper.server.SyncRequestProcessor;
 import org.apache.zookeeper.server.SyncRequestProcessor;
+import org.apache.zookeeper.server.ZooKeeperServer;
 
 
 /**
 /**
  * This RequestProcessor simply forwards requests to an AckRequestProcessor and
  * This RequestProcessor simply forwards requests to an AckRequestProcessor and
  * SyncRequestProcessor.
  * SyncRequestProcessor.
  */
  */
 public class ProposalRequestProcessor implements RequestProcessor {
 public class ProposalRequestProcessor implements RequestProcessor {
+    private static final Logger LOG =
+        Logger.getLogger(ProposalRequestProcessor.class);
+
     LeaderZooKeeperServer zks;
     LeaderZooKeeperServer zks;
     
     
     RequestProcessor nextProcessor;
     RequestProcessor nextProcessor;
@@ -75,6 +80,7 @@ public class ProposalRequestProcessor implements RequestProcessor {
     }
     }
 
 
     public void shutdown() {
     public void shutdown() {
+        LOG.info("Shutting down");
         nextProcessor.shutdown();
         nextProcessor.shutdown();
         syncProcessor.shutdown();
         syncProcessor.shutdown();
     }
     }

+ 5 - 3
src/java/main/org/apache/zookeeper/server/quorum/QuorumCnxManager.java

@@ -433,9 +433,11 @@ public class QuorumCnxManager {
                 try {
                 try {
                     ss = ServerSocketChannel.open();
                     ss = ServerSocketChannel.open();
                     int port = self.quorumPeers.get(self.getId()).electionAddr.getPort();
                     int port = self.quorumPeers.get(self.getId()).electionAddr.getPort();
-                    LOG.info("My election bind port: " + port);
-                    ss.socket().setReuseAddress(true); 
-                    ss.socket().bind(new InetSocketAddress(port));
+                    ss.socket().setReuseAddress(true);
+                    InetSocketAddress addr = new InetSocketAddress(port);
+                    LOG.info("My election bind port: " + addr.toString());
+                    setName(addr.toString());
+                    ss.socket().bind(addr);
 
 
                     while (!shutdown) {
                     while (!shutdown) {
                         SocketChannel client = ss.accept();
                         SocketChannel client = ss.accept();

+ 23 - 20
src/java/main/org/apache/zookeeper/server/quorum/QuorumPeer.java

@@ -25,6 +25,7 @@ import java.net.InetSocketAddress;
 import java.net.SocketException;
 import java.net.SocketException;
 import java.nio.ByteBuffer;
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.ArrayList;
+import java.util.Collection;
 import java.util.Collections;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashMap;
 import java.util.List;
 import java.util.List;
@@ -33,7 +34,7 @@ import java.util.Map;
 import org.apache.log4j.Logger;
 import org.apache.log4j.Logger;
 import org.apache.zookeeper.jmx.MBeanRegistry;
 import org.apache.zookeeper.jmx.MBeanRegistry;
 import org.apache.zookeeper.jmx.ZKMBeanInfo;
 import org.apache.zookeeper.jmx.ZKMBeanInfo;
-import org.apache.zookeeper.server.NIOServerCnxn;
+import org.apache.zookeeper.server.ServerCnxnFactory;
 import org.apache.zookeeper.server.ZKDatabase;
 import org.apache.zookeeper.server.ZKDatabase;
 import org.apache.zookeeper.server.ZooKeeperServer;
 import org.apache.zookeeper.server.ZooKeeperServer;
 import org.apache.zookeeper.server.persistence.FileTxnSnapLog;
 import org.apache.zookeeper.server.persistence.FileTxnSnapLog;
@@ -73,7 +74,7 @@ public class QuorumPeer extends Thread implements QuorumStats.Provider {
     QuorumBean jmxQuorumBean;
     QuorumBean jmxQuorumBean;
     LocalPeerBean jmxLocalPeerBean;
     LocalPeerBean jmxLocalPeerBean;
     LeaderElectionBean jmxLeaderElectionBean;
     LeaderElectionBean jmxLeaderElectionBean;
-    
+
     /* ZKDatabase is a top level member of quorumpeer 
     /* ZKDatabase is a top level member of quorumpeer 
      * which will be used in all the zookeeperservers
      * which will be used in all the zookeeperservers
      * instantiated later. Also, it is created once on 
      * instantiated later. Also, it is created once on 
@@ -81,14 +82,6 @@ public class QuorumPeer extends Thread implements QuorumStats.Provider {
      * message from the leader
      * message from the leader
      */
      */
     private ZKDatabase zkDb;
     private ZKDatabase zkDb;
-    
-    /**
-     * Create an instance of a quorum peer
-     */
-    public interface Factory{
-        public QuorumPeer create(NIOServerCnxn.Factory cnxnFactory) throws IOException;
-        public NIOServerCnxn.Factory createConnectionFactory() throws IOException;
-    }
 
 
     public static class QuorumServer {
     public static class QuorumServer {
         public QuorumServer(long id, InetSocketAddress addr,
         public QuorumServer(long id, InetSocketAddress addr,
@@ -343,7 +336,7 @@ public class QuorumPeer extends Thread implements QuorumStats.Provider {
 
 
     Election electionAlg;
     Election electionAlg;
 
 
-    NIOServerCnxn.Factory cnxnFactory;
+    ServerCnxnFactory cnxnFactory;
     private FileTxnSnapLog logFactory = null;
     private FileTxnSnapLog logFactory = null;
 
 
     private final QuorumStats quorumStats;
     private final QuorumStats quorumStats;
@@ -361,7 +354,7 @@ public class QuorumPeer extends Thread implements QuorumStats.Provider {
     public QuorumPeer(Map<Long, QuorumServer> quorumPeers, File dataDir,
     public QuorumPeer(Map<Long, QuorumServer> quorumPeers, File dataDir,
             File dataLogDir, int electionType,
             File dataLogDir, int electionType,
             long myid, int tickTime, int initLimit, int syncLimit,
             long myid, int tickTime, int initLimit, int syncLimit,
-            NIOServerCnxn.Factory cnxnFactory) throws IOException {
+            ServerCnxnFactory cnxnFactory) throws IOException {
         this(quorumPeers, dataDir, dataLogDir, electionType, myid, tickTime, 
         this(quorumPeers, dataDir, dataLogDir, electionType, myid, tickTime, 
         		initLimit, syncLimit, cnxnFactory, 
         		initLimit, syncLimit, cnxnFactory, 
         		new QuorumMaj(countParticipants(quorumPeers)));
         		new QuorumMaj(countParticipants(quorumPeers)));
@@ -370,7 +363,7 @@ public class QuorumPeer extends Thread implements QuorumStats.Provider {
     public QuorumPeer(Map<Long, QuorumServer> quorumPeers, File dataDir,
     public QuorumPeer(Map<Long, QuorumServer> quorumPeers, File dataDir,
             File dataLogDir, int electionType,
             File dataLogDir, int electionType,
             long myid, int tickTime, int initLimit, int syncLimit,
             long myid, int tickTime, int initLimit, int syncLimit,
-            NIOServerCnxn.Factory cnxnFactory, 
+            ServerCnxnFactory cnxnFactory, 
             QuorumVerifier quorumConfig) throws IOException {
             QuorumVerifier quorumConfig) throws IOException {
         this();
         this();
         this.cnxnFactory = cnxnFactory;
         this.cnxnFactory = cnxnFactory;
@@ -459,8 +452,7 @@ public class QuorumPeer extends Thread implements QuorumStats.Provider {
     {
     {
         this(quorumPeers, snapDir, logDir, electionAlg,
         this(quorumPeers, snapDir, logDir, electionAlg,
                 myid,tickTime, initLimit,syncLimit,
                 myid,tickTime, initLimit,syncLimit,
-                new NIOServerCnxn.Factory(
-                        new InetSocketAddress(clientPort)),
+                ServerCnxnFactory.createFactory(new InetSocketAddress(clientPort), -1),
                 new QuorumMaj(countParticipants(quorumPeers)));
                 new QuorumMaj(countParticipants(quorumPeers)));
     }
     }
     
     
@@ -476,8 +468,8 @@ public class QuorumPeer extends Thread implements QuorumStats.Provider {
     {
     {
         this(quorumPeers, snapDir, logDir, electionAlg,
         this(quorumPeers, snapDir, logDir, electionAlg,
                 myid,tickTime, initLimit,syncLimit,
                 myid,tickTime, initLimit,syncLimit,
-                new NIOServerCnxn.Factory(new InetSocketAddress(clientPort)),
-                    quorumConfig);
+                ServerCnxnFactory.createFactory(new InetSocketAddress(clientPort), -1),
+                quorumConfig);
     }
     }
     
     
     /**
     /**
@@ -759,7 +751,9 @@ public class QuorumPeer extends Thread implements QuorumStats.Provider {
         synchronized (this) {
         synchronized (this) {
             if (leader != null) {
             if (leader != null) {
                 synchronized (leader.learners) {
                 synchronized (leader.learners) {
-                    for (LearnerHandler fh : leader.learners) {
+                    for (LearnerHandler fh :
+                        (Collection<LearnerHandler>)leader.learners)
+                    {
                         if (fh.getSocket() == null)
                         if (fh.getSocket() == null)
                             continue;
                             continue;
                         String s = fh.getSocket().getRemoteSocketAddress().toString();
                         String s = fh.getSocket().getRemoteSocketAddress().toString();
@@ -819,6 +813,15 @@ public class QuorumPeer extends Thread implements QuorumStats.Provider {
         this.tickTime = tickTime;
         this.tickTime = tickTime;
     }
     }
 
 
+    /** Maximum number of connections allowed from particular host (ip) */
+    public int getMaxClientCnxnsPerHost() {
+        ServerCnxnFactory fac = getCnxnFactory();
+        if (fac == null) {
+            return -1;
+        }
+        return fac.getMaxClientCnxnsPerHost();
+    }
+    
     /** minimum session timeout in milliseconds */
     /** minimum session timeout in milliseconds */
     public int getMinSessionTimeout() {
     public int getMinSessionTimeout() {
         return minSessionTimeout == -1 ? tickTime * 2 : minSessionTimeout;
         return minSessionTimeout == -1 ? tickTime * 2 : minSessionTimeout;
@@ -912,11 +915,11 @@ public class QuorumPeer extends Thread implements QuorumStats.Provider {
         this.electionType = electionType;
         this.electionType = electionType;
     }
     }
 
 
-    public NIOServerCnxn.Factory getCnxnFactory() {
+    public ServerCnxnFactory getCnxnFactory() {
         return cnxnFactory;
         return cnxnFactory;
     }
     }
 
 
-    public void setCnxnFactory(NIOServerCnxn.Factory cnxnFactory) {
+    public void setCnxnFactory(ServerCnxnFactory cnxnFactory) {
         this.cnxnFactory = cnxnFactory;
         this.cnxnFactory = cnxnFactory;
     }
     }
 
 

+ 4 - 4
src/java/main/org/apache/zookeeper/server/quorum/QuorumPeerMain.java

@@ -24,7 +24,7 @@ import javax.management.JMException;
 
 
 import org.apache.log4j.Logger;
 import org.apache.log4j.Logger;
 import org.apache.zookeeper.jmx.ManagedUtil;
 import org.apache.zookeeper.jmx.ManagedUtil;
-import org.apache.zookeeper.server.NIOServerCnxn;
+import org.apache.zookeeper.server.ServerCnxnFactory;
 import org.apache.zookeeper.server.ZKDatabase;
 import org.apache.zookeeper.server.ZKDatabase;
 import org.apache.zookeeper.server.ZooKeeperServerMain;
 import org.apache.zookeeper.server.ZooKeeperServerMain;
 import org.apache.zookeeper.server.persistence.FileTxnSnapLog;
 import org.apache.zookeeper.server.persistence.FileTxnSnapLog;
@@ -118,9 +118,9 @@ public class QuorumPeerMain {
   
   
       LOG.info("Starting quorum peer");
       LOG.info("Starting quorum peer");
       try {
       try {
-          NIOServerCnxn.Factory cnxnFactory =
-              new NIOServerCnxn.Factory(config.getClientPortAddress(),
-                      config.getMaxClientCnxns());
+          ServerCnxnFactory cnxnFactory = ServerCnxnFactory.createFactory();
+          cnxnFactory.configure(config.getClientPortAddress(),
+                                config.getMaxClientCnxns());
   
   
           quorumPeer = new QuorumPeer();
           quorumPeer = new QuorumPeer();
           quorumPeer.setClientPortAddress(config.getClientPortAddress());
           quorumPeer.setClientPortAddress(config.getClientPortAddress());

+ 1 - 3
src/java/test/org/apache/zookeeper/server/CRCTest.java

@@ -26,7 +26,6 @@ import java.io.FileInputStream;
 import java.io.IOException;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.InputStream;
 import java.io.RandomAccessFile;
 import java.io.RandomAccessFile;
-import java.net.InetSocketAddress;
 import java.util.List;
 import java.util.List;
 import java.util.Map;
 import java.util.Map;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentHashMap;
@@ -116,8 +115,7 @@ public class CRCTest extends ZKTestCase implements Watcher {
         ZooKeeperServer zks = new ZooKeeperServer(tmpDir, tmpDir, 3000);
         ZooKeeperServer zks = new ZooKeeperServer(tmpDir, tmpDir, 3000);
         SyncRequestProcessor.setSnapCount(150);
         SyncRequestProcessor.setSnapCount(150);
         final int PORT = Integer.parseInt(HOSTPORT.split(":")[1]);
         final int PORT = Integer.parseInt(HOSTPORT.split(":")[1]);
-        NIOServerCnxn.Factory f = new NIOServerCnxn.Factory(
-                new InetSocketAddress(PORT));
+        ServerCnxnFactory f = ServerCnxnFactory.createFactory(PORT, -1);
         f.startup(zks);
         f.startup(zks);
         LOG.info("starting up the zookeeper server .. waiting");
         LOG.info("starting up the zookeeper server .. waiting");
         Assert.assertTrue("waiting for server being up",
         Assert.assertTrue("waiting for server being up",

+ 2 - 4
src/java/test/org/apache/zookeeper/server/InvalidSnapshotTest.java

@@ -20,7 +20,6 @@ package org.apache.zookeeper.server;
 
 
 import java.io.File;
 import java.io.File;
 import java.io.RandomAccessFile;
 import java.io.RandomAccessFile;
-import java.net.InetSocketAddress;
 
 
 import org.apache.log4j.Logger;
 import org.apache.log4j.Logger;
 import org.apache.zookeeper.CreateMode;
 import org.apache.zookeeper.CreateMode;
@@ -60,8 +59,7 @@ public class InvalidSnapshotTest extends ZKTestCase implements Watcher {
        ZooKeeperServer zks = new ZooKeeperServer(tmpDir, tmpDir, 3000);
        ZooKeeperServer zks = new ZooKeeperServer(tmpDir, tmpDir, 3000);
        SyncRequestProcessor.setSnapCount(100);
        SyncRequestProcessor.setSnapCount(100);
        final int PORT = Integer.parseInt(HOSTPORT.split(":")[1]);
        final int PORT = Integer.parseInt(HOSTPORT.split(":")[1]);
-       NIOServerCnxn.Factory f = new NIOServerCnxn.Factory(
-               new InetSocketAddress(PORT));
+       ServerCnxnFactory f = ServerCnxnFactory.createFactory(PORT, -1);
        f.startup(zks);
        f.startup(zks);
        Assert.assertTrue("waiting for server being up ",
        Assert.assertTrue("waiting for server being up ",
                ClientBase.waitForServerUp(HOSTPORT,CONNECTION_TIMEOUT));
                ClientBase.waitForServerUp(HOSTPORT,CONNECTION_TIMEOUT));
@@ -85,7 +83,7 @@ public class InvalidSnapshotTest extends ZKTestCase implements Watcher {
        // now restart the server and see if it starts
        // now restart the server and see if it starts
        zks = new ZooKeeperServer(tmpDir, tmpDir, 3000);
        zks = new ZooKeeperServer(tmpDir, tmpDir, 3000);
        SyncRequestProcessor.setSnapCount(100);
        SyncRequestProcessor.setSnapCount(100);
-       f = new NIOServerCnxn.Factory(new InetSocketAddress(PORT));
+       f = ServerCnxnFactory.createFactory(PORT, -1);
        f.startup(zks);
        f.startup(zks);
        Assert.assertTrue("waiting for server being up ",
        Assert.assertTrue("waiting for server being up ",
                ClientBase.waitForServerUp(HOSTPORT,CONNECTION_TIMEOUT));
                ClientBase.waitForServerUp(HOSTPORT,CONNECTION_TIMEOUT));

+ 80 - 76
src/java/test/org/apache/zookeeper/test/ACLTest.java

@@ -21,7 +21,6 @@ package org.apache.zookeeper.test;
 import static org.apache.zookeeper.test.ClientBase.CONNECTION_TIMEOUT;
 import static org.apache.zookeeper.test.ClientBase.CONNECTION_TIMEOUT;
 
 
 import java.io.File;
 import java.io.File;
-import java.net.InetSocketAddress;
 import java.util.ArrayList;
 import java.util.ArrayList;
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeUnit;
@@ -37,7 +36,7 @@ import org.apache.zookeeper.Watcher.Event.KeeperState;
 import org.apache.zookeeper.ZooDefs.Ids;
 import org.apache.zookeeper.ZooDefs.Ids;
 import org.apache.zookeeper.data.ACL;
 import org.apache.zookeeper.data.ACL;
 import org.apache.zookeeper.data.Id;
 import org.apache.zookeeper.data.Id;
-import org.apache.zookeeper.server.NIOServerCnxn;
+import org.apache.zookeeper.server.ServerCnxnFactory;
 import org.apache.zookeeper.server.SyncRequestProcessor;
 import org.apache.zookeeper.server.SyncRequestProcessor;
 import org.apache.zookeeper.server.ZooKeeperServer;
 import org.apache.zookeeper.server.ZooKeeperServer;
 import org.junit.Assert;
 import org.junit.Assert;
@@ -56,25 +55,26 @@ public class ACLTest extends ZKTestCase implements Watcher {
         ZooKeeperServer zks = new ZooKeeperServer(tmpDir, tmpDir, 3000);
         ZooKeeperServer zks = new ZooKeeperServer(tmpDir, tmpDir, 3000);
         SyncRequestProcessor.setSnapCount(1000);
         SyncRequestProcessor.setSnapCount(1000);
         final int PORT = Integer.parseInt(HOSTPORT.split(":")[1]);
         final int PORT = Integer.parseInt(HOSTPORT.split(":")[1]);
-        NIOServerCnxn.Factory f = new NIOServerCnxn.Factory(
-                new InetSocketAddress(PORT));
+        ServerCnxnFactory f = ServerCnxnFactory.createFactory(PORT, -1);
         f.startup(zks);
         f.startup(zks);
-        LOG.info("starting up the zookeeper server .. waiting");
-        Assert.assertTrue("waiting for server being up",
-                ClientBase.waitForServerUp(HOSTPORT, CONNECTION_TIMEOUT));
-        ZooKeeper zk = new ZooKeeper(HOSTPORT, CONNECTION_TIMEOUT, this);
         try {
         try {
-            zk.addAuthInfo("digest", "pat:test".getBytes());
-            zk.setACL("/", Ids.CREATOR_ALL_ACL, -1);
+            LOG.info("starting up the zookeeper server .. waiting");
+            Assert.assertTrue("waiting for server being up",
+                    ClientBase.waitForServerUp(HOSTPORT, CONNECTION_TIMEOUT));
+            ZooKeeper zk = new ZooKeeper(HOSTPORT, CONNECTION_TIMEOUT, this);
+            try {
+                zk.addAuthInfo("digest", "pat:test".getBytes());
+                zk.setACL("/", Ids.CREATOR_ALL_ACL, -1);
+            } finally {
+                zk.close();
+            }
         } finally {
         } finally {
-            zk.close();
-        }
+            f.shutdown();
 
 
-        f.shutdown();
-
-        Assert.assertTrue("waiting for server down",
-                   ClientBase.waitForServerDown(HOSTPORT,
-                           ClientBase.CONNECTION_TIMEOUT));
+            Assert.assertTrue("waiting for server down",
+                    ClientBase.waitForServerDown(HOSTPORT,
+                            ClientBase.CONNECTION_TIMEOUT));
+        }
     }
     }
 
 
     /**
     /**
@@ -89,74 +89,78 @@ public class ACLTest extends ZKTestCase implements Watcher {
         ZooKeeperServer zks = new ZooKeeperServer(tmpDir, tmpDir, 3000);
         ZooKeeperServer zks = new ZooKeeperServer(tmpDir, tmpDir, 3000);
         SyncRequestProcessor.setSnapCount(1000);
         SyncRequestProcessor.setSnapCount(1000);
         final int PORT = Integer.parseInt(HOSTPORT.split(":")[1]);
         final int PORT = Integer.parseInt(HOSTPORT.split(":")[1]);
-        NIOServerCnxn.Factory f = new NIOServerCnxn.Factory(
-                new InetSocketAddress(PORT));
+        ServerCnxnFactory f = ServerCnxnFactory.createFactory(PORT, -1);
         f.startup(zks);
         f.startup(zks);
-        LOG.info("starting up the zookeeper server .. waiting");
-        Assert.assertTrue("waiting for server being up",
-                ClientBase.waitForServerUp(HOSTPORT, CONNECTION_TIMEOUT));
-        ZooKeeper zk = new ZooKeeper(HOSTPORT, CONNECTION_TIMEOUT, this);
+        ZooKeeper zk;
         String path;
         String path;
-        LOG.info("starting creating acls");
-        for (int i = 0; i < 100; i++) {
-            path = "/" + i;
-            zk.create(path, path.getBytes(), Ids.OPEN_ACL_UNSAFE,
-                    CreateMode.PERSISTENT);
-        }
-        Assert.assertTrue("size of the acl map ", (1 == zks.getZKDatabase().getAclSize()));
-        for (int j = 100; j < 200; j++) {
-            path = "/" + j;
-            ACL acl = new ACL();
-            acl.setPerms(0);
-            Id id = new Id();
-            id.setId("1.1.1."+j);
-            id.setScheme("ip");
-            acl.setId(id);
-            ArrayList<ACL> list = new ArrayList<ACL>();
-            list.add(acl);
-            zk.create(path, path.getBytes(), list, CreateMode.PERSISTENT);
+        try {
+            LOG.info("starting up the zookeeper server .. waiting");
+            Assert.assertTrue("waiting for server being up",
+                    ClientBase.waitForServerUp(HOSTPORT, CONNECTION_TIMEOUT));
+            zk = new ZooKeeper(HOSTPORT, CONNECTION_TIMEOUT, this);
+            LOG.info("starting creating acls");
+            for (int i = 0; i < 100; i++) {
+                path = "/" + i;
+                zk.create(path, path.getBytes(), Ids.OPEN_ACL_UNSAFE,
+                        CreateMode.PERSISTENT);
+            }
+            Assert.assertTrue("size of the acl map ", (1 == zks.getZKDatabase().getAclSize()));
+            for (int j = 100; j < 200; j++) {
+                path = "/" + j;
+                ACL acl = new ACL();
+                acl.setPerms(0);
+                Id id = new Id();
+                id.setId("1.1.1."+j);
+                id.setScheme("ip");
+                acl.setId(id);
+                ArrayList<ACL> list = new ArrayList<ACL>();
+                list.add(acl);
+                zk.create(path, path.getBytes(), list, CreateMode.PERSISTENT);
+            }
+            Assert.assertTrue("size of the acl map ", (101 == zks.getZKDatabase().getAclSize()));
+        } finally {
+            // now shutdown the server and restart it
+            f.shutdown();
+            Assert.assertTrue("waiting for server down",
+                    ClientBase.waitForServerDown(HOSTPORT, CONNECTION_TIMEOUT));
         }
         }
-        Assert.assertTrue("size of the acl map ", (101 == zks.getZKDatabase().getAclSize()));
-        // now shutdown the server and restart it
-        f.shutdown();
-        Assert.assertTrue("waiting for server down",
-                ClientBase.waitForServerDown(HOSTPORT, CONNECTION_TIMEOUT));
         startSignal = new CountDownLatch(1);
         startSignal = new CountDownLatch(1);
 
 
         zks = new ZooKeeperServer(tmpDir, tmpDir, 3000);
         zks = new ZooKeeperServer(tmpDir, tmpDir, 3000);
-        f = new NIOServerCnxn.Factory(new InetSocketAddress(PORT));
+        f = ServerCnxnFactory.createFactory(PORT, -1);
 
 
         f.startup(zks);
         f.startup(zks);
-
-        Assert.assertTrue("waiting for server up",
-                   ClientBase.waitForServerUp(HOSTPORT, CONNECTION_TIMEOUT));
-
-        startSignal.await(CONNECTION_TIMEOUT,
-                TimeUnit.MILLISECONDS);
-        Assert.assertTrue("count == 0", startSignal.getCount() == 0);
-
-        Assert.assertTrue("acl map ", (101 == zks.getZKDatabase().getAclSize()));
-        for (int j = 200; j < 205; j++) {
-            path = "/" + j;
-            ACL acl = new ACL();
-            acl.setPerms(0);
-            Id id = new Id();
-            id.setId("1.1.1."+j);
-            id.setScheme("ip");
-            acl.setId(id);
-            ArrayList<ACL> list = new ArrayList<ACL>();
-            list.add(acl);
-            zk.create(path, path.getBytes(), list, CreateMode.PERSISTENT);
+        try {
+            Assert.assertTrue("waiting for server up",
+                       ClientBase.waitForServerUp(HOSTPORT, CONNECTION_TIMEOUT));
+    
+            startSignal.await(CONNECTION_TIMEOUT,
+                    TimeUnit.MILLISECONDS);
+            Assert.assertTrue("count == 0", startSignal.getCount() == 0);
+    
+            Assert.assertTrue("acl map ", (101 == zks.getZKDatabase().getAclSize()));
+            for (int j = 200; j < 205; j++) {
+                path = "/" + j;
+                ACL acl = new ACL();
+                acl.setPerms(0);
+                Id id = new Id();
+                id.setId("1.1.1."+j);
+                id.setScheme("ip");
+                acl.setId(id);
+                ArrayList<ACL> list = new ArrayList<ACL>();
+                list.add(acl);
+                zk.create(path, path.getBytes(), list, CreateMode.PERSISTENT);
+            }
+            Assert.assertTrue("acl map ", (106 == zks.getZKDatabase().getAclSize()));
+    
+            zk.close();
+        } finally {
+            f.shutdown();
+    
+            Assert.assertTrue("waiting for server down",
+                       ClientBase.waitForServerDown(HOSTPORT,
+                               ClientBase.CONNECTION_TIMEOUT));
         }
         }
-        Assert.assertTrue("acl map ", (106 == zks.getZKDatabase().getAclSize()));
-
-        zk.close();
-
-        f.shutdown();
-
-        Assert.assertTrue("waiting for server down",
-                   ClientBase.waitForServerDown(HOSTPORT,
-                           ClientBase.CONNECTION_TIMEOUT));
 
 
     }
     }
 
 

+ 0 - 2
src/java/test/org/apache/zookeeper/test/AsyncHammerTest.java

@@ -35,9 +35,7 @@ import org.apache.zookeeper.AsyncCallback.VoidCallback;
 import org.apache.zookeeper.ZooDefs.Ids;
 import org.apache.zookeeper.ZooDefs.Ids;
 import org.apache.zookeeper.data.Stat;
 import org.apache.zookeeper.data.Stat;
 import org.apache.zookeeper.test.ClientBase.CountdownWatcher;
 import org.apache.zookeeper.test.ClientBase.CountdownWatcher;
-import org.junit.After;
 import org.junit.Assert;
 import org.junit.Assert;
-import org.junit.Before;
 import org.junit.Test;
 import org.junit.Test;
 
 
 public class AsyncHammerTest extends ZKTestCase
 public class AsyncHammerTest extends ZKTestCase

+ 58 - 36
src/java/test/org/apache/zookeeper/test/ClientBase.java

@@ -25,7 +25,6 @@ import java.io.InputStreamReader;
 import java.io.OutputStream;
 import java.io.OutputStream;
 import java.lang.management.ManagementFactory;
 import java.lang.management.ManagementFactory;
 import java.lang.management.OperatingSystemMXBean;
 import java.lang.management.OperatingSystemMXBean;
-import java.net.InetSocketAddress;
 import java.net.Socket;
 import java.net.Socket;
 import java.util.ArrayList;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Arrays;
@@ -45,10 +44,10 @@ import org.apache.zookeeper.PortAssignment;
 import org.apache.zookeeper.TestableZooKeeper;
 import org.apache.zookeeper.TestableZooKeeper;
 import org.apache.zookeeper.WatchedEvent;
 import org.apache.zookeeper.WatchedEvent;
 import org.apache.zookeeper.Watcher;
 import org.apache.zookeeper.Watcher;
+import org.apache.zookeeper.Watcher.Event.KeeperState;
 import org.apache.zookeeper.ZKTestCase;
 import org.apache.zookeeper.ZKTestCase;
 import org.apache.zookeeper.ZooKeeper;
 import org.apache.zookeeper.ZooKeeper;
-import org.apache.zookeeper.Watcher.Event.KeeperState;
-import org.apache.zookeeper.server.NIOServerCnxn;
+import org.apache.zookeeper.server.ServerCnxnFactory;
 import org.apache.zookeeper.server.ZKDatabase;
 import org.apache.zookeeper.server.ZKDatabase;
 import org.apache.zookeeper.server.ZooKeeperServer;
 import org.apache.zookeeper.server.ZooKeeperServer;
 import org.apache.zookeeper.server.persistence.FileTxnLog;
 import org.apache.zookeeper.server.persistence.FileTxnLog;
@@ -56,6 +55,7 @@ import org.junit.After;
 import org.junit.Assert;
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Before;
 
 
+import com.j2speed.accessor.FieldAccessor;
 import com.sun.management.UnixOperatingSystemMXBean;
 import com.sun.management.UnixOperatingSystemMXBean;
 
 
 public abstract class ClientBase extends ZKTestCase {
 public abstract class ClientBase extends ZKTestCase {
@@ -67,8 +67,11 @@ public abstract class ClientBase extends ZKTestCase {
 
 
     protected String hostPort = "127.0.0.1:" + PortAssignment.unique();
     protected String hostPort = "127.0.0.1:" + PortAssignment.unique();
     protected int maxCnxns = 0;
     protected int maxCnxns = 0;
-    protected NIOServerCnxn.Factory serverFactory = null;
+    protected ServerCnxnFactory serverFactory = null;
     protected File tmpDir = null;
     protected File tmpDir = null;
+    
+    long initialFdCount;
+    
     public ClientBase() {
     public ClientBase() {
         super();
         super();
     }
     }
@@ -330,14 +333,14 @@ public abstract class ClientBase extends ZKTestCase {
         return Integer.parseInt(portstr);
         return Integer.parseInt(portstr);
     }
     }
 
 
-    static NIOServerCnxn.Factory createNewServerInstance(File dataDir,
-            NIOServerCnxn.Factory factory, String hostPort, int maxCnxns)
+    static ServerCnxnFactory createNewServerInstance(File dataDir,
+            ServerCnxnFactory factory, String hostPort, int maxCnxns)
         throws IOException, InterruptedException
         throws IOException, InterruptedException
     {
     {
         ZooKeeperServer zks = new ZooKeeperServer(dataDir, dataDir, 3000);
         ZooKeeperServer zks = new ZooKeeperServer(dataDir, dataDir, 3000);
         final int PORT = getPort(hostPort);
         final int PORT = getPort(hostPort);
         if (factory == null) {
         if (factory == null) {
-            factory = new NIOServerCnxn.Factory(new InetSocketAddress(PORT),maxCnxns);
+            factory = ServerCnxnFactory.createFactory(PORT, maxCnxns);
         }
         }
         factory.startup(zks);
         factory.startup(zks);
         Assert.assertTrue("waiting for server up",
         Assert.assertTrue("waiting for server up",
@@ -347,11 +350,16 @@ public abstract class ClientBase extends ZKTestCase {
         return factory;
         return factory;
     }
     }
 
 
-    static void shutdownServerInstance(NIOServerCnxn.Factory factory,
+    static void shutdownServerInstance(ServerCnxnFactory factory,
             String hostPort)
             String hostPort)
     {
     {
         if (factory != null) {
         if (factory != null) {
-            ZKDatabase zkDb = factory.getZooKeeperServer().getZKDatabase();
+            ZKDatabase zkDb;
+            {
+                ZooKeeperServer zs = getServer(factory);
+        
+                zkDb = zs.getZKDatabase();
+            }
             factory.shutdown();
             factory.shutdown();
             try {
             try {
                 zkDb.close();
                 zkDb.close();
@@ -385,16 +393,6 @@ public abstract class ClientBase extends ZKTestCase {
 
 
     @Before
     @Before
     public void setUp() throws Exception {
     public void setUp() throws Exception {
-        setupTestEnv();
-
-        JMXEnv.setUp();
-
-        setUpAll();
-
-        tmpDir = createTmpDir(BASETEST);
-
-        startServer();
-
         /* some useful information - log the number of fds used before
         /* some useful information - log the number of fds used before
          * and after a test is run. Helps to verify we are freeing resources
          * and after a test is run. Helps to verify we are freeing resources
          * correctly. Unfortunately this only works on unix systems (the
          * correctly. Unfortunately this only works on unix systems (the
@@ -405,10 +403,21 @@ public abstract class ClientBase extends ZKTestCase {
         if (osMbean != null && osMbean instanceof UnixOperatingSystemMXBean) {
         if (osMbean != null && osMbean instanceof UnixOperatingSystemMXBean) {
             UnixOperatingSystemMXBean unixos =
             UnixOperatingSystemMXBean unixos =
                 (UnixOperatingSystemMXBean)osMbean;
                 (UnixOperatingSystemMXBean)osMbean;
+            initialFdCount = unixos.getOpenFileDescriptorCount();
             LOG.info("Initial fdcount is: "
             LOG.info("Initial fdcount is: "
-                    + unixos.getOpenFileDescriptorCount());
+                    + initialFdCount);
         }
         }
 
 
+        setupTestEnv();
+
+        JMXEnv.setUp();
+
+        setUpAll();
+
+        tmpDir = createTmpDir(BASETEST);
+
+        startServer();
+
         LOG.info("Client test setup finished");
         LOG.info("Client test setup finished");
     }
     }
 
 
@@ -427,8 +436,14 @@ public abstract class ClientBase extends ZKTestCase {
         JMXEnv.ensureOnly();
         JMXEnv.ensureOnly();
     }
     }
 
 
-    protected ZooKeeperServer getServer() {
-        return serverFactory.getZooKeeperServer();
+    protected static ZooKeeperServer getServer(ServerCnxnFactory fac) {
+        // access the private field - test only
+        FieldAccessor<ServerCnxnFactory,ZooKeeperServer> zkServerAcc =
+            new FieldAccessor<ServerCnxnFactory,ZooKeeperServer>
+                    ("zkServer", ServerCnxnFactory.class);
+        ZooKeeperServer zs = zkServerAcc.get(fac);
+
+        return zs;
     }
     }
 
 
     protected void tearDownAll() throws Exception {
     protected void tearDownAll() throws Exception {
@@ -449,20 +464,6 @@ public abstract class ClientBase extends ZKTestCase {
     public void tearDown() throws Exception {
     public void tearDown() throws Exception {
         LOG.info("tearDown starting");
         LOG.info("tearDown starting");
 
 
-        /* some useful information - log the number of fds used before
-         * and after a test is run. Helps to verify we are freeing resources
-         * correctly. Unfortunately this only works on unix systems (the
-         * only place sun has implemented as part of the mgmt bean api.
-         */
-        OperatingSystemMXBean osMbean =
-            ManagementFactory.getOperatingSystemMXBean();
-        if (osMbean != null && osMbean instanceof UnixOperatingSystemMXBean) {
-            UnixOperatingSystemMXBean unixos =
-                (UnixOperatingSystemMXBean)osMbean;
-            LOG.info("fdcount after test is: "
-                    + unixos.getOpenFileDescriptorCount());
-        }
-
         tearDownAll();
         tearDownAll();
 
 
         stopServer();
         stopServer();
@@ -475,6 +476,27 @@ public abstract class ClientBase extends ZKTestCase {
         serverFactory = null;
         serverFactory = null;
 
 
         JMXEnv.tearDown();
         JMXEnv.tearDown();
+
+        /* some useful information - log the number of fds used before
+         * and after a test is run. Helps to verify we are freeing resources
+         * correctly. Unfortunately this only works on unix systems (the
+         * only place sun has implemented as part of the mgmt bean api.
+         */
+        OperatingSystemMXBean osMbean =
+            ManagementFactory.getOperatingSystemMXBean();
+        if (osMbean != null && osMbean instanceof UnixOperatingSystemMXBean) {
+            UnixOperatingSystemMXBean unixos =
+                (UnixOperatingSystemMXBean)osMbean;
+            long fdCount = unixos.getOpenFileDescriptorCount();
+            String message = "fdcount after test is: "
+                    + fdCount + " at start it was " + initialFdCount;
+            LOG.info(message);
+            if (fdCount > initialFdCount) {
+                LOG.info("sleeping for 20 secs");
+                //Thread.sleep(60000);
+                //assertTrue(message, fdCount <= initialFdCount);
+            }
+        }
     }
     }
 
 
     public static MBeanServerConnection jmxConn() throws IOException {
     public static MBeanServerConnection jmxConn() throws IOException {

+ 3 - 3
src/java/test/org/apache/zookeeper/test/ClientPortBindTest.java

@@ -35,7 +35,7 @@ import org.apache.zookeeper.Watcher;
 import org.apache.zookeeper.ZKTestCase;
 import org.apache.zookeeper.ZKTestCase;
 import org.apache.zookeeper.ZooKeeper;
 import org.apache.zookeeper.ZooKeeper;
 import org.apache.zookeeper.Watcher.Event.KeeperState;
 import org.apache.zookeeper.Watcher.Event.KeeperState;
-import org.apache.zookeeper.server.NIOServerCnxn;
+import org.apache.zookeeper.server.ServerCnxnFactory;
 import org.apache.zookeeper.server.ZooKeeperServer;
 import org.apache.zookeeper.server.ZooKeeperServer;
 import org.junit.Assert;
 import org.junit.Assert;
 import org.junit.Test;
 import org.junit.Test;
@@ -82,8 +82,8 @@ public class ClientPortBindTest extends ZKTestCase implements Watcher {
         ClientBase.setupTestEnv();
         ClientBase.setupTestEnv();
         ZooKeeperServer zks = new ZooKeeperServer(tmpDir, tmpDir, 3000);
         ZooKeeperServer zks = new ZooKeeperServer(tmpDir, tmpDir, 3000);
 
 
-        NIOServerCnxn.Factory f = new NIOServerCnxn.Factory(
-                new InetSocketAddress(bindAddress, PORT));
+        ServerCnxnFactory f = ServerCnxnFactory.createFactory(
+                new InetSocketAddress(bindAddress, PORT), -1);
         f.startup(zks);
         f.startup(zks);
         LOG.info("starting up the the server, waiting");
         LOG.info("starting up the the server, waiting");
 
 

+ 9 - 0
src/java/test/org/apache/zookeeper/test/DisconnectableZooKeeper.java

@@ -46,4 +46,13 @@ public class DisconnectableZooKeeper extends ZooKeeper {
         cnxn.disconnect();
         cnxn.disconnect();
     }
     }
 
 
+    /**
+     * Prevent the client from automatically reconnecting if the connection to the
+     * server is lost
+     */
+    public void dontReconnect() throws Exception {
+        java.lang.reflect.Field f = cnxn.getClass().getDeclaredField("closing");
+        f.setAccessible(true);
+        f.setBoolean(cnxn, true);
+    }
 }
 }

+ 2 - 4
src/java/test/org/apache/zookeeper/test/InvalidSnapshotTest.java

@@ -21,7 +21,6 @@ package org.apache.zookeeper.test;
 import static org.apache.zookeeper.test.ClientBase.CONNECTION_TIMEOUT;
 import static org.apache.zookeeper.test.ClientBase.CONNECTION_TIMEOUT;
 
 
 import java.io.File;
 import java.io.File;
-import java.net.InetSocketAddress;
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.CountDownLatch;
 
 
 import org.apache.log4j.Logger;
 import org.apache.log4j.Logger;
@@ -32,7 +31,7 @@ import org.apache.zookeeper.ZKTestCase;
 import org.apache.zookeeper.ZooKeeper;
 import org.apache.zookeeper.ZooKeeper;
 import org.apache.zookeeper.Watcher.Event.KeeperState;
 import org.apache.zookeeper.Watcher.Event.KeeperState;
 import org.apache.zookeeper.server.LogFormatter;
 import org.apache.zookeeper.server.LogFormatter;
-import org.apache.zookeeper.server.NIOServerCnxn;
+import org.apache.zookeeper.server.ServerCnxnFactory;
 import org.apache.zookeeper.server.SyncRequestProcessor;
 import org.apache.zookeeper.server.SyncRequestProcessor;
 import org.apache.zookeeper.server.ZooKeeperServer;
 import org.apache.zookeeper.server.ZooKeeperServer;
 import org.junit.Assert;
 import org.junit.Assert;
@@ -68,8 +67,7 @@ public class InvalidSnapshotTest extends ZKTestCase implements Watcher {
         ZooKeeperServer zks = new ZooKeeperServer(snapDir, snapDir, 3000);
         ZooKeeperServer zks = new ZooKeeperServer(snapDir, snapDir, 3000);
         SyncRequestProcessor.setSnapCount(1000);
         SyncRequestProcessor.setSnapCount(1000);
         final int PORT = Integer.parseInt(HOSTPORT.split(":")[1]);
         final int PORT = Integer.parseInt(HOSTPORT.split(":")[1]);
-        NIOServerCnxn.Factory f = new NIOServerCnxn.Factory(
-                new InetSocketAddress(PORT));
+        ServerCnxnFactory f = ServerCnxnFactory.createFactory(PORT, -1);
         f.startup(zks);
         f.startup(zks);
         LOG.info("starting up the zookeeper server .. waiting");
         LOG.info("starting up the zookeeper server .. waiting");
         Assert.assertTrue("waiting for server being up",
         Assert.assertTrue("waiting for server being up",

+ 6 - 3
src/java/test/org/apache/zookeeper/test/LENonTerminateTest.java

@@ -25,6 +25,7 @@ import java.net.DatagramSocket;
 import java.net.InetSocketAddress;
 import java.net.InetSocketAddress;
 import java.net.SocketException;
 import java.net.SocketException;
 import java.nio.ByteBuffer;
 import java.nio.ByteBuffer;
+import java.util.Collection;
 import java.util.HashMap;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.HashSet;
 import java.util.Map;
 import java.util.Map;
@@ -34,7 +35,7 @@ import java.util.concurrent.TimeUnit;
 import org.apache.log4j.Logger;
 import org.apache.log4j.Logger;
 import org.apache.zookeeper.PortAssignment;
 import org.apache.zookeeper.PortAssignment;
 import org.apache.zookeeper.ZKTestCase;
 import org.apache.zookeeper.ZKTestCase;
-import org.apache.zookeeper.server.NIOServerCnxn;
+import org.apache.zookeeper.server.ServerCnxnFactory;
 import org.apache.zookeeper.server.quorum.Election;
 import org.apache.zookeeper.server.quorum.Election;
 import org.apache.zookeeper.server.quorum.LeaderElection;
 import org.apache.zookeeper.server.quorum.LeaderElection;
 import org.apache.zookeeper.server.quorum.QuorumPeer;
 import org.apache.zookeeper.server.quorum.QuorumPeer;
@@ -93,7 +94,9 @@ public class LENonTerminateTest extends ZKTestCase {
                 requestBuffer.putInt(xid);
                 requestBuffer.putInt(xid);
                 requestPacket.setLength(4);
                 requestPacket.setLength(4);
                 HashSet<Long> heardFrom = new HashSet<Long>();
                 HashSet<Long> heardFrom = new HashSet<Long>();
-                for (QuorumServer server : self.getVotingView().values()) {
+                for (QuorumServer server :
+                    (Collection<QuorumServer>)self.getVotingView().values())
+                {
                     LOG.info("Server address: " + server.addr);
                     LOG.info("Server address: " + server.addr);
                     try {
                     try {
                         requestPacket.setSocketAddress(server.addr);
                         requestPacket.setSocketAddress(server.addr);
@@ -213,7 +216,7 @@ public class LENonTerminateTest extends ZKTestCase {
         {
         {
             super(quorumPeers, snapDir, logDir, electionAlg,
             super(quorumPeers, snapDir, logDir, electionAlg,
                     myid,tickTime, initLimit,syncLimit,
                     myid,tickTime, initLimit,syncLimit,
-                    new NIOServerCnxn.Factory(new InetSocketAddress(clientPort)),
+                    ServerCnxnFactory.createFactory(clientPort, -1),
                     new QuorumMaj(countParticipants(quorumPeers)));
                     new QuorumMaj(countParticipants(quorumPeers)));
         }
         }
         
         

+ 43 - 0
src/java/test/org/apache/zookeeper/test/NioNettySuiteBase.java

@@ -0,0 +1,43 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.zookeeper.test;
+
+import org.apache.zookeeper.server.NettyServerCnxnFactory;
+import org.apache.zookeeper.server.ServerCnxnFactory;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.runner.RunWith;
+import org.junit.runners.Suite;
+
+/**
+ * Run tests with: Nio Client against Netty server
+ */
+@RunWith(Suite.class)
+public class NioNettySuiteBase {
+    @BeforeClass
+    public static void setUp() {
+        System.setProperty(ServerCnxnFactory.ZOOKEEPER_SERVER_CNXN_FACTORY,
+                NettyServerCnxnFactory.class.getName());
+    }
+
+    @AfterClass
+    public static void tearDown() {
+        System.clearProperty(ServerCnxnFactory.ZOOKEEPER_SERVER_CNXN_FACTORY);
+    }
+}

+ 30 - 0
src/java/test/org/apache/zookeeper/test/NioNettySuiteHammerTest.java

@@ -0,0 +1,30 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.zookeeper.test;
+
+import org.junit.runners.Suite;
+
+/**
+ * Run tests with: Nio Client against Netty server
+ */
+@Suite.SuiteClasses({
+        AsyncHammerTest.class
+        })
+public class NioNettySuiteHammerTest extends NioNettySuiteBase {
+}

+ 37 - 0
src/java/test/org/apache/zookeeper/test/NioNettySuiteTest.java

@@ -0,0 +1,37 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.zookeeper.test;
+
+import org.junit.runners.Suite;
+
+/**
+ * Run tests with: Nio Client against Netty server
+ */
+@Suite.SuiteClasses({
+        ACLTest.class,
+        AsyncOpsTest.class,
+        ChrootClientTest.class,
+        ClientTest.class,
+        FourLetterWordsTest.class,
+        NullDataTest.class,
+        SessionTest.class,
+        WatcherTest.class
+        })
+public class NioNettySuiteTest extends NioNettySuiteBase {
+}

+ 3 - 5
src/java/test/org/apache/zookeeper/test/OOMTest.java

@@ -22,7 +22,6 @@ import static org.apache.zookeeper.test.ClientBase.CONNECTION_TIMEOUT;
 
 
 import java.io.File;
 import java.io.File;
 import java.io.IOException;
 import java.io.IOException;
-import java.net.InetSocketAddress;
 import java.util.ArrayList;
 import java.util.ArrayList;
 
 
 import org.apache.zookeeper.CreateMode;
 import org.apache.zookeeper.CreateMode;
@@ -34,7 +33,7 @@ import org.apache.zookeeper.ZKTestCase;
 import org.apache.zookeeper.ZooKeeper;
 import org.apache.zookeeper.ZooKeeper;
 import org.apache.zookeeper.ZooDefs.Ids;
 import org.apache.zookeeper.ZooDefs.Ids;
 import org.apache.zookeeper.data.Stat;
 import org.apache.zookeeper.data.Stat;
-import org.apache.zookeeper.server.NIOServerCnxn;
+import org.apache.zookeeper.server.ServerCnxnFactory;
 import org.apache.zookeeper.server.ZooKeeperServer;
 import org.apache.zookeeper.server.ZooKeeperServer;
 import org.junit.Assert;
 import org.junit.Assert;
 import org.junit.Test;
 import org.junit.Test;
@@ -42,7 +41,7 @@ import org.junit.Test;
 public class OOMTest extends ZKTestCase implements Watcher {
 public class OOMTest extends ZKTestCase implements Watcher {
     @Test
     @Test
     public void testOOM() throws IOException, InterruptedException, KeeperException {
     public void testOOM() throws IOException, InterruptedException, KeeperException {
-        // This test takes too long to run!
+        // This test takes too long tos run!
         if (true)
         if (true)
             return;
             return;
         File tmpDir = ClientBase.createTmpDir();
         File tmpDir = ClientBase.createTmpDir();
@@ -61,8 +60,7 @@ public class OOMTest extends ZKTestCase implements Watcher {
         ZooKeeperServer zks = new ZooKeeperServer(tmpDir, tmpDir, 3000);
         ZooKeeperServer zks = new ZooKeeperServer(tmpDir, tmpDir, 3000);
 
 
         final int PORT = PortAssignment.unique();
         final int PORT = PortAssignment.unique();
-        NIOServerCnxn.Factory f = new NIOServerCnxn.Factory(
-                new InetSocketAddress(PORT));
+        ServerCnxnFactory f = ServerCnxnFactory.createFactory(PORT, -1);
         f.startup(zks);
         f.startup(zks);
         Assert.assertTrue("waiting for server up",
         Assert.assertTrue("waiting for server up",
                    ClientBase.waitForServerUp("127.0.0.1:" + PORT,
                    ClientBase.waitForServerUp("127.0.0.1:" + PORT,

+ 15 - 1
src/java/test/org/apache/zookeeper/test/ObserverTest.java

@@ -111,18 +111,23 @@ public class ObserverTest extends QuorumPeerTestBase implements Watcher{
         
         
         Assert.assertEquals(zk.getState(), States.CONNECTED);
         Assert.assertEquals(zk.getState(), States.CONNECTED);
         
         
+        LOG.info("Shutting down server 2");
         // Now kill one of the other real servers        
         // Now kill one of the other real servers        
         q2.shutdown();
         q2.shutdown();
                 
                 
         Assert.assertTrue("Waiting for server 2 to shut down",
         Assert.assertTrue("Waiting for server 2 to shut down",
                     ClientBase.waitForServerDown("127.0.0.1:"+CLIENT_PORT_QP2, 
                     ClientBase.waitForServerDown("127.0.0.1:"+CLIENT_PORT_QP2, 
                                     ClientBase.CONNECTION_TIMEOUT));
                                     ClientBase.CONNECTION_TIMEOUT));
-        
+
+        LOG.info("Server 2 down");
+
         // Now the resulting ensemble shouldn't be quorate         
         // Now the resulting ensemble shouldn't be quorate         
         latch.await();        
         latch.await();        
         Assert.assertNotSame("Client is still connected to non-quorate cluster", 
         Assert.assertNotSame("Client is still connected to non-quorate cluster", 
                 KeeperState.SyncConnected,lastEvent.getState());
                 KeeperState.SyncConnected,lastEvent.getState());
 
 
+        LOG.info("Latch returned");
+
         try {
         try {
             Assert.assertFalse("Shouldn't get a response when cluster not quorate!",
             Assert.assertFalse("Shouldn't get a response when cluster not quorate!",
                     new String(zk.getData("/obstest", null, null)).equals("test"));
                     new String(zk.getData("/obstest", null, null)).equals("test"));
@@ -133,14 +138,19 @@ public class ObserverTest extends QuorumPeerTestBase implements Watcher{
         
         
         latch = new CountDownLatch(1);
         latch = new CountDownLatch(1);
 
 
+        LOG.info("Restarting server 2");
+
         // Bring it back
         // Bring it back
         q2 = new MainThread(2, CLIENT_PORT_QP2, quorumCfgSection);
         q2 = new MainThread(2, CLIENT_PORT_QP2, quorumCfgSection);
         q2.start();
         q2.start();
+        
         LOG.info("Waiting for server 2 to come up");
         LOG.info("Waiting for server 2 to come up");
         Assert.assertTrue("waiting for server 2 being up",
         Assert.assertTrue("waiting for server 2 being up",
                 ClientBase.waitForServerUp("127.0.0.1:" + CLIENT_PORT_QP2,
                 ClientBase.waitForServerUp("127.0.0.1:" + CLIENT_PORT_QP2,
                         CONNECTION_TIMEOUT));
                         CONNECTION_TIMEOUT));
         
         
+        LOG.info("Server 2 started, waiting for latch");
+
         latch.await();
         latch.await();
         // It's possible our session expired - but this is ok, shows we 
         // It's possible our session expired - but this is ok, shows we 
         // were able to talk to the ensemble
         // were able to talk to the ensemble
@@ -149,10 +159,14 @@ public class ObserverTest extends QuorumPeerTestBase implements Watcher{
                 (KeeperState.SyncConnected==lastEvent.getState() ||
                 (KeeperState.SyncConnected==lastEvent.getState() ||
                 KeeperState.Expired==lastEvent.getState())); 
                 KeeperState.Expired==lastEvent.getState())); 
 
 
+        LOG.info("Shutting down all servers");
+
         q1.shutdown();
         q1.shutdown();
         q2.shutdown();
         q2.shutdown();
         q3.shutdown();
         q3.shutdown();
         
         
+        LOG.info("Closing zk client");
+
         zk.close();        
         zk.close();        
         Assert.assertTrue("Waiting for server 1 to shut down",
         Assert.assertTrue("Waiting for server 1 to shut down",
                 ClientBase.waitForServerDown("127.0.0.1:"+CLIENT_PORT_QP1, 
                 ClientBase.waitForServerDown("127.0.0.1:"+CLIENT_PORT_QP1, 

+ 2 - 4
src/java/test/org/apache/zookeeper/test/PurgeTxnTest.java

@@ -19,7 +19,6 @@
 package org.apache.zookeeper.test;
 package org.apache.zookeeper.test;
 
 
 import java.io.File;
 import java.io.File;
-import java.net.InetSocketAddress;
 import java.util.List;
 import java.util.List;
 
 
 import org.apache.zookeeper.CreateMode;
 import org.apache.zookeeper.CreateMode;
@@ -29,8 +28,8 @@ import org.apache.zookeeper.Watcher;
 import org.apache.zookeeper.ZKTestCase;
 import org.apache.zookeeper.ZKTestCase;
 import org.apache.zookeeper.ZooKeeper;
 import org.apache.zookeeper.ZooKeeper;
 import org.apache.zookeeper.ZooDefs.Ids;
 import org.apache.zookeeper.ZooDefs.Ids;
-import org.apache.zookeeper.server.NIOServerCnxn;
 import org.apache.zookeeper.server.PurgeTxnLog;
 import org.apache.zookeeper.server.PurgeTxnLog;
+import org.apache.zookeeper.server.ServerCnxnFactory;
 import org.apache.zookeeper.server.SyncRequestProcessor;
 import org.apache.zookeeper.server.SyncRequestProcessor;
 import org.apache.zookeeper.server.ZooKeeperServer;
 import org.apache.zookeeper.server.ZooKeeperServer;
 import org.apache.zookeeper.server.persistence.FileTxnSnapLog;
 import org.apache.zookeeper.server.persistence.FileTxnSnapLog;
@@ -52,8 +51,7 @@ public class PurgeTxnTest extends ZKTestCase implements  Watcher {
         ZooKeeperServer zks = new ZooKeeperServer(tmpDir, tmpDir, 3000);
         ZooKeeperServer zks = new ZooKeeperServer(tmpDir, tmpDir, 3000);
         SyncRequestProcessor.setSnapCount(100);
         SyncRequestProcessor.setSnapCount(100);
         final int PORT = Integer.parseInt(HOSTPORT.split(":")[1]);
         final int PORT = Integer.parseInt(HOSTPORT.split(":")[1]);
-        NIOServerCnxn.Factory f = new NIOServerCnxn.Factory(
-                new InetSocketAddress(PORT));
+        ServerCnxnFactory f = ServerCnxnFactory.createFactory(PORT, -1);
         f.startup(zks);
         f.startup(zks);
         Assert.assertTrue("waiting for server being up ",
         Assert.assertTrue("waiting for server being up ",
                 ClientBase.waitForServerUp(HOSTPORT,CONNECTION_TIMEOUT));
                 ClientBase.waitForServerUp(HOSTPORT,CONNECTION_TIMEOUT));

+ 6 - 0
src/java/test/org/apache/zookeeper/test/QuorumBase.java

@@ -35,6 +35,7 @@ import org.apache.zookeeper.server.quorum.QuorumPeer;
 import org.apache.zookeeper.server.quorum.QuorumPeer.LearnerType;
 import org.apache.zookeeper.server.quorum.QuorumPeer.LearnerType;
 import org.apache.zookeeper.server.quorum.QuorumPeer.QuorumServer;
 import org.apache.zookeeper.server.quorum.QuorumPeer.QuorumServer;
 import org.junit.Assert;
 import org.junit.Assert;
+import org.junit.Test;
 
 
 import com.sun.management.UnixOperatingSystemMXBean;
 import com.sun.management.UnixOperatingSystemMXBean;
 
 
@@ -55,6 +56,11 @@ public class QuorumBase extends ClientBase {
     private int portLE4;
     private int portLE4;
     private int portLE5;
     private int portLE5;
 
 
+    @Test
+    // This just avoids complaints by junit
+    public void testNull() {
+    }
+    
     @Override
     @Override
     public void setUp() throws Exception {
     public void setUp() throws Exception {
         setUp(false);
         setUp(false);

+ 17 - 9
src/java/test/org/apache/zookeeper/test/QuorumTest.java

@@ -29,10 +29,10 @@ import org.apache.zookeeper.CreateMode;
 import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.WatchedEvent;
 import org.apache.zookeeper.WatchedEvent;
 import org.apache.zookeeper.Watcher;
 import org.apache.zookeeper.Watcher;
-import org.apache.zookeeper.ZooDefs;
-import org.apache.zookeeper.ZooKeeper;
 import org.apache.zookeeper.Watcher.Event.KeeperState;
 import org.apache.zookeeper.Watcher.Event.KeeperState;
+import org.apache.zookeeper.ZooDefs;
 import org.apache.zookeeper.ZooDefs.Ids;
 import org.apache.zookeeper.ZooDefs.Ids;
+import org.apache.zookeeper.ZooKeeper;
 import org.apache.zookeeper.data.Stat;
 import org.apache.zookeeper.data.Stat;
 import org.apache.zookeeper.server.quorum.Leader;
 import org.apache.zookeeper.server.quorum.Leader;
 import org.apache.zookeeper.server.quorum.LearnerHandler;
 import org.apache.zookeeper.server.quorum.LearnerHandler;
@@ -40,6 +40,7 @@ import org.apache.zookeeper.server.quorum.QuorumPeer;
 import org.junit.After;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Before;
+import org.junit.Ignore;
 import org.junit.Test;
 import org.junit.Test;
 
 
 public class QuorumTest extends QuorumBase {
 public class QuorumTest extends QuorumBase {
@@ -187,16 +188,21 @@ public class QuorumTest extends QuorumBase {
      * @throws KeeperException
      * @throws KeeperException
      */
      */
     @Test
     @Test
-    public void testSessionMoved() throws IOException, InterruptedException, KeeperException {
+    @Ignore
+    public void testSessionMoved() throws Exception {
         String hostPorts[] = qb.hostPort.split(",");
         String hostPorts[] = qb.hostPort.split(",");
-        DisconnectableZooKeeper zk = new DisconnectableZooKeeper(hostPorts[0], ClientBase.CONNECTION_TIMEOUT, new Watcher() {
+        DisconnectableZooKeeper zk = new DisconnectableZooKeeper(hostPorts[0],
+                ClientBase.CONNECTION_TIMEOUT, new Watcher() {
             public void process(WatchedEvent event) {
             public void process(WatchedEvent event) {
             }});
             }});
         zk.create("/sessionMoveTest", new byte[0], Ids.OPEN_ACL_UNSAFE, CreateMode.EPHEMERAL);
         zk.create("/sessionMoveTest", new byte[0], Ids.OPEN_ACL_UNSAFE, CreateMode.EPHEMERAL);
         // we want to loop through the list twice
         // we want to loop through the list twice
         for(int i = 0; i < hostPorts.length*2; i++) {
         for(int i = 0; i < hostPorts.length*2; i++) {
+            zk.dontReconnect();
             // This should stomp the zk handle
             // This should stomp the zk handle
-            DisconnectableZooKeeper zknew = new DisconnectableZooKeeper(hostPorts[(i+1)%hostPorts.length], ClientBase.CONNECTION_TIMEOUT,
+            DisconnectableZooKeeper zknew =
+                new DisconnectableZooKeeper(hostPorts[(i+1)%hostPorts.length],
+                    ClientBase.CONNECTION_TIMEOUT,
                     new Watcher() {public void process(WatchedEvent event) {
                     new Watcher() {public void process(WatchedEvent event) {
                     }},
                     }},
                     zk.getSessionId(),
                     zk.getSessionId(),
@@ -207,7 +213,6 @@ public class QuorumTest extends QuorumBase {
                 Assert.fail("Should have lost the connection");
                 Assert.fail("Should have lost the connection");
             } catch(KeeperException.ConnectionLossException e) {
             } catch(KeeperException.ConnectionLossException e) {
             }
             }
-            zk.disconnect(); // close w/o closing session
             zk = zknew;
             zk = zknew;
         }
         }
         zk.close();
         zk.close();
@@ -227,15 +232,17 @@ public class QuorumTest extends QuorumBase {
      * make sure we cannot do any changes.
      * make sure we cannot do any changes.
      */
      */
     @Test
     @Test
-    public void testSessionMove() throws IOException, InterruptedException, KeeperException {
+    @Ignore
+    public void testSessionMove() throws Exception {
         String hps[] = qb.hostPort.split(",");
         String hps[] = qb.hostPort.split(",");
         DiscoWatcher oldWatcher = new DiscoWatcher();
         DiscoWatcher oldWatcher = new DiscoWatcher();
-        ZooKeeper zk = new DisconnectableZooKeeper(hps[0],
+        DisconnectableZooKeeper zk = new DisconnectableZooKeeper(hps[0],
                 ClientBase.CONNECTION_TIMEOUT, oldWatcher);
                 ClientBase.CONNECTION_TIMEOUT, oldWatcher);
         zk.create("/t1", new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.EPHEMERAL);
         zk.create("/t1", new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.EPHEMERAL);
+        zk.dontReconnect();
         // This should stomp the zk handle
         // This should stomp the zk handle
         DiscoWatcher watcher = new DiscoWatcher();
         DiscoWatcher watcher = new DiscoWatcher();
-        ZooKeeper zknew = new DisconnectableZooKeeper(hps[1],
+        DisconnectableZooKeeper zknew = new DisconnectableZooKeeper(hps[1],
                 ClientBase.CONNECTION_TIMEOUT, watcher, zk.getSessionId(),
                 ClientBase.CONNECTION_TIMEOUT, watcher, zk.getSessionId(),
                 zk.getSessionPasswd());
                 zk.getSessionPasswd());
         zknew.create("/t2", new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.EPHEMERAL);
         zknew.create("/t2", new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.EPHEMERAL);
@@ -257,6 +264,7 @@ public class QuorumTest extends QuorumBase {
         toClose.add(zknew);
         toClose.add(zknew);
         // Let's just make sure it can still move
         // Let's just make sure it can still move
         for(int i = 0; i < 10; i++) {
         for(int i = 0; i < 10; i++) {
+            zknew.dontReconnect();
             zknew = new DisconnectableZooKeeper(hps[1],
             zknew = new DisconnectableZooKeeper(hps[1],
                     ClientBase.CONNECTION_TIMEOUT, new DiscoWatcher(),
                     ClientBase.CONNECTION_TIMEOUT, new DiscoWatcher(),
                     zk.getSessionId(), zk.getSessionPasswd());
                     zk.getSessionId(), zk.getSessionPasswd());

+ 4 - 6
src/java/test/org/apache/zookeeper/test/RecoveryTest.java

@@ -21,7 +21,6 @@ package org.apache.zookeeper.test;
 import static org.apache.zookeeper.test.ClientBase.CONNECTION_TIMEOUT;
 import static org.apache.zookeeper.test.ClientBase.CONNECTION_TIMEOUT;
 
 
 import java.io.File;
 import java.io.File;
-import java.net.InetSocketAddress;
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeUnit;
 
 
@@ -35,7 +34,7 @@ import org.apache.zookeeper.ZooKeeper;
 import org.apache.zookeeper.Watcher.Event.KeeperState;
 import org.apache.zookeeper.Watcher.Event.KeeperState;
 import org.apache.zookeeper.ZooDefs.Ids;
 import org.apache.zookeeper.ZooDefs.Ids;
 import org.apache.zookeeper.data.Stat;
 import org.apache.zookeeper.data.Stat;
-import org.apache.zookeeper.server.NIOServerCnxn;
+import org.apache.zookeeper.server.ServerCnxnFactory;
 import org.apache.zookeeper.server.SyncRequestProcessor;
 import org.apache.zookeeper.server.SyncRequestProcessor;
 import org.apache.zookeeper.server.ZooKeeperServer;
 import org.apache.zookeeper.server.ZooKeeperServer;
 import org.junit.Assert;
 import org.junit.Assert;
@@ -72,8 +71,7 @@ public class RecoveryTest extends ZKTestCase implements Watcher {
         SyncRequestProcessor.setSnapCount(1000);
         SyncRequestProcessor.setSnapCount(1000);
         try {
         try {
             final int PORT = Integer.parseInt(HOSTPORT.split(":")[1]);
             final int PORT = Integer.parseInt(HOSTPORT.split(":")[1]);
-            NIOServerCnxn.Factory f = new NIOServerCnxn.Factory(
-                    new InetSocketAddress(PORT));
+            ServerCnxnFactory f = ServerCnxnFactory.createFactory(PORT, -1);
             f.startup(zks);
             f.startup(zks);
             LOG.info("starting up the the server, waiting");
             LOG.info("starting up the the server, waiting");
 
 
@@ -111,7 +109,7 @@ public class RecoveryTest extends ZKTestCase implements Watcher {
                                           CONNECTION_TIMEOUT));
                                           CONNECTION_TIMEOUT));
 
 
             zks = new ZooKeeperServer(tmpDir, tmpDir, 3000);
             zks = new ZooKeeperServer(tmpDir, tmpDir, 3000);
-            f = new NIOServerCnxn.Factory(new InetSocketAddress(PORT));
+            f = ServerCnxnFactory.createFactory(PORT, -1);
 
 
             startSignal = new CountDownLatch(1);
             startSignal = new CountDownLatch(1);
 
 
@@ -149,7 +147,7 @@ public class RecoveryTest extends ZKTestCase implements Watcher {
                                           ClientBase.CONNECTION_TIMEOUT));
                                           ClientBase.CONNECTION_TIMEOUT));
 
 
             zks = new ZooKeeperServer(tmpDir, tmpDir, 3000);
             zks = new ZooKeeperServer(tmpDir, tmpDir, 3000);
-            f = new NIOServerCnxn.Factory(new InetSocketAddress(PORT));
+            f = ServerCnxnFactory.createFactory(PORT, -1);
 
 
             startSignal = new CountDownLatch(1);
             startSignal = new CountDownLatch(1);
 
 

+ 2 - 5
src/java/test/org/apache/zookeeper/test/RepeatStartupTest.java

@@ -18,13 +18,11 @@
 
 
 package org.apache.zookeeper.test;
 package org.apache.zookeeper.test;
 
 
-import java.net.InetSocketAddress;
-
 import org.apache.zookeeper.CreateMode;
 import org.apache.zookeeper.CreateMode;
 import org.apache.zookeeper.ZKTestCase;
 import org.apache.zookeeper.ZKTestCase;
 import org.apache.zookeeper.ZooKeeper;
 import org.apache.zookeeper.ZooKeeper;
 import org.apache.zookeeper.ZooDefs.Ids;
 import org.apache.zookeeper.ZooDefs.Ids;
-import org.apache.zookeeper.server.NIOServerCnxn;
+import org.apache.zookeeper.server.ServerCnxnFactory;
 import org.apache.zookeeper.server.ZooKeeperServer;
 import org.apache.zookeeper.server.ZooKeeperServer;
 import org.junit.Assert;
 import org.junit.Assert;
 import org.junit.Test;
 import org.junit.Test;
@@ -54,8 +52,7 @@ public class RepeatStartupTest extends ZKTestCase {
         ZooKeeperServer zks = new ZooKeeperServer(qb.s1.getTxnFactory().getSnapDir(),
         ZooKeeperServer zks = new ZooKeeperServer(qb.s1.getTxnFactory().getSnapDir(),
                 qb.s1.getTxnFactory().getDataDir(), 3000);
                 qb.s1.getTxnFactory().getDataDir(), 3000);
         final int PORT = Integer.parseInt(hp.split(":")[1]);
         final int PORT = Integer.parseInt(hp.split(":")[1]);
-        NIOServerCnxn.Factory factory = new NIOServerCnxn.Factory(
-                new InetSocketAddress(PORT));
+        ServerCnxnFactory factory = ServerCnxnFactory.createFactory(PORT, -1);
 
 
         factory.startup(zks);
         factory.startup(zks);
         System.out.println("Comment: starting factory");
         System.out.println("Comment: starting factory");

+ 44 - 26
src/java/test/org/apache/zookeeper/test/SessionTest.java

@@ -22,7 +22,6 @@ import static org.apache.zookeeper.test.ClientBase.CONNECTION_TIMEOUT;
 
 
 import java.io.File;
 import java.io.File;
 import java.io.IOException;
 import java.io.IOException;
-import java.net.InetSocketAddress;
 import java.util.ArrayList;
 import java.util.ArrayList;
 import java.util.LinkedList;
 import java.util.LinkedList;
 import java.util.List;
 import java.util.List;
@@ -35,25 +34,27 @@ import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.PortAssignment;
 import org.apache.zookeeper.PortAssignment;
 import org.apache.zookeeper.WatchedEvent;
 import org.apache.zookeeper.WatchedEvent;
 import org.apache.zookeeper.Watcher;
 import org.apache.zookeeper.Watcher;
-import org.apache.zookeeper.ZKTestCase;
-import org.apache.zookeeper.ZooKeeper;
 import org.apache.zookeeper.Watcher.Event.EventType;
 import org.apache.zookeeper.Watcher.Event.EventType;
 import org.apache.zookeeper.Watcher.Event.KeeperState;
 import org.apache.zookeeper.Watcher.Event.KeeperState;
+import org.apache.zookeeper.ZKTestCase;
 import org.apache.zookeeper.ZooDefs.Ids;
 import org.apache.zookeeper.ZooDefs.Ids;
+import org.apache.zookeeper.ZooKeeper;
 import org.apache.zookeeper.data.Stat;
 import org.apache.zookeeper.data.Stat;
-import org.apache.zookeeper.server.NIOServerCnxn;
+import org.apache.zookeeper.server.ServerCnxnFactory;
 import org.apache.zookeeper.server.ZooKeeperServer;
 import org.apache.zookeeper.server.ZooKeeperServer;
 import org.junit.After;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Before;
+import org.junit.Ignore;
 import org.junit.Test;
 import org.junit.Test;
 
 
-public class SessionTest extends ZKTestCase implements Watcher {
+public class SessionTest extends ZKTestCase {
     protected static final Logger LOG = Logger.getLogger(SessionTest.class);
     protected static final Logger LOG = Logger.getLogger(SessionTest.class);
 
 
     private static final String HOSTPORT = "127.0.0.1:" +
     private static final String HOSTPORT = "127.0.0.1:" +
             PortAssignment.unique();
             PortAssignment.unique();
-    private NIOServerCnxn.Factory serverFactory;
+    
+    private ServerCnxnFactory serverFactory;
 
 
     private CountDownLatch startSignal;
     private CountDownLatch startSignal;
 
 
@@ -71,7 +72,7 @@ public class SessionTest extends ZKTestCase implements Watcher {
         ZooKeeperServer zs = new ZooKeeperServer(tmpDir, tmpDir, TICK_TIME);
         ZooKeeperServer zs = new ZooKeeperServer(tmpDir, tmpDir, TICK_TIME);
 
 
         final int PORT = Integer.parseInt(HOSTPORT.split(":")[1]);
         final int PORT = Integer.parseInt(HOSTPORT.split(":")[1]);
-        serverFactory = new NIOServerCnxn.Factory(new InetSocketAddress(PORT));
+        serverFactory = ServerCnxnFactory.createFactory(PORT, -1);
         serverFactory.startup(zs);
         serverFactory.startup(zs);
 
 
         Assert.assertTrue("waiting for server up",
         Assert.assertTrue("waiting for server up",
@@ -156,6 +157,22 @@ public class SessionTest extends ZKTestCase implements Watcher {
 //        zk.close();
 //        zk.close();
 //    }
 //    }
 
 
+    private class MyWatcher implements Watcher {
+        private String name;
+        public MyWatcher(String name) {
+            this.name = name;
+        }
+        public void process(WatchedEvent event) {
+            LOG.info(name + " event:" + event.getState() + " "
+                    + event.getType() + " " + event.getPath());
+            if (event.getState() == KeeperState.SyncConnected
+                    && startSignal != null && startSignal.getCount() > 0)
+            {
+                startSignal.countDown();
+            }
+        }
+    }
+
     /**
     /**
      * This test verifies that when the session id is reused, and the original
      * This test verifies that when the session id is reused, and the original
      * client is disconnected, but not session closed, that the server
      * client is disconnected, but not session closed, that the server
@@ -178,9 +195,9 @@ public class SessionTest extends ZKTestCase implements Watcher {
 
 
         Stat stat = new Stat();
         Stat stat = new Stat();
         startSignal = new CountDownLatch(1);
         startSignal = new CountDownLatch(1);
-        zk = new DisconnectableZooKeeper(HOSTPORT, CONNECTION_TIMEOUT, this,
-                               zk.getSessionId(),
-                               zk.getSessionPasswd());
+        zk = new DisconnectableZooKeeper(HOSTPORT, CONNECTION_TIMEOUT,
+                new MyWatcher("testSession"), zk.getSessionId(),
+                zk.getSessionPasswd());
         startSignal.await();
         startSignal.await();
 
 
         LOG.info("zk with session id 0x" + Long.toHexString(zk.getSessionId())
         LOG.info("zk with session id 0x" + Long.toHexString(zk.getSessionId())
@@ -270,14 +287,21 @@ public class SessionTest extends ZKTestCase implements Watcher {
      * @throws KeeperException
      * @throws KeeperException
      */
      */
     @Test
     @Test
-    public void testSessionMove() throws IOException, InterruptedException, KeeperException {
+    @Ignore
+    public void testSessionMove() throws Exception {
         String hostPorts[] = HOSTPORT.split(",");
         String hostPorts[] = HOSTPORT.split(",");
-        ZooKeeper zk = new DisconnectableZooKeeper(hostPorts[0], CONNECTION_TIMEOUT, this);
-        zk.create("/sessionMoveTest", new byte[0], Ids.OPEN_ACL_UNSAFE, CreateMode.EPHEMERAL);
+        DisconnectableZooKeeper zk = new DisconnectableZooKeeper(hostPorts[0],
+                CONNECTION_TIMEOUT, new MyWatcher("0"));
+        zk.create("/sessionMoveTest", new byte[0], Ids.OPEN_ACL_UNSAFE,
+                CreateMode.EPHEMERAL);
         // we want to loop through the list twice
         // we want to loop through the list twice
         for(int i = 0; i < hostPorts.length*2; i++) {
         for(int i = 0; i < hostPorts.length*2; i++) {
+            zk.dontReconnect();
             // This should stomp the zk handle
             // This should stomp the zk handle
-            ZooKeeper zknew = new DisconnectableZooKeeper(hostPorts[(i+1)%hostPorts.length], CONNECTION_TIMEOUT, this,
+            DisconnectableZooKeeper zknew = new DisconnectableZooKeeper(
+                    hostPorts[(i+1)%hostPorts.length],
+                    CONNECTION_TIMEOUT,
+                    new MyWatcher(Integer.toString(i+1)),
                     zk.getSessionId(),
                     zk.getSessionId(),
                     zk.getSessionPasswd());
                     zk.getSessionPasswd());
             zknew.setData("/", new byte[1], -1);
             zknew.setData("/", new byte[1], -1);
@@ -285,6 +309,7 @@ public class SessionTest extends ZKTestCase implements Watcher {
                 zk.setData("/", new byte[1], -1);
                 zk.setData("/", new byte[1], -1);
                 Assert.fail("Should have lost the connection");
                 Assert.fail("Should have lost the connection");
             } catch(KeeperException.ConnectionLossException e) {
             } catch(KeeperException.ConnectionLossException e) {
+                LOG.info("Got connection loss exception as expected");
             }
             }
             //zk.close();
             //zk.close();
             zk = zknew;
             zk = zknew;
@@ -362,23 +387,16 @@ public class SessionTest extends ZKTestCase implements Watcher {
         }
         }
     }
     }
 
 
-    public void process(WatchedEvent event) {
-        LOG.info("Event:" + event.getState() + " " + event.getType() + " " + event.getPath());
-        if (event.getState() == KeeperState.SyncConnected
-                && startSignal != null && startSignal.getCount() > 0)
-        {
-            startSignal.countDown();
-        }
-    }
-
     @Test
     @Test
     public void testMinMaxSessionTimeout() throws Exception {
     public void testMinMaxSessionTimeout() throws Exception {
         // override the defaults
         // override the defaults
         final int MINSESS = 20000;
         final int MINSESS = 20000;
         final int MAXSESS = 240000;
         final int MAXSESS = 240000;
-        ZooKeeperServer zs = serverFactory.getZooKeeperServer();
-        zs.setMinSessionTimeout(MINSESS);
-        zs.setMaxSessionTimeout(MAXSESS);
+        {
+            ZooKeeperServer zs = ClientBase.getServer(serverFactory);
+            zs.setMinSessionTimeout(MINSESS);
+            zs.setMaxSessionTimeout(MAXSESS);
+        }
 
 
         // validate typical case - requested == negotiated
         // validate typical case - requested == negotiated
         int timeout = 120000;
         int timeout = 120000;

+ 10 - 3
src/java/test/org/apache/zookeeper/test/TruncateTest.java

@@ -32,8 +32,9 @@ import org.apache.zookeeper.ZKTestCase;
 import org.apache.zookeeper.ZooDefs;
 import org.apache.zookeeper.ZooDefs;
 import org.apache.zookeeper.ZooKeeper;
 import org.apache.zookeeper.ZooKeeper;
 import org.apache.zookeeper.data.Stat;
 import org.apache.zookeeper.data.Stat;
-import org.apache.zookeeper.server.NIOServerCnxn;
+import org.apache.zookeeper.server.ServerCnxnFactory;
 import org.apache.zookeeper.server.ZKDatabase;
 import org.apache.zookeeper.server.ZKDatabase;
+import org.apache.zookeeper.server.ZooKeeperServer;
 import org.apache.zookeeper.server.quorum.QuorumPeer;
 import org.apache.zookeeper.server.quorum.QuorumPeer;
 import org.apache.zookeeper.server.quorum.QuorumPeer.QuorumServer;
 import org.apache.zookeeper.server.quorum.QuorumPeer.QuorumServer;
 import org.junit.After;
 import org.junit.After;
@@ -71,13 +72,19 @@ public class TruncateTest extends ZKTestCase {
     @Test
     @Test
     public void testTruncate() throws IOException, InterruptedException, KeeperException {
     public void testTruncate() throws IOException, InterruptedException, KeeperException {
         // Prime the server that is going to come in late with 50 txns
         // Prime the server that is going to come in late with 50 txns
-        NIOServerCnxn.Factory factory = ClientBase.createNewServerInstance(dataDir1, null, "127.0.0.1:" + baseHostPort, 100);
+        ServerCnxnFactory factory = ClientBase.createNewServerInstance(dataDir1, null, "127.0.0.1:" + baseHostPort, 100);
         ZooKeeper zk = new ZooKeeper("127.0.0.1:" + baseHostPort, 15000, nullWatcher);
         ZooKeeper zk = new ZooKeeper("127.0.0.1:" + baseHostPort, 15000, nullWatcher);
         for(int i = 0; i < 50; i++) {
         for(int i = 0; i < 50; i++) {
             zk.create("/" + i, new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
             zk.create("/" + i, new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
         }
         }
         zk.close();
         zk.close();
-        ZKDatabase zkDb = factory.getZooKeeperServer().getZKDatabase();
+        
+        ZKDatabase zkDb;
+        {
+            ZooKeeperServer zs = ClientBase.getServer(factory);
+    
+            zkDb = zs.getZKDatabase();
+        }
         factory.shutdown();
         factory.shutdown();
         try {
         try {
             zkDb.close();
             zkDb.close();

+ 2 - 4
src/java/test/org/apache/zookeeper/test/UpgradeTest.java

@@ -21,7 +21,6 @@ package org.apache.zookeeper.test;
 import static org.apache.zookeeper.test.ClientBase.CONNECTION_TIMEOUT;
 import static org.apache.zookeeper.test.ClientBase.CONNECTION_TIMEOUT;
 
 
 import java.io.File;
 import java.io.File;
-import java.net.InetSocketAddress;
 import java.util.Collections;
 import java.util.Collections;
 import java.util.List;
 import java.util.List;
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.CountDownLatch;
@@ -36,7 +35,7 @@ import org.apache.zookeeper.ZooKeeper;
 import org.apache.zookeeper.Watcher.Event.KeeperState;
 import org.apache.zookeeper.Watcher.Event.KeeperState;
 import org.apache.zookeeper.ZooDefs.Ids;
 import org.apache.zookeeper.ZooDefs.Ids;
 import org.apache.zookeeper.data.Stat;
 import org.apache.zookeeper.data.Stat;
-import org.apache.zookeeper.server.NIOServerCnxn;
+import org.apache.zookeeper.server.ServerCnxnFactory;
 import org.apache.zookeeper.server.SyncRequestProcessor;
 import org.apache.zookeeper.server.SyncRequestProcessor;
 import org.apache.zookeeper.server.ZooKeeperServer;
 import org.apache.zookeeper.server.ZooKeeperServer;
 import org.apache.zookeeper.server.upgrade.UpgradeMain;
 import org.apache.zookeeper.server.upgrade.UpgradeMain;
@@ -63,8 +62,7 @@ public class UpgradeTest extends ZKTestCase implements Watcher {
         ZooKeeperServer zks = new ZooKeeperServer(upgradeDir, upgradeDir, 3000);
         ZooKeeperServer zks = new ZooKeeperServer(upgradeDir, upgradeDir, 3000);
         SyncRequestProcessor.setSnapCount(1000);
         SyncRequestProcessor.setSnapCount(1000);
         final int PORT = Integer.parseInt(HOSTPORT.split(":")[1]);
         final int PORT = Integer.parseInt(HOSTPORT.split(":")[1]);
-        NIOServerCnxn.Factory f = new NIOServerCnxn.Factory(
-                new InetSocketAddress(PORT));
+        ServerCnxnFactory f = ServerCnxnFactory.createFactory(PORT, -1);
         f.startup(zks);
         f.startup(zks);
         LOG.info("starting up the zookeeper server .. waiting");
         LOG.info("starting up the zookeeper server .. waiting");
         Assert.assertTrue("waiting for server being up",
         Assert.assertTrue("waiting for server being up",

+ 2 - 1
src/java/test/org/apache/zookeeper/test/WatcherTest.java

@@ -148,7 +148,8 @@ public class WatcherTest extends ClientBase {
             zk1.getData("/watch-count-test-2", w1, stat);
             zk1.getData("/watch-count-test-2", w1, stat);
             zk2.getData("/watch-count-test", w2, stat);
             zk2.getData("/watch-count-test", w2, stat);
 
 
-            Assert.assertEquals(getServer().getZKDatabase().getDataTree().getWatchCount(), 3);
+            Assert.assertEquals(ClientBase.getServer(serverFactory)
+                    .getZKDatabase().getDataTree().getWatchCount(), 3);
 
 
         } finally {
         } finally {
             if(zk1 != null) {
             if(zk1 != null) {

Some files were not shown because too many files changed in this diff