You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@zookeeper.apache.org by ph...@apache.org on 2010/08/17 09:24:11 UTC

svn commit: r986215 [2/2] - in /hadoop/zookeeper/trunk: ./ src/contrib/rest/ src/contrib/rest/conf/ src/contrib/rest/conf/keys/ src/contrib/rest/src/java/org/apache/zookeeper/server/jersey/ src/contrib/rest/src/java/org/apache/zookeeper/server/jersey/c...

Added: hadoop/zookeeper/trunk/src/contrib/rest/src/python/test.py
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/contrib/rest/src/python/test.py?rev=986215&view=auto
==============================================================================
--- hadoop/zookeeper/trunk/src/contrib/rest/src/python/test.py (added)
+++ hadoop/zookeeper/trunk/src/contrib/rest/src/python/test.py Tue Aug 17 07:24:09 2010
@@ -0,0 +1,163 @@
+#! /usr/bin/env python
+
+# 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.
+
+import time
+import unittest
+
+from zkrest import ZooKeeper
+
+class ZooKeeperREST_TestCase(unittest.TestCase):
+    
+    BASE_URI = 'http://localhost:9998'
+
+    def setUp(self):
+        self.zk = ZooKeeper(self.BASE_URI)
+
+    def tearDown(self):
+        try:
+            self.zk.delete('/test')
+        except ZooKeeper.NotFound:
+            pass
+
+    def test_get_root_node(self):
+        assert self.zk.get('/') is not None
+
+    def test_get_node_not_found(self):
+        self.assertRaises(ZooKeeper.NotFound, \
+            self.zk.get, '/dummy-node')
+
+    def test_exists_node(self):
+        assert self.zk.exists('/zookeeper') is True
+
+    def test_get_children(self):
+        assert any([child['path'] == '/zookeeper/quota' \
+            for child in self.zk.get_children('/zookeeper')])
+            
+    def test_create_znode(self):
+        try:
+            self.zk.create('/test')
+        except ZooKeeper.ZNodeExists:
+            pass # it's ok if already exists
+        assert self.zk.exists('/test') is True
+
+    def test_create_hierarchy(self):
+        try:
+            self.zk.delete(['/a/b', '/a'])
+        except ZooKeeper.NotFound:
+            pass
+
+        self.zk.create('/a')
+        self.zk.create('/a/b')
+
+        self.zk.delete(['/a/b', '/a'])
+
+    def test_create_with_data(self):
+        self.zk.create('/test', 'some-data')
+
+        zn = self.zk.get('/test')
+        self.assertEqual(zn.get('data64', None), \
+            'some-data'.encode('base64').strip())
+
+    def test_delete_znode(self):
+        self.zk.create('/test')
+
+        self.zk.delete('/test')
+        assert not self.zk.exists('/test')
+
+    def test_delete_older_version(self):
+        self.zk.create('/test')
+
+        zn = self.zk.get('/test')
+        # do one more modification in order to increase the version number
+        self.zk.set('/test', 'dummy-data')
+
+        self.assertRaises(ZooKeeper.WrongVersion, \
+            self.zk.delete, '/test', version=zn['version'])
+
+    def test_delete_raise_not_found(self):
+        self.zk.create('/test')
+
+        zn = self.zk.get('/test')
+        self.zk.delete('/test')
+ 
+        self.assertRaises(ZooKeeper.NotFound, \
+            self.zk.delete, '/test', version=zn['version'])
+
+    def test_set(self):
+        self.zk.create('/test')
+
+        self.zk.set('/test', 'dummy')
+
+        self.assertEqual(self.zk.get('/test')['data64'], \
+            'dummy'.encode('base64').strip())
+
+    def test_set_with_older_version(self):
+        if not self.zk.exists('/test'):
+            self.zk.create('/test', 'random-data')
+
+        zn = self.zk.get('/test')
+        self.zk.set('/test', 'new-data')
+        self.assertRaises(ZooKeeper.WrongVersion, self.zk.set, \
+            '/test', 'older-version', version=zn['version'])
+
+    def test_set_null(self):
+        if not self.zk.exists('/test'):
+            self.zk.create('/test', 'random-data')
+        self.zk.set('/test', 'data')
+        assert 'data64' in self.zk.get('/test')
+
+        self.zk.set('/test', null=True)
+        assert 'data64' not in self.zk.get('/test')
+
+    def test_create_ephemeral_node(self):
+        with self.zk.session():
+            if self.zk.exists('/ephemeral-test'):
+                self.zk.delete('/ephemeral-test')
+
+            self.zk.create('/ephemeral-test', ephemeral=True)
+            zn = self.zk.get('/ephemeral-test')
+
+            assert zn['ephemeralOwner'] != 0
+
+    def test_create_session(self):
+        with self.zk.session() as sid:
+            self.assertEqual(len(sid), 36) # UUID
+
+    def test_session_invalidation(self):
+        self.zk.start_session(expire=1)
+        self.zk.create('/ephemeral-test', ephemeral=True)
+
+        # keep the session alive by sending heartbeat requests
+        for _ in range(1,2):
+            self.zk.heartbeat()
+            time.sleep(0.9)
+
+        time.sleep(2) # wait for the session to expire
+        self.assertRaises(ZooKeeper.InvalidSession, \
+            self.zk.create, '/ephemeral-test', ephemeral=True)
+
+    def test_presence_signaling(self):
+        with self.zk.session(expire=1):
+            self.zk.create('/i-am-online', ephemeral=True)
+            assert self.zk.exists('/i-am-online')
+        assert not self.zk.exists('/i-am-online')
+
+
+if __name__ == '__main__':
+    unittest.main()
+

