You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@zookeeper.apache.org by ma...@apache.org on 2010/01/26 03:22:43 UTC

svn commit: r903050 - in /hadoop/zookeeper/trunk: ./ docs/ src/c/include/ src/docs/src/documentation/content/xdocs/ src/java/main/org/apache/zookeeper/ src/java/test/org/apache/zookeeper/test/

Author: mahadev
Date: Tue Jan 26 02:22:40 2010
New Revision: 903050

URL: http://svn.apache.org/viewvc?rev=903050&view=rev
Log:
ZOOKEEPER-593.  java client api does not allow client to access negotiated session timeout (phunt via mahadev)

Modified:
    hadoop/zookeeper/trunk/CHANGES.txt
    hadoop/zookeeper/trunk/docs/zookeeperProgrammers.html
    hadoop/zookeeper/trunk/docs/zookeeperProgrammers.pdf
    hadoop/zookeeper/trunk/src/c/include/zookeeper.h
    hadoop/zookeeper/trunk/src/docs/src/documentation/content/xdocs/zookeeperProgrammers.xml
    hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/ClientCnxn.java
    hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/ZooKeeper.java
    hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/SessionTest.java

Modified: hadoop/zookeeper/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/CHANGES.txt?rev=903050&r1=903049&r2=903050&view=diff
==============================================================================
--- hadoop/zookeeper/trunk/CHANGES.txt (original)
+++ hadoop/zookeeper/trunk/CHANGES.txt Tue Jan 26 02:22:40 2010
@@ -257,6 +257,9 @@
   ZOOKEEPER-456. CREATOR_ALL_ACL has unnecessary PERMS.ADMIN in the
   declartion. (phunt via mahadev)
 
+  ZOOKEEPER-593.  java client api does not allow client to access negotiated
+  session timeout (phunt via mahadev)
+
 NEW FEATURES:
   ZOOKEEPER-539. generate eclipse project via ant target. (phunt via mahadev)
 

Modified: hadoop/zookeeper/trunk/docs/zookeeperProgrammers.html
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/docs/zookeeperProgrammers.html?rev=903050&r1=903049&r2=903050&view=diff
==============================================================================
--- hadoop/zookeeper/trunk/docs/zookeeperProgrammers.html (original)
+++ hadoop/zookeeper/trunk/docs/zookeeperProgrammers.html Tue Jan 26 02:22:40 2010
@@ -827,7 +827,8 @@
     responds with the timeout that it can give the client. The current
     implementation requires that the timeout be a minimum of 2 times
     the tickTime (as set in the server configuration) and a maximum of
-    20 times the tickTime.</p>
+    20 times the tickTime. The ZooKeeper client API allows access to
+    the negotiated timeout.</p>
 <p>Another parameter to the ZooKeeper session establishment
     call is the default watcher. Watchers are notified when any state
     change occurs in the client. For example if the client loses

Modified: hadoop/zookeeper/trunk/docs/zookeeperProgrammers.pdf
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/docs/zookeeperProgrammers.pdf?rev=903050&r1=903049&r2=903050&view=diff
==============================================================================
Binary files - no diff available.

Modified: hadoop/zookeeper/trunk/src/c/include/zookeeper.h
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/c/include/zookeeper.h?rev=903050&r1=903049&r2=903050&view=diff
==============================================================================
--- hadoop/zookeeper/trunk/src/c/include/zookeeper.h (original)
+++ hadoop/zookeeper/trunk/src/c/include/zookeeper.h Tue Jan 26 02:22:40 2010
@@ -343,10 +343,21 @@
  */
 ZOOAPI const clientid_t *zoo_client_id(zhandle_t *zh);
 
+/**
+ * \brief return the timeout for this session, only valid if the connections
+ * is currently connected (ie. last watcher state is ZOO_CONNECTED_STATE). This
+ * value may change after a server re-connect.
+ */
 ZOOAPI int zoo_recv_timeout(zhandle_t *zh);
 
+/**
+ * \brief return the context for this handle.
+ */
 ZOOAPI const void *zoo_get_context(zhandle_t *zh);
 
+/**
+ * \brief set the context for this handle.
+ */
 ZOOAPI void zoo_set_context(zhandle_t *zh, void *context);
 
 /**

Modified: hadoop/zookeeper/trunk/src/docs/src/documentation/content/xdocs/zookeeperProgrammers.xml
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/docs/src/documentation/content/xdocs/zookeeperProgrammers.xml?rev=903050&r1=903049&r2=903050&view=diff
==============================================================================
--- hadoop/zookeeper/trunk/src/docs/src/documentation/content/xdocs/zookeeperProgrammers.xml (original)
+++ hadoop/zookeeper/trunk/src/docs/src/documentation/content/xdocs/zookeeperProgrammers.xml Tue Jan 26 02:22:40 2010
@@ -435,7 +435,8 @@
     responds with the timeout that it can give the client. The current
     implementation requires that the timeout be a minimum of 2 times
     the tickTime (as set in the server configuration) and a maximum of
-    20 times the tickTime.</para>
+    20 times the tickTime. The ZooKeeper client API allows access to
+    the negotiated timeout.</para>
 
     <para>Another parameter to the ZooKeeper session establishment
     call is the default watcher. Watchers are notified when any state

Modified: hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/ClientCnxn.java
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/ClientCnxn.java?rev=903050&r1=903049&r2=903050&view=diff
==============================================================================
--- hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/ClientCnxn.java (original)
+++ hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/ClientCnxn.java Tue Jan 26 02:22:40 2010
@@ -129,6 +129,13 @@
 
     private int connectTimeout;
 
+    /** The timeout in ms the client negotiated with the server. This is the 
+     *  "real" timeout, not the timeout request by the client (which may
+     *  have been increased/decreased by the server which applies bounds
+     *  to this value.
+     */
+    private volatile int negotiatedSessionTimeout;
+
     private int readTimeout;
 
     private final int sessionTimeout;
