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 2012/03/17 01:31:16 UTC

svn commit: r1301837 - in /zookeeper/trunk: ./ src/java/main/org/apache/zookeeper/ src/java/test/org/apache/zookeeper/test/

Author: phunt
Date: Sat Mar 17 00:31:16 2012
New Revision: 1301837

URL: http://svn.apache.org/viewvc?rev=1301837&view=rev
Log:
ZOOKEEPER-1344. ZooKeeper client multi-update command is not considering the Chroot request (Rakesh R via phunt)

Modified:
    zookeeper/trunk/CHANGES.txt
    zookeeper/trunk/src/java/main/org/apache/zookeeper/Op.java
    zookeeper/trunk/src/java/main/org/apache/zookeeper/Transaction.java
    zookeeper/trunk/src/java/main/org/apache/zookeeper/ZooKeeper.java
    zookeeper/trunk/src/java/test/org/apache/zookeeper/test/MultiTransactionTest.java

Modified: zookeeper/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/zookeeper/trunk/CHANGES.txt?rev=1301837&r1=1301836&r2=1301837&view=diff
==============================================================================
--- zookeeper/trunk/CHANGES.txt (original)
+++ zookeeper/trunk/CHANGES.txt Sat Mar 17 00:31:16 2012
@@ -145,6 +145,9 @@ BUGFIXES:
 
   ZOOKEEPER-1412. java client watches inconsistently triggered on
   reconnect (phunt)
+
+  ZOOKEEPER-1344. ZooKeeper client multi-update command is not
+  considering the Chroot request (Rakesh R via phunt)
   
 IMPROVEMENTS:
 