Added: hadoop/zookeeper/trunk/src/contrib/rest/src/python/zkrest.py
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/contrib/rest/src/python/zkrest.py?rev=986215&view=auto
==============================================================================
--- hadoop/zookeeper/trunk/src/contrib/rest/src/python/zkrest.py (added)
+++ hadoop/zookeeper/trunk/src/contrib/rest/src/python/zkrest.py Tue Aug 17 07:24:09 2010
@@ -0,0 +1,218 @@
+
+# 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.
+
+import urllib2
+import urllib
+import simplejson
+
+from contextlib import contextmanager
+
+class RequestWithMethod(urllib2.Request):
+    """ Request class that know how to set the method name """
+    def __init__(self, *args, **kwargs):
+        urllib2.Request.__init__(self, *args, **kwargs)
+        self._method = None
+
+    def get_method(self):
+        return self._method or \
+            urllib2.Request.get_method(self)
+
+    def set_method(self, method):
+        self._method = method
+
+class ZooKeeper(object):
+
+    class Error(Exception): pass
+
+    class NotFound(Error): pass
+
+    class ZNodeExists(Error): pass
+
+    class InvalidSession(Error): pass
+
+    class WrongVersion(Error): pass
+
+    def __init__(self, uri = 'http://localhost:9998'):
+        self._base = uri
+        self._session = None
+
+    def start_session(self, expire=5, id=None):
+        """ Create a session and return the ID """
+        if id is None:
+            url = "%s/sessions/v1/?op=create&expire=%d" % (self._base, expire)
+            self._session = self._do_post(url)['id']
+        else:
+            self._session = id
+        return self._session
+
+    def close_session(self):
+        """ Close the session on the server """
+        if self._session is not None:
+            url = '%s/sessions/v1/%s' % (self._base, self._session)
+            self._do_delete(url)
+            self._session = None
+
+    def heartbeat(self):
+        """ Send a heartbeat request. This is needed in order to keep a session alive """
+        if self._session is not None:
+            url = '%s/sessions/v1/%s' % (self._base, self._session)
+            self._do_put(url, '')
+
+    @contextmanager
+    def session(self, *args, **kwargs):
+        """ Session handling using a context manager """
+        yield self.start_session(*args, **kwargs)
+        self.close_session()
+
+    def get(self, path):
+        """ Get a node """
+        url = "%s/znodes/v1%s" % (self._base, path)
+        return self._do_get(url)
+
+    def get_children(self, path):
+        """ Get all the children for a given path. This function creates a generator """
+        url = "%s/znodes/v1%s?view=children" % (self._base, path)
+        resp = self._do_get(url)
+        for child in resp.get('children', []):
+            try:
+                yield self._do_get(resp['child_uri_template']\
+                    .replace('{child}', urllib2.quote(child)))
+            except ZooKeeper.NotFound:
+                continue
+
+    def create(self, path, data=None, sequence=False, ephemeral=False):
+        """ Create a new node. By default this call creates a persistent znode.
+
+        You can also create an ephemeral or a sequential znode.
+        """
+        ri = path.rindex('/')
+        head, name = path[:ri+1], path[ri+1:]
+        if head != '/': head = head[:-1]
+
+        flags = {
+            'null': 'true' if data is None else 'false',
+            'ephemeral': 'true' if ephemeral else 'false',
+            'sequence': 'true' if sequence else 'false'
+        }
+        if ephemeral:
+            if self._session:
+                flags['session'] = self._session
+            else:
+                raise ZooKeeper.Error, 'You need a session '\
+                    'to create an ephemeral node'
+        flags = urllib.urlencode(flags)
+
+        url = "%s/znodes/v1%s?op=create&name=%s&%s" % \
+            (self._base, head, name, flags)
+
+        return self._do_post(url, data)
+
+    def set(self, path, data=None, version=-1, null=False):
+        """ Set the value of node """
+        url = "%s/znodes/v1%s?%s" % (self._base, path, \
+            urllib.urlencode({
+                'version': version,
+                'null': 'true' if null else 'false'
+        }))
+        return self._do_put(url, data)
+
+    def delete(self, path, version=-1):
+        """ Delete a znode """
+        if type(path) is list:
+            map(lambda el: self.delete(el, version), path)
+            return
+
+        url = '%s/znodes/v1%s?%s' % (self._base, path, \
+            urllib.urlencode({
+                'version':version
+        }))
+        try:
+            return self._do_delete(url)
+        except urllib2.HTTPError, e:
+            if e.code == 412:
+                raise ZooKeeper.WrongVersion(path)
+            elif e.code == 404:
+                raise ZooKeeper.NotFound(path)
+            raise
+
+    def exists(self, path):
+        """ Do a znode exists """
+        try:
+            self.get(path)
+            return True
+        except ZooKeeper.NotFound:
+            return False
+
+    def _do_get(self, uri):
+        """ Send a GET request and convert errors to exceptions """
+        try:
+            req = urllib2.urlopen(uri)
+            resp = simplejson.load(req)
+
+            if 'Error' in resp:
+               raise ZooKeeper.Error(resp['Error'])
+
+            return resp
+        except urllib2.HTTPError, e:
+            if e.code == 404:
+                raise ZooKeeper.NotFound(uri)
+            raise
+
+    def _do_post(self, uri, data=None):
+        """ Send a POST request and convert errors to exceptions """
+        try:
+            req = urllib2.Request(uri, {})
+            req.add_header('Content-Type', 'application/octet-stream')
+            if data is not None:
+                req.add_data(data)
+
+            resp = simplejson.load(urllib2.urlopen(req))
+            if 'Error' in resp:
+                raise ZooKeeper.Error(resp['Error'])
+            return resp
+
+        except urllib2.HTTPError, e:
+            if e.code == 201:
+                return True
+            elif e.code == 409:
+                raise ZooKeeper.ZNodeExists(uri)
+            elif e.code == 401:
+                raise ZooKeeper.InvalidSession(uri)
+            raise
+
+    def _do_delete(self, uri):
+        """ Send a DELETE request """
+        req = RequestWithMethod(uri)
+        req.set_method('DELETE')
+        req.add_header('Content-Type', 'application/octet-stream')
+        return urllib2.urlopen(req).read()
+
+    def _do_put(self, uri, data):
+        """ Send a PUT request """
+        try:
+            req = RequestWithMethod(uri)
+            req.set_method('PUT')
+            req.add_header('Content-Type', 'application/octet-stream')
+            if data is not None:
+                req.add_data(data)
+
+            return urllib2.urlopen(req).read()
+        except urllib2.HTTPError, e:
+            if e.code == 412: # precondition failed
+                raise ZooKeeper.WrongVersion(uri)
+            raise
+