@@ -165,6 +172,10 @@
         return sessionPasswd;
     }
 
+    public int getSessionTimeout() {
+        return negotiatedSessionTimeout;
+    }
+
     @Override
     public String toString() {
         StringBuilder sb = new StringBuilder();
@@ -668,8 +679,8 @@
             BinaryInputArchive bbia = BinaryInputArchive.getArchive(bbis);
             ConnectResponse conRsp = new ConnectResponse();
             conRsp.deserialize(bbia, "connect");
-            int sessionTimeout = conRsp.getTimeOut();
-            if (sessionTimeout <= 0) {
+            negotiatedSessionTimeout = conRsp.getTimeOut();
+            if (negotiatedSessionTimeout <= 0) {
                 zooKeeper.state = States.CLOSED;
 
                 eventThread.queueEvent(new WatchedEvent(
@@ -679,8 +690,8 @@
                         "Unable to reconnect to ZooKeeper service, session 0x"
                         + Long.toHexString(sessionId) + " has expired");
             }
-            readTimeout = sessionTimeout * 2 / 3;
-            connectTimeout = sessionTimeout / serverAddrs.size();
+            readTimeout = negotiatedSessionTimeout * 2 / 3;
+            connectTimeout = negotiatedSessionTimeout / serverAddrs.size();
             sessionId = conRsp.getSessionId();
             sessionPasswd = conRsp.getPasswd();
             zooKeeper.state = States.CONNECTED;
@@ -689,7 +700,7 @@
                         .socket().getRemoteSocketAddress()
                     + ", sessionid = 0x"
                     + Long.toHexString(sessionId)
-                    + ", negotiated timeout = " + sessionTimeout);
+                    + ", negotiated timeout = " + negotiatedSessionTimeout);
             eventThread.queueEvent(new WatchedEvent(Watcher.Event.EventType.None,
                     Watcher.Event.KeeperState.SyncConnected, null));
         }

Modified: hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/ZooKeeper.java
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/ZooKeeper.java?rev=903050&r1=903049&r2=903050&view=diff
==============================================================================
--- hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/ZooKeeper.java (original)
+++ hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/ZooKeeper.java Tue Jan 26 02:22:40 2010
@@ -473,6 +473,19 @@
     }
 
     /**
+     * The negotiated session timeout for this ZooKeeper client instance. The
+     * value returned is not valid until the client connects to a server and
+     * may change after a re-connect.
+     *
+     * This method is NOT thread safe
+     *
+     * @return current session timeout
+     */
+    public int getSessionTimeout() {
+        return cnxn.getSessionTimeout();
+    }
+
+    /**
      * Add the specified scheme:auth information to this connection.
      *
      * This method is NOT thread safe
@@ -1469,7 +1482,12 @@
      */
     @Override
     public String toString() {
-        return ("State:" + getState().toString() + " " + cnxn);
+        States state = getState();
+        return ("State:" + state.toString()
+                + (state == States.CONNECTED ?
+                        " Timeout:" + getSessionTimeout() + " " :
+                        " ")
+                + cnxn);
     }
 
     /*

Modified: hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/SessionTest.java
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/SessionTest.java?rev=903050&r1=903049&r2=903050&view=diff
==============================================================================
--- hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/SessionTest.java (original)
+++ hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/SessionTest.java Tue Jan 26 02:22:40 2010
@@ -53,6 +53,8 @@
     private CountDownLatch startSignal;
 
     File tmpDir;
+    
+    private final int TICK_TIME = 3000;
 
     @Override
     protected void setUp() throws Exception {
@@ -63,7 +65,7 @@
         }
 
         ClientBase.setupTestEnv();
-        ZooKeeperServer zs = new ZooKeeperServer(tmpDir, tmpDir, 3000);
+        ZooKeeperServer zs = new ZooKeeperServer(tmpDir, tmpDir, TICK_TIME);
 
         final int PORT = Integer.parseInt(HOSTPORT.split(":")[1]);
         serverFactory = new NIOServerCnxn.Factory(PORT);
@@ -283,6 +285,34 @@
         zk.close();
     }
 
+    @Test
+    /**
+     * Verify access to the negotiated session timeout.
+     */
+    public void testSessionTimeoutAccess() throws Exception {
+        // validate typical case - requested == negotiated
+        DisconnectableZooKeeper zk = createClient(TICK_TIME * 4);
+        assertEquals(TICK_TIME * 4, zk.getSessionTimeout());
+        // make sure tostring works in both cases
+        LOG.info(zk.toString());
+        zk.close();
+        LOG.info(zk.toString());
+
+        // validate lower limit
+        zk = createClient(TICK_TIME);
+        assertEquals(TICK_TIME * 2, zk.getSessionTimeout());
+        LOG.info(zk.toString());
+        zk.close();
+        LOG.info(zk.toString());
+
+        // validate upper limit
+        zk = createClient(TICK_TIME * 30);
+        assertEquals(TICK_TIME * 20, zk.getSessionTimeout());
+        LOG.info(zk.toString());
+        zk.close();
+        LOG.info(zk.toString());
+    }
+
     private class DupWatcher extends CountdownWatcher {
         public LinkedList<WatchedEvent> states = new LinkedList<WatchedEvent>();
         public void process(WatchedEvent event) {