Modified: zookeeper/trunk/src/java/main/org/apache/zookeeper/Op.java
URL: http://svn.apache.org/viewvc/zookeeper/trunk/src/java/main/org/apache/zookeeper/Op.java?rev=1301837&r1=1301836&r2=1301837&view=diff
==============================================================================
--- zookeeper/trunk/src/java/main/org/apache/zookeeper/Op.java (original)
+++ zookeeper/trunk/src/java/main/org/apache/zookeeper/Op.java Sat Mar 17 00:31:16 2012
@@ -154,6 +154,12 @@ public abstract class Op {
      * @return An appropriate Record structure.
      */
     public abstract Record toRequestRecord() ;
+    
+    /**
+     * Reconstructs the transaction with the chroot prefix.
+     * @return transaction with chroot.
+     */
+    abstract Op withChroot(String addRootPrefix);
 
     //////////////////
     // these internal classes are public, but should not generally be referenced.
@@ -210,6 +216,11 @@ public abstract class Op {
         public Record toRequestRecord() {
             return new CreateRequest(getPath(), data, acl, flags);
         }
+
+        @Override
+        Op withChroot(String path) {
+            return new Create(path, data, acl, flags);
+        }
     }
 
     public static class Delete extends Op {
@@ -240,6 +251,11 @@ public abstract class Op {
         public Record toRequestRecord() {
             return new DeleteRequest(getPath(), version);
         }
+
+        @Override
+        Op withChroot(String path) {
+            return new Delete(path, version);
+        }
     }
 
     public static class SetData extends Op {
@@ -272,6 +288,11 @@ public abstract class Op {
         public Record toRequestRecord() {
             return new SetDataRequest(getPath(), data, version);
         }
+
+        @Override
+        Op withChroot(String path) {
+            return new SetData(path, data, version);
+        }
     }
 
     public static class Check extends Op {
@@ -301,5 +322,11 @@ public abstract class Op {
         public Record toRequestRecord() {
             return new CheckVersionRequest(getPath(), version);
         }
+
+        @Override
+        Op withChroot(String path) {
+            return new Check(path, version);
+        }
     }
+
 }

Modified: zookeeper/trunk/src/java/main/org/apache/zookeeper/Transaction.java
URL: http://svn.apache.org/viewvc/zookeeper/trunk/src/java/main/org/apache/zookeeper/Transaction.java?rev=1301837&r1=1301836&r2=1301837&view=diff
==============================================================================
--- zookeeper/trunk/src/java/main/org/apache/zookeeper/Transaction.java (original)
+++ zookeeper/trunk/src/java/main/org/apache/zookeeper/Transaction.java Sat Mar 17 00:31:16 2012
@@ -18,7 +18,7 @@
 package org.apache.zookeeper;
 
 import org.apache.zookeeper.data.ACL;
-
+import java.util.ArrayList;
 import java.util.List;
 
 /**
@@ -30,7 +30,7 @@ import java.util.List;
  */
 public class Transaction {
     private ZooKeeper zk;
-    private MultiTransactionRecord request = new MultiTransactionRecord();
+    private List<Op> ops = new ArrayList<Op>();
 
     protected Transaction(ZooKeeper zk) {
         this.zk = zk;
@@ -38,26 +38,26 @@ public class Transaction {
 
     public Transaction create(final String path, byte data[], List<ACL> acl,
                               CreateMode createMode) {
-        request.add(Op.create(path, data, acl, createMode.toFlag()));
+        ops.add(Op.create(path, data, acl, createMode.toFlag()));
         return this;
     }
 
     public Transaction delete(final String path, int version) {
-        request.add(Op.delete(path, version));
+        ops.add(Op.delete(path, version));
         return this;
     }
 
     public Transaction check(String path, int version) {
-        request.add(Op.check(path, version));
+        ops.add(Op.check(path, version));
         return this;
     }
 
     public Transaction setData(final String path, byte data[], int version) {
-        request.add(Op.setData(path, data, version));
+        ops.add(Op.setData(path, data, version));
         return this;
     }
 
     public List<OpResult> commit() throws InterruptedException, KeeperException {
-        return zk.multiInternal(request);
+        return zk.multi(ops);
     }
 }

Modified: zookeeper/trunk/src/java/main/org/apache/zookeeper/ZooKeeper.java
URL: http://svn.apache.org/viewvc/zookeeper/trunk/src/java/main/org/apache/zookeeper/ZooKeeper.java?rev=1301837&r1=1301836&r2=1301837&view=diff
==============================================================================
--- zookeeper/trunk/src/java/main/org/apache/zookeeper/ZooKeeper.java (original)
+++ zookeeper/trunk/src/java/main/org/apache/zookeeper/ZooKeeper.java Sat Mar 17 00:31:16 2012
@@ -898,7 +898,22 @@ public class ZooKeeper {
      * @since 3.4.0
      */
     public List<OpResult> multi(Iterable<Op> ops) throws InterruptedException, KeeperException {
-        return multiInternal(new MultiTransactionRecord(ops));
+        // reconstructing transaction with the chroot prefix
+        List<Op> transaction = new ArrayList<Op>();
+        for (Op op : ops) {
+            transaction.add(withRootPrefix(op));
+        }
+        return multiInternal(new MultiTransactionRecord(transaction));
+    }
+    
+    private Op withRootPrefix(Op op) {
+        if (null != op.getPath()) {
+            final String serverPath = prependChroot(op.getPath());
+            if (!op.getPath().equals(serverPath)) {
+                return op.withChroot(serverPath);
+            }
+        }
+        return op;
     }
 
     protected List<OpResult> multiInternal(MultiTransactionRecord request)

Modified: zookeeper/trunk/src/java/test/org/apache/zookeeper/test/MultiTransactionTest.java
URL: http://svn.apache.org/viewvc/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/MultiTransactionTest.java?rev=1301837&r1=1301836&r2=1301837&view=diff
==============================================================================
--- zookeeper/trunk/src/java/test/org/apache/zookeeper/test/MultiTransactionTest.java (original)
+++ zookeeper/trunk/src/java/test/org/apache/zookeeper/test/MultiTransactionTest.java Sat Mar 17 00:31:16 2012
@@ -31,6 +31,7 @@ import org.apache.zookeeper.CreateMode;
 import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.Op;
 import org.apache.zookeeper.OpResult;
+import org.apache.zookeeper.Transaction;
 import org.apache.zookeeper.WatchedEvent;
 import org.apache.zookeeper.Watcher;
 import org.apache.zookeeper.ZooKeeper;
@@ -48,8 +49,8 @@ import org.junit.Test;
 
 public class MultiTransactionTest extends ClientBase {
     private static final Logger LOG = Logger.getLogger(MultiTransactionTest.class);
-
     private ZooKeeper zk;
+    private ZooKeeper zk_chroot;
 
     @Before
     public void setUp() throws Exception {
@@ -57,6 +58,119 @@ public class MultiTransactionTest extend
         super.setUp();
         zk = createClient();
     }
+    
+    @Test
+    public void testChRootCreateDelete() throws Exception {
+        // creating the subtree for chRoot clients.
+        String chRoot = createNameSpace();
+        // Creating child using chRoot client.
+        zk_chroot = createClient(this.hostPort + chRoot);
+        Op createChild = Op.create("/myid", new byte[0],
+                Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
+        zk_chroot.multi(Arrays.asList(createChild));
+        
+        Assert.assertNotNull("zNode is not created under chroot:" + chRoot, zk
+                .exists(chRoot + "/myid", false));
+        Assert.assertNotNull("zNode is not created under chroot:" + chRoot,
+                zk_chroot.exists("/myid", false));
+        Assert.assertNull("zNode is created directly under '/', ignored configured chroot",
+                zk.exists("/myid", false));
+        
+        // Deleting child using chRoot client.
+        Op deleteChild = Op.delete("/myid", 0);
+        zk_chroot.multi(Arrays.asList(deleteChild));
+        Assert.assertNull("zNode exists under chroot:" + chRoot, zk.exists(
+                chRoot + "/myid", false));
+        Assert.assertNull("zNode exists under chroot:" + chRoot, zk_chroot
+                .exists("/myid", false));
+    }
+
+    @Test
+    public void testChRootSetData() throws Exception {
+        // creating the subtree for chRoot clients.
+        String chRoot = createNameSpace();
+        // setData using chRoot client.
+        zk_chroot = createClient(this.hostPort + chRoot);
+        String[] names = {"/multi0", "/multi1", "/multi2"};
+        List<Op> ops = new ArrayList<Op>();
+
+        for (int i = 0; i < names.length; i++) {
+            ops.add(Op.create(names[i], new byte[0], Ids.OPEN_ACL_UNSAFE,
+                    CreateMode.PERSISTENT));
+            ops.add(Op.setData(names[i], names[i].getBytes(), 0));
+        }
+
+        zk_chroot.multi(ops) ;
+
+        for (int i = 0; i < names.length; i++) {
+            Assert.assertArrayEquals("zNode data not matching", names[i]
+                    .getBytes(), zk_chroot.getData(names[i], false, null));
+        }
+    }
+
+    @Test
+    public void testChRootCheck() throws Exception {
+        // creating the subtree for chRoot clients.
+        String chRoot = createNameSpace();
+        // checking the child version using chRoot client.
+        zk_chroot = createClient(this.hostPort + chRoot);
+        String[] names = {"/multi0", "/multi1", "/multi2"};
+        List<Op> ops = new ArrayList<Op>();
+
+        for (int i = 0; i < names.length; i++) {
+            zk.create(chRoot + names[i], new byte[0], Ids.OPEN_ACL_UNSAFE,
+                    CreateMode.PERSISTENT);
+        }
+        for (int i = 0; i < names.length; i++) {
+            ops.add(Op.check(names[i], 0));
+        }
+        zk_chroot.multi(ops) ;
+    }
+
+    @Test
+    public void testChRootTransaction() throws Exception {
+        // creating the subtree for chRoot clients.
+        String chRoot = createNameSpace();
+        // checking the child version using chRoot client.
+        zk_chroot = createClient(this.hostPort + chRoot);
+        String childPath = "/myid";
+        Transaction transaction = zk_chroot.transaction();
+        transaction.create(childPath, new byte[0], Ids.OPEN_ACL_UNSAFE,
+                CreateMode.PERSISTENT);
+        transaction.check(childPath, 0);
+        transaction.setData(childPath, childPath.getBytes(), 0);
+        transaction.commit();
+
+        Assert.assertNotNull("zNode is not created under chroot:" + chRoot, zk
+                .exists(chRoot + childPath, false));
+        Assert.assertNotNull("zNode is not created under chroot:" + chRoot,
+                zk_chroot.exists(childPath, false));
+        Assert.assertNull("zNode is created directly under '/', ignored configured chroot",
+                        zk.exists(childPath, false));
+        Assert.assertArrayEquals("zNode data not matching", childPath
+                .getBytes(), zk_chroot.getData(childPath, false, null));
+
+        transaction = zk_chroot.transaction();
+        // Deleting child using chRoot client.
+        transaction.delete(childPath, 1);
+        transaction.commit();
+
+        Assert.assertNull("chroot:" + chRoot + " exists after delete", zk
+                .exists(chRoot + "/myid", false));
+        Assert.assertNull("chroot:" + chRoot + " exists after delete",
+                zk_chroot.exists("/myid", false));
+    }
+
+    private String createNameSpace() throws InterruptedException,
+            KeeperException {
+        // creating the subtree for chRoot clients.
+        String chRoot = "/appsX";
+        Op createChRoot = Op.create(chRoot, new byte[0], Ids.OPEN_ACL_UNSAFE,
+                CreateMode.PERSISTENT);
+        zk.multi(Arrays.asList(createChRoot));
+        return chRoot;
+    }
+
 
     @Test
     public void testCreate() throws Exception {