Modified: hadoop/zookeeper/trunk/src/contrib/rest/src/test/org/apache/zookeeper/server/jersey/Base.java
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/contrib/rest/src/test/org/apache/zookeeper/server/jersey/Base.java?rev=986215&r1=986214&r2=986215&view=diff
==============================================================================
--- hadoop/zookeeper/trunk/src/contrib/rest/src/test/org/apache/zookeeper/server/jersey/Base.java (original)
+++ hadoop/zookeeper/trunk/src/contrib/rest/src/test/org/apache/zookeeper/server/jersey/Base.java Tue Aug 17 07:24:09 2010
@@ -18,6 +18,8 @@
 
 package org.apache.zookeeper.server.jersey;
 
+import java.io.ByteArrayInputStream;
+
 import junit.framework.TestCase;
 
 import org.apache.log4j.Logger;
@@ -25,65 +27,67 @@ import org.apache.zookeeper.CreateMode;
 import org.apache.zookeeper.ZooKeeper;
 import org.apache.zookeeper.ZooDefs.Ids;
 import org.apache.zookeeper.server.jersey.SetTest.MyWatcher;
+import org.apache.zookeeper.server.jersey.cfg.RestCfg;
 import org.junit.After;
 import org.junit.Before;
 
-import com.sun.grizzly.http.SelectorThread;
 import com.sun.jersey.api.client.Client;
 import com.sun.jersey.api.client.WebResource;
 
-
 /**
  * Test stand-alone server.
- *
+ * 
  */
 public class Base extends TestCase {
-    protected static final Logger LOG = Logger.getLogger(Base.class);
-
-    protected static final String BASEURI = "http://localhost:10104/";
-    protected static final String ZKHOSTPORT = "localhost:22182";
-    protected Client c;
-    protected WebResource r;
-
-    protected ZooKeeper zk;
+   protected static final Logger LOG = Logger.getLogger(Base.class);
 
-    private SelectorThread threadSelector;
+   protected static final String CONTEXT_PATH = "/zk";
+   protected static final int GRIZZLY_PORT = 10104;
+   protected static final String BASEURI = String.format(
+           "http://localhost:%d%s", GRIZZLY_PORT, CONTEXT_PATH);
+   protected static final String ZKHOSTPORT = "localhost:22182";
+   protected Client client;
+   protected WebResource znodesr, sessionsr;
 
-    @Before
-    public void setUp() throws Exception {
-        super.setUp();
+   protected ZooKeeper zk;
 
-        ZooKeeperService.mapUriBase(BASEURI, ZKHOSTPORT);
+   private RestMain rest;
 
-        RestMain main = new RestMain(BASEURI);
-        threadSelector = main.execute();
+   @Before
+   public void setUp() throws Exception {
+       super.setUp();
 
-        zk = new ZooKeeper(ZKHOSTPORT, 30000, new MyWatcher());
+       RestCfg cfg = new RestCfg(new ByteArrayInputStream(String.format(
+               "rest.port=%s\n" + 
+               "rest.endpoint.1=%s;%s\n",
+               GRIZZLY_PORT, CONTEXT_PATH, ZKHOSTPORT).getBytes()));
 
-        c = Client.create();
-        r = c.resource(BASEURI);
-        r = r.path("znodes/v1");
-    }
+       rest = new RestMain(cfg);
+       rest.start();
 
-    @After
-    public void tearDown() throws Exception {
-        super.tearDown();
+       zk = new ZooKeeper(ZKHOSTPORT, 30000, new MyWatcher());
 
-        c.destroy();
+       client = Client.create();
+       znodesr = client.resource(BASEURI).path("znodes/v1");
+       sessionsr = client.resource(BASEURI).path("sessions/v1/");
+   }
 
-        zk.close();
-        ZooKeeperService.close(BASEURI);
+   @After
+   public void tearDown() throws Exception {
+       super.tearDown();
 
-        threadSelector.stopEndpoint();
-    }
+       client.destroy();
+       zk.close();
+       rest.stop();
+   }
 
-    protected static String createBaseZNode() throws Exception {
-        ZooKeeper zk = new ZooKeeper(ZKHOSTPORT, 30000, new MyWatcher());
+   protected static String createBaseZNode() throws Exception {
+       ZooKeeper zk = new ZooKeeper(ZKHOSTPORT, 30000, new MyWatcher());
 
-        String baseZnode = zk.create("/test-", null, Ids.OPEN_ACL_UNSAFE,
-                CreateMode.PERSISTENT_SEQUENTIAL);
-        zk.close();
+       String baseZnode = zk.create("/test-", null, Ids.OPEN_ACL_UNSAFE,
+               CreateMode.PERSISTENT_SEQUENTIAL);
+       zk.close();
 
-        return baseZnode;
-    }
+       return baseZnode;
+   }
 }

Modified: hadoop/zookeeper/trunk/src/contrib/rest/src/test/org/apache/zookeeper/server/jersey/CreateTest.java
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/contrib/rest/src/test/org/apache/zookeeper/server/jersey/CreateTest.java?rev=986215&r1=986214&r2=986215&view=diff
==============================================================================
--- hadoop/zookeeper/trunk/src/contrib/rest/src/test/org/apache/zookeeper/server/jersey/CreateTest.java (original)
+++ hadoop/zookeeper/trunk/src/contrib/rest/src/test/org/apache/zookeeper/server/jersey/CreateTest.java Tue Aug 17 07:24:09 2010
@@ -116,7 +116,7 @@ public class CreateTest extends Base {
     public void testCreate() throws Exception {
         LOG.info("STARTING " + getName());
 
-        WebResource wr = r.path(path).queryParam("dataformat", encoding)
+        WebResource wr = znodesr.path(path).queryParam("dataformat", encoding)
             .queryParam("name", name);
         if (data == null) {
             wr = wr.queryParam("null", "true");
@@ -142,10 +142,10 @@ public class CreateTest extends Base {
         ZPath zpath = cr.getEntity(ZPath.class);
         if (sequence) {
             assertTrue(zpath.path.startsWith(expectedPath.path));
-            assertTrue(zpath.uri.startsWith(r.path(path).toString()));
+            assertTrue(zpath.uri.startsWith(znodesr.path(path).toString()));
         } else {
             assertEquals(expectedPath, zpath);
-            assertEquals(r.path(path).toString(), zpath.uri);
+            assertEquals(znodesr.path(path).toString(), zpath.uri);
         }
 
         // use out-of-band method to verify

Modified: hadoop/zookeeper/trunk/src/contrib/rest/src/test/org/apache/zookeeper/server/jersey/DeleteTest.java
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/contrib/rest/src/test/org/apache/zookeeper/server/jersey/DeleteTest.java?rev=986215&r1=986214&r2=986215&view=diff
==============================================================================
--- hadoop/zookeeper/trunk/src/contrib/rest/src/test/org/apache/zookeeper/server/jersey/DeleteTest.java (original)
+++ hadoop/zookeeper/trunk/src/contrib/rest/src/test/org/apache/zookeeper/server/jersey/DeleteTest.java Tue Aug 17 07:24:09 2010
@@ -75,7 +75,7 @@ public class DeleteTest extends Base {
                     CreateMode.PERSISTENT_SEQUENTIAL);
         }
 
-        ClientResponse cr = r.path(zpath).accept(type).type(type)
+        ClientResponse cr = znodesr.path(zpath).accept(type).type(type)
             .delete(ClientResponse.class);
         assertEquals(expectedStatus, cr.getClientResponseStatus());
 

Modified: hadoop/zookeeper/trunk/src/contrib/rest/src/test/org/apache/zookeeper/server/jersey/ExistsTest.java
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/contrib/rest/src/test/org/apache/zookeeper/server/jersey/ExistsTest.java?rev=986215&r1=986214&r2=986215&view=diff
==============================================================================
--- hadoop/zookeeper/trunk/src/contrib/rest/src/test/org/apache/zookeeper/server/jersey/ExistsTest.java (original)
+++ hadoop/zookeeper/trunk/src/contrib/rest/src/test/org/apache/zookeeper/server/jersey/ExistsTest.java Tue Aug 17 07:24:09 2010
@@ -59,7 +59,7 @@ public class ExistsTest extends Base {
     }
 
     private void verify(String type) {
-        ClientResponse cr = r.path(path).accept(type).type(type).head();
+        ClientResponse cr = znodesr.path(path).accept(type).type(type).head();
         if (type.equals(MediaType.APPLICATION_OCTET_STREAM)
                 && expectedStatus == ClientResponse.Status.OK) {
             assertEquals(ClientResponse.Status.NO_CONTENT,

Modified: hadoop/zookeeper/trunk/src/contrib/rest/src/test/org/apache/zookeeper/server/jersey/GetChildrenTest.java
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/contrib/rest/src/test/org/apache/zookeeper/server/jersey/GetChildrenTest.java?rev=986215&r1=986214&r2=986215&view=diff
==============================================================================
--- hadoop/zookeeper/trunk/src/contrib/rest/src/test/org/apache/zookeeper/server/jersey/GetChildrenTest.java (original)
+++ hadoop/zookeeper/trunk/src/contrib/rest/src/test/org/apache/zookeeper/server/jersey/GetChildrenTest.java Tue Aug 17 07:24:09 2010
@@ -107,7 +107,7 @@ public class GetChildrenTest extends Bas
             }
         }
 
-        ClientResponse cr = r.path(path).queryParam("view", "children")
+        ClientResponse cr = znodesr.path(path).queryParam("view", "children")
             .accept(accept).get(ClientResponse.class);
         assertEquals(expectedStatus, cr.getClientResponseStatus());
 
@@ -120,16 +120,16 @@ public class GetChildrenTest extends Bas
             Collections.sort(expectedChildren);
             Collections.sort(zchildren.children);
             assertEquals(expectedChildren, zchildren.children);
-            assertEquals(r.path(path).toString(), zchildren.uri);
-            assertEquals(r.path(path).toString() + "/{child}",
+            assertEquals(znodesr.path(path).toString(), zchildren.uri);
+            assertEquals(znodesr.path(path).toString() + "/{child}",
                     zchildren.child_uri_template);
         } else if (accept.equals(MediaType.APPLICATION_XML)) {
             ZChildren zchildren = cr.getEntity(ZChildren.class);
             Collections.sort(expectedChildren);
             Collections.sort(zchildren.children);
             assertEquals(expectedChildren, zchildren.children);
-            assertEquals(r.path(path).toString(), zchildren.uri);
-            assertEquals(r.path(path).toString() + "/{child}",
+            assertEquals(znodesr.path(path).toString(), zchildren.uri);
+            assertEquals(znodesr.path(path).toString() + "/{child}",
                     zchildren.child_uri_template);
         } else {
             fail("unknown accept type");

Modified: hadoop/zookeeper/trunk/src/contrib/rest/src/test/org/apache/zookeeper/server/jersey/GetTest.java
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/contrib/rest/src/test/org/apache/zookeeper/server/jersey/GetTest.java?rev=986215&r1=986214&r2=986215&view=diff
==============================================================================
--- hadoop/zookeeper/trunk/src/contrib/rest/src/test/org/apache/zookeeper/server/jersey/GetTest.java (original)
+++ hadoop/zookeeper/trunk/src/contrib/rest/src/test/org/apache/zookeeper/server/jersey/GetTest.java Tue Aug 17 07:24:09 2010
@@ -107,7 +107,7 @@ public class GetTest extends Base {
             }
         }
 
-        ClientResponse cr = r.path(path).queryParam("dataformat", encoding)
+        ClientResponse cr = znodesr.path(path).queryParam("dataformat", encoding)
             .accept(accept).get(ClientResponse.class);
         assertEquals(expectedStatus, cr.getClientResponseStatus());
 
@@ -117,6 +117,6 @@ public class GetTest extends Base {
 
         ZStat zstat = cr.getEntity(ZStat.class);
         assertEquals(expectedStat, zstat);
-        assertEquals(r.path(path).toString(), zstat.uri);
+        assertEquals(znodesr.path(path).toString(), zstat.uri);
     }
 }

Modified: hadoop/zookeeper/trunk/src/contrib/rest/src/test/org/apache/zookeeper/server/jersey/RootTest.java
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/contrib/rest/src/test/org/apache/zookeeper/server/jersey/RootTest.java?rev=986215&r1=986214&r2=986215&view=diff
==============================================================================
--- hadoop/zookeeper/trunk/src/contrib/rest/src/test/org/apache/zookeeper/server/jersey/RootTest.java (original)
+++ hadoop/zookeeper/trunk/src/contrib/rest/src/test/org/apache/zookeeper/server/jersey/RootTest.java Tue Aug 17 07:24:09 2010
@@ -19,19 +19,13 @@
 package org.apache.zookeeper.server.jersey;
 
 import java.util.Arrays;
-import java.util.Collection;
 
 import javax.ws.rs.core.MediaType;
 
 import org.apache.log4j.Logger;
-import org.apache.zookeeper.WatchedEvent;
-import org.apache.zookeeper.Watcher;
 import org.apache.zookeeper.data.Stat;
 import org.apache.zookeeper.server.jersey.jaxb.ZPath;
 import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-import org.junit.runners.Parameterized.Parameters;
 
 import com.sun.jersey.api.client.ClientResponse;
 import com.sun.jersey.api.client.WebResource;
@@ -53,7 +47,7 @@ public class RootTest extends Base {
         String name = "roottest-create";
         byte[] data = "foo".getBytes();
 
-        WebResource wr = r.path(path).queryParam("dataformat", "utf8")
+        WebResource wr = znodesr.path(path).queryParam("dataformat", "utf8")
             .queryParam("name", name);
         Builder builder = wr.accept(MediaType.APPLICATION_JSON);
 
@@ -63,7 +57,7 @@ public class RootTest extends Base {
 
         ZPath zpath = cr.getEntity(ZPath.class);
         assertEquals(new ZPath(path + name), zpath);
-        assertEquals(r.path(path).toString(), zpath.uri);
+        assertEquals(znodesr.path(path).toString(), zpath.uri);
 
         // use out-of-band method to verify
         byte[] rdata = zk.getData(zpath.path, false, new Stat());

Added: hadoop/zookeeper/trunk/src/contrib/rest/src/test/org/apache/zookeeper/server/jersey/SessionTest.java
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/contrib/rest/src/test/org/apache/zookeeper/server/jersey/SessionTest.java?rev=986215&view=auto
==============================================================================
--- hadoop/zookeeper/trunk/src/contrib/rest/src/test/org/apache/zookeeper/server/jersey/SessionTest.java (added)
+++ hadoop/zookeeper/trunk/src/contrib/rest/src/test/org/apache/zookeeper/server/jersey/SessionTest.java Tue Aug 17 07:24:09 2010
@@ -0,0 +1,133 @@
+/**
+ * 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.jersey;
+
+import java.io.IOException;
+
+import javax.ws.rs.core.MediaType;
+
+import org.apache.log4j.Logger;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.ZooKeeper;
+import org.apache.zookeeper.data.Stat;
+import org.apache.zookeeper.server.jersey.jaxb.ZSession;
+import org.codehaus.jettison.json.JSONException;
+import org.junit.Test;
+
+import com.sun.jersey.api.client.Client;
+import com.sun.jersey.api.client.ClientResponse;
+import com.sun.jersey.api.client.WebResource;
+import com.sun.jersey.api.client.WebResource.Builder;
+
+public class SessionTest extends Base {
+    protected static final Logger LOG = Logger.getLogger(SessionTest.class);
+
+    private ZSession createSession() {
+        return createSession("30");
+    }
+
+    private ZSession createSession(String expire) {
+        WebResource wr = sessionsr.queryParam("op", "create")
+            .queryParam("expire", expire);
+        Builder b = wr.accept(MediaType.APPLICATION_JSON);
+
+        ClientResponse cr = b.post(ClientResponse.class, null);
+        assertEquals(ClientResponse.Status.CREATED, cr
+                .getClientResponseStatus());
+
+        return cr.getEntity(ZSession.class);
+    }
+
+    @Test
+    public void testCreateNewSession() throws JSONException {
+        ZSession session = createSession();
+        assertEquals(session.id.length(), 36);
+
+        // use out-of-band method to verify
+        assertTrue(ZooKeeperService.isConnected(CONTEXT_PATH, session.id));
+    }
+
+    @Test
+    public void testSessionExpires() throws InterruptedException {
+        ZSession session = createSession("1");
+
+        // use out-of-band method to verify
+        assertTrue(ZooKeeperService.isConnected(CONTEXT_PATH, session.id));
+
+        // wait for the session to be closed
+        Thread.sleep(1500);
+        assertFalse(ZooKeeperService.isConnected(CONTEXT_PATH, session.id));
+    }
+
+    @Test
+    public void testDeleteSession() {
+        ZSession session = createSession("30");
+
+        WebResource wr = sessionsr.path(session.id);
+        Builder b = wr.accept(MediaType.APPLICATION_JSON);
+
+        assertTrue(ZooKeeperService.isConnected(CONTEXT_PATH, session.id));
+        ClientResponse cr = b.delete(ClientResponse.class, null);
+        assertEquals(ClientResponse.Status.NO_CONTENT, 
+                cr.getClientResponseStatus());
+
+        assertFalse(ZooKeeperService.isConnected(CONTEXT_PATH, session.id));
+    }
+    
+    @Test
+    public void testSendHeartbeat() throws InterruptedException {
+        ZSession session = createSession("2");
+        
+        Thread.sleep(1000);
+        WebResource wr = sessionsr.path(session.id);
+        Builder b = wr.accept(MediaType.APPLICATION_JSON);
+        
+        ClientResponse cr = b.put(ClientResponse.class, null);
+        assertEquals(ClientResponse.Status.OK, cr.getClientResponseStatus());
+        
+        Thread.sleep(1500);
+        assertTrue(ZooKeeperService.isConnected(CONTEXT_PATH, session.id));
+        
+        Thread.sleep(1000);
+        assertFalse(ZooKeeperService.isConnected(CONTEXT_PATH, session.id));
+    }
+    
+    @Test
+    public void testCreateEphemeralZNode() 
+    throws KeeperException, InterruptedException, IOException {
+        ZSession session = createSession("30");
+        
+        WebResource wr = znodesr.path("/")
+            .queryParam("op", "create")
+            .queryParam("name", "ephemeral-test")
+            .queryParam("ephemeral", "true")
+            .queryParam("session", session.id)
+            .queryParam("null", "true");
+        
+        Builder b = wr.accept(MediaType.APPLICATION_JSON);
+        ClientResponse cr = b.post(ClientResponse.class);
+        assertEquals(ClientResponse.Status.CREATED, cr.getClientResponseStatus());
+        
+        Stat stat = new Stat();
+        zk.getData("/ephemeral-test", false, stat);
+        
+        ZooKeeper sessionZK = ZooKeeperService.getClient(CONTEXT_PATH, session.id);
+        assertEquals(stat.getEphemeralOwner(), sessionZK.getSessionId());
+    }
+}

Modified: hadoop/zookeeper/trunk/src/contrib/rest/src/test/org/apache/zookeeper/server/jersey/SetTest.java
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/contrib/rest/src/test/org/apache/zookeeper/server/jersey/SetTest.java?rev=986215&r1=986214&r2=986215&view=diff
==============================================================================
--- hadoop/zookeeper/trunk/src/contrib/rest/src/test/org/apache/zookeeper/server/jersey/SetTest.java (original)
+++ hadoop/zookeeper/trunk/src/contrib/rest/src/test/org/apache/zookeeper/server/jersey/SetTest.java Tue Aug 17 07:24:09 2010
@@ -112,7 +112,7 @@ public class SetTest extends Base {
                     CreateMode.PERSISTENT);
         }
 
-        WebResource wr = r.path(path).queryParam("dataformat", encoding);
+        WebResource wr = znodesr.path(path).queryParam("dataformat", encoding);
         if (data == null) {
             wr = wr.queryParam("null", "true");
         }

Modified: hadoop/zookeeper/trunk/src/contrib/rest/src/test/org/apache/zookeeper/server/jersey/WadlTest.java
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/contrib/rest/src/test/org/apache/zookeeper/server/jersey/WadlTest.java?rev=986215&r1=986214&r2=986215&view=diff
==============================================================================
--- hadoop/zookeeper/trunk/src/contrib/rest/src/test/org/apache/zookeeper/server/jersey/WadlTest.java (original)
+++ hadoop/zookeeper/trunk/src/contrib/rest/src/test/org/apache/zookeeper/server/jersey/WadlTest.java Tue Aug 17 07:24:09 2010
@@ -34,7 +34,7 @@ public class WadlTest extends Base {
 
     @Test
     public void testApplicationWadl() {
-        WebResource r = c.resource(BASEURI);
+        WebResource r = client.resource(BASEURI);
         String serviceWadl = r.path("application.wadl").
                 accept(MediaTypes.WADL).get(String.class);
         assertTrue("Something wrong. Returned wadl length not > 0.",