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/18 07:03:06 UTC

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

Author: phunt
Date: Sun Mar 18 06:03:05 2012
New Revision: 1302068

URL: http://svn.apache.org/viewvc?rev=1302068&view=rev
Log:
ZOOKEEPER-271. Better command line parsing in ZookeeperMain. (Hartmut Lang via phunt)

Added:
    zookeeper/trunk/src/java/main/org/apache/zookeeper/cli/
    zookeeper/trunk/src/java/main/org/apache/zookeeper/cli/AclParser.java
    zookeeper/trunk/src/java/main/org/apache/zookeeper/cli/AddAuthCommand.java
    zookeeper/trunk/src/java/main/org/apache/zookeeper/cli/CliCommand.java
    zookeeper/trunk/src/java/main/org/apache/zookeeper/cli/CloseCommand.java
    zookeeper/trunk/src/java/main/org/apache/zookeeper/cli/CreateCommand.java
    zookeeper/trunk/src/java/main/org/apache/zookeeper/cli/DelQuotaCommand.java
    zookeeper/trunk/src/java/main/org/apache/zookeeper/cli/DeleteAllCommand.java
    zookeeper/trunk/src/java/main/org/apache/zookeeper/cli/DeleteCommand.java
    zookeeper/trunk/src/java/main/org/apache/zookeeper/cli/GetAclCommand.java
    zookeeper/trunk/src/java/main/org/apache/zookeeper/cli/GetCommand.java
    zookeeper/trunk/src/java/main/org/apache/zookeeper/cli/ListQuotaCommand.java
    zookeeper/trunk/src/java/main/org/apache/zookeeper/cli/Ls2Command.java
    zookeeper/trunk/src/java/main/org/apache/zookeeper/cli/LsCommand.java
    zookeeper/trunk/src/java/main/org/apache/zookeeper/cli/SetAclCommand.java
    zookeeper/trunk/src/java/main/org/apache/zookeeper/cli/SetCommand.java
    zookeeper/trunk/src/java/main/org/apache/zookeeper/cli/SetQuotaCommand.java
    zookeeper/trunk/src/java/main/org/apache/zookeeper/cli/StatCommand.java
    zookeeper/trunk/src/java/main/org/apache/zookeeper/cli/StatPrinter.java
    zookeeper/trunk/src/java/main/org/apache/zookeeper/cli/SyncCommand.java
Modified:
    zookeeper/trunk/CHANGES.txt
    zookeeper/trunk/ivy.xml
    zookeeper/trunk/src/java/main/org/apache/zookeeper/ZooKeeperMain.java
    zookeeper/trunk/src/java/test/org/apache/zookeeper/ZooKeeperTest.java

Modified: zookeeper/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/zookeeper/trunk/CHANGES.txt?rev=1302068&r1=1302067&r2=1302068&view=diff
==============================================================================
--- zookeeper/trunk/CHANGES.txt (original)
+++ zookeeper/trunk/CHANGES.txt Sun Mar 18 06:03:05 2012
@@ -246,11 +246,14 @@ IMPROVEMENTS:
   in order to get rid of the intermediate shell process
   (Roman Shaposhnik via phunt)
 
-  ZOOKEEPER-1161. Provide an option for disabling auto-creation of the data directory 
-  (phunt via henry)
+  ZOOKEEPER-1161. Provide an option for disabling auto-creation of the
+  data directory (phunt via henry)
   
   ZOOKEEPER-1397. Remove BookKeeper documentation links. (flavio via camille)
 
+  ZOOKEEPER-271. Better command line parsing in ZookeeperMain.
+  (Hartmut Lang via phunt)
+
 
 Release 3.4.0 - 
 

Modified: zookeeper/trunk/ivy.xml
URL: http://svn.apache.org/viewvc/zookeeper/trunk/ivy.xml?rev=1302068&r1=1302067&r2=1302068&view=diff
==============================================================================
--- zookeeper/trunk/ivy.xml (original)
+++ zookeeper/trunk/ivy.xml Sun Mar 18 06:03:05 2012
@@ -41,6 +41,7 @@
   <dependencies>
     <dependency org="org.slf4j" name="slf4j-api" rev="1.6.2"/>
     <dependency org="org.slf4j" name="slf4j-log4j12" rev="1.6.2" transitive="false"/>
+    <dependency org="commons-cli" name="commons-cli" rev="1.2" />
   
     <!-- transitive false turns off dependency checking, log4j deps seem borked -->
     <dependency org="log4j" name="log4j" rev="1.2.16" transitive="false" conf="default"/>

Modified: zookeeper/trunk/src/java/main/org/apache/zookeeper/ZooKeeperMain.java
URL: http://svn.apache.org/viewvc/zookeeper/trunk/src/java/main/org/apache/zookeeper/ZooKeeperMain.java?rev=1302068&r1=1302067&r2=1302068&view=diff
==============================================================================
--- zookeeper/trunk/src/java/main/org/apache/zookeeper/ZooKeeperMain.java (original)
+++ zookeeper/trunk/src/java/main/org/apache/zookeeper/ZooKeeperMain.java Sun Mar 18 06:03:05 2012
@@ -25,22 +25,38 @@ import java.lang.reflect.InvocationTarge
 import java.lang.reflect.Method;
 import java.util.ArrayList;
 import java.util.Arrays;
-import java.util.Date;
+import java.util.Collections;
 import java.util.HashMap;
 import java.util.Iterator;
 import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
+import java.util.Map.Entry;
 import java.util.NoSuchElementException;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-import org.apache.zookeeper.AsyncCallback.DataCallback;
 import org.apache.zookeeper.ZooDefs.Ids;
-import org.apache.zookeeper.data.ACL;
-import org.apache.zookeeper.data.Id;
 import org.apache.zookeeper.data.Stat;
 import java.util.StringTokenizer;
+import org.apache.commons.cli.ParseException;
+import org.apache.zookeeper.cli.AddAuthCommand;
+import org.apache.zookeeper.cli.CliCommand;
+import org.apache.zookeeper.cli.CloseCommand;
+import org.apache.zookeeper.cli.CreateCommand;
+import org.apache.zookeeper.cli.DelQuotaCommand;
+import org.apache.zookeeper.cli.DeleteAllCommand;
+import org.apache.zookeeper.cli.DeleteCommand;
+import org.apache.zookeeper.cli.GetAclCommand;
+import org.apache.zookeeper.cli.GetCommand;
+import org.apache.zookeeper.cli.ListQuotaCommand;
+import org.apache.zookeeper.cli.Ls2Command;
+import org.apache.zookeeper.cli.LsCommand;
+import org.apache.zookeeper.cli.SetAclCommand;
+import org.apache.zookeeper.cli.SetCommand;
+import org.apache.zookeeper.cli.SetQuotaCommand;
+import org.apache.zookeeper.cli.StatCommand;
+import org.apache.zookeeper.cli.SyncCommand;
 
 /**
  * The command line client to ZooKeeper.
@@ -49,6 +65,8 @@ import java.util.StringTokenizer;
 public class ZooKeeperMain {
     private static final Logger LOG = LoggerFactory.getLogger(ZooKeeperMain.class);
     protected static final Map<String,String> commandMap = new HashMap<String,String>( );
+    protected static final Map<String,CliCommand> commandMapCli = 
+            new HashMap<String,CliCommand>( );
 
     protected MyCommandOptions cl = new MyCommandOptions();
     protected HashMap<Integer,String> history = new HashMap<Integer,String>( );
@@ -64,32 +82,41 @@ public class ZooKeeperMain {
 
     static {
         commandMap.put("connect", "host:port");
-        commandMap.put("close","");
-        commandMap.put("create", "[-s] [-e] path data acl");
-        commandMap.put("delete","path [version]");
-        commandMap.put("deleteall","path");
-        commandMap.put("rmr","path (Deprecated - Use 'deleteall' instead.)");
-        commandMap.put("set","path data [version]");
-        commandMap.put("get","path [watch]");
-        commandMap.put("ls","path [watch]");
-        commandMap.put("ls2","path [watch]");
-        commandMap.put("getAcl","path");
-        commandMap.put("setAcl","path acl");
-        commandMap.put("stat","path [watch]");
-        commandMap.put("sync","path");
-        commandMap.put("setquota","-n|-b val path");
-        commandMap.put("listquota","path");
-        commandMap.put("delquota","[-n|-b] path");
         commandMap.put("history","");
         commandMap.put("redo","cmdno");
         commandMap.put("printwatches", "on|off");
-        commandMap.put("quit","");
-        commandMap.put("addauth", "scheme auth");
+        commandMap.put("quit", "");
+
+        new CloseCommand().addToMap(commandMapCli);
+        new CreateCommand().addToMap(commandMapCli);
+        new DeleteCommand().addToMap(commandMapCli);
+        new DeleteAllCommand().addToMap(commandMapCli);
+        // Depricated: rmr
+        new DeleteAllCommand("rmr").addToMap(commandMapCli);
+        new SetCommand().addToMap(commandMapCli);
+        new GetCommand().addToMap(commandMapCli);
+        new LsCommand().addToMap(commandMapCli);
+        new Ls2Command().addToMap(commandMapCli);
+        new GetAclCommand().addToMap(commandMapCli);
+        new SetAclCommand().addToMap(commandMapCli);
+        new StatCommand().addToMap(commandMapCli);
+        new SyncCommand().addToMap(commandMapCli);
+        new SetQuotaCommand().addToMap(commandMapCli);
+        new ListQuotaCommand().addToMap(commandMapCli);
+        new DelQuotaCommand().addToMap(commandMapCli);
+        new AddAuthCommand().addToMap(commandMapCli);
+        
+        // add all to commandMap
+        for (Entry<String, CliCommand> entry : commandMapCli.entrySet()) {
+            commandMap.put(entry.getKey(), entry.getValue().getOptionStr());
+    }
     }
 
     static void usage() {
         System.err.println("ZooKeeper -server host:port cmd args");
-        for (String cmd : commandMap.keySet()) {
+        List<String> cmdList = new ArrayList<String>(commandMap.keySet());
+        Collections.sort(cmdList);
+        for (String cmd : cmdList) {
             System.err.println("\t"+cmd+ " " + commandMap.get(cmd));
         }
     }
@@ -103,48 +130,6 @@ public class ZooKeeperMain {
         }
     }
 
-    static private int getPermFromString(String permString) {
-        int perm = 0;
-        for (int i = 0; i < permString.length(); i++) {
-            switch (permString.charAt(i)) {
-            case 'r':
-                perm |= ZooDefs.Perms.READ;
-                break;
-            case 'w':
-                perm |= ZooDefs.Perms.WRITE;
-                break;
-            case 'c':
-                perm |= ZooDefs.Perms.CREATE;
-                break;
-            case 'd':
-                perm |= ZooDefs.Perms.DELETE;
-                break;
-            case 'a':
-                perm |= ZooDefs.Perms.ADMIN;
-                break;
-            default:
-                System.err
-                .println("Unknown perm type: " + permString.charAt(i));
-            }
-        }
-        return perm;
-    }
-
-    private static void printStat(Stat stat) {
-        System.err.println("cZxid = 0x" + Long.toHexString(stat.getCzxid()));
-        System.err.println("ctime = " + new Date(stat.getCtime()).toString());
-        System.err.println("mZxid = 0x" + Long.toHexString(stat.getMzxid()));
-        System.err.println("mtime = " + new Date(stat.getMtime()).toString());
-        System.err.println("pZxid = 0x" + Long.toHexString(stat.getPzxid()));
-        System.err.println("cversion = " + stat.getCversion());
-        System.err.println("dataVersion = " + stat.getVersion());
-        System.err.println("aclVersion = " + stat.getAversion());
-        System.err.println("ephemeralOwner = 0x"
-        		+ Long.toHexString(stat.getEphemeralOwner()));
-        System.err.println("dataLength = " + stat.getDataLength());
-        System.err.println("numChildren = " + stat.getNumChildren());
-    }
-
     /**
      * A storage class for both command line options and shell commands.
      *
@@ -254,7 +239,9 @@ public class ZooKeeperMain {
     }
 
     public static List<String> getCommands() {
-        return new LinkedList<String>(commandMap.keySet());
+        List<String> cmdList = new ArrayList<String>(commandMap.keySet());
+        Collections.sort(cmdList);
+        return cmdList;
     }
 
     protected String getPrompt() {       
@@ -367,17 +354,6 @@ public class ZooKeeperMain {
       }
     }
 
-    private static DataCallback dataCallback = new DataCallback() {
-
-        public void processResult(int rc, String path, Object ctx, byte[] data,
-                Stat stat) {
-            System.out.println("rc = " + rc + " path = " + path + " data = "
-                    + (data == null ? "null" : new String(data)) + " stat = ");
-            printStat(stat);
-        }
-
-    };
-
     /**
      * trim the quota tree to recover unwanted tree elements
      * in the quota's tree
@@ -619,7 +595,6 @@ public class ZooKeeperMain {
     protected boolean processZKCmd(MyCommandOptions co)
         throws KeeperException, IOException, InterruptedException
     {
-        Stat stat = new Stat();
         String[] args = co.getArgArray();
         String cmd = co.getCommand();
         if (args.length < 1) {
@@ -632,13 +607,11 @@ public class ZooKeeperMain {
             return false;
         }
         
-        boolean watch = args.length > 2;
-        String path = null;
-        List<ACL> acl = Ids.OPEN_ACL_UNSAFE;
+        boolean watch = false;
         LOG.debug("Processing " + cmd);
         
+        try {
         if (cmd.equals("quit")) {
-            System.out.println("Quitting...");
             zk.close();
             System.exit(0);
         } else if (cmd.equals("redo") && args.length >= 2) {
@@ -679,205 +652,19 @@ public class ZooKeeperMain {
             return false;
         }
         
-        if (cmd.equals("create") && args.length >= 2) {
-            int first = 0;
-            CreateMode flags = CreateMode.PERSISTENT;
-            if ((args.length >= 3)
-                    && ((args[1].equals("-e") && args[2].equals("-s")) || (args[1])
-                            .equals("-s")
-                            && (args[2].equals("-e")))) {
-                first+=2;
-                flags = CreateMode.EPHEMERAL_SEQUENTIAL;
-            } else if (args[1].equals("-e")) {
-                first++;
-                flags = CreateMode.EPHEMERAL;
-            } else if (args[1].equals("-s")) {
-                first++;
-                flags = CreateMode.PERSISTENT_SEQUENTIAL;
-            }
-            if (args.length >= first+2) {
-                path = args[first+1];
-            }
-            if(path==null)
-            {
-                usage();
-                return false;
-            }
-            byte[] data = null;
-            if (args.length >= first + 3) {
-                data = args[first + 2].getBytes();
-            }
-            if (args.length >= first + 4) {
-                acl = parseACLs(args[first+3]);
-            }
-            String newPath = zk.create(path, data, acl, flags);
-            System.err.println("Created " + newPath);
-            return true;
-        } else if (cmd.equals("delete") && args.length >= 2) {
-            path = args[1];
-            zk.delete(path, watch ? Integer.parseInt(args[2]) : -1);
-        } else if (
-            (cmd.equals("deleteall") || cmd.equals("rmr")) &&
-            args.length >= 2) {
-            if (cmd.equals("rmr")) {
-              LOG.warn("The command 'rmr' has been deprecated. " +
-                  "Please use 'deleteall' instead.");
-            }
-            path = args[1];
-            ZKUtil.deleteRecursive(zk, path);
-        } else if (cmd.equals("set") && args.length >= 3) {
-            path = args[1];
-            stat = zk.setData(path, args[2].getBytes(),
-                    args.length > 3 ? Integer.parseInt(args[3]) : -1);
-            printStat(stat);
-        } else if (cmd.equals("aget") && args.length >= 2) {
-            path = args[1];
-            zk.getData(path, watch, dataCallback, path);
-        } else if (cmd.equals("get") && args.length >= 2) {
-            path = args[1];
-            byte data[] = zk.getData(path, watch, stat);
-            data = (data == null)? "null".getBytes() : data;
-            System.out.println(new String(data));
-            printStat(stat);
-        } else if (cmd.equals("ls") && args.length >= 2) {
-            path = args[1];
-            List<String> children = zk.getChildren(path, watch);
-            System.out.println(children);
-        } else if (cmd.equals("ls2") && args.length >= 2) {
-            path = args[1];
-            List<String> children = zk.getChildren(path, watch, stat);
-            System.out.println(children);
-            printStat(stat);
-        } else if (cmd.equals("getAcl") && args.length >= 2) {
-            path = args[1];
-            acl = zk.getACL(path, stat);
-            for (ACL a : acl) {
-                System.out.println(a.getId() + ": "
-                        + getPermString(a.getPerms()));
-            }
-        } else if (cmd.equals("setAcl") && args.length >= 3) {
-            path = args[1];
-            stat = zk.setACL(path, parseACLs(args[2]),
-                    args.length > 4 ? Integer.parseInt(args[3]) : -1);
-            printStat(stat);
-        } else if (cmd.equals("stat") && args.length >= 2) {
-            path = args[1];
-            stat = zk.exists(path, watch);
-            if (stat == null) {
-              throw new KeeperException.NoNodeException(path);	
-            }
-            printStat(stat);
-        } else if (cmd.equals("listquota") && args.length >= 2) {
-            path = args[1];
-            String absolutePath = Quotas.quotaZookeeper + path + "/" + Quotas.limitNode;
-            byte[] data =  null;
-            try {
-                System.err.println("absolute path is " + absolutePath);
-                data = zk.getData(absolutePath, false, stat);
-                StatsTrack st = new StatsTrack(new String(data));
-                System.out.println("Output quota for " + path + " "
-                        + st.toString());
-
-                data = zk.getData(Quotas.quotaZookeeper + path + "/" +
-                        Quotas.statNode, false, stat);
-                System.out.println("Output stat for " + path + " " +
-                        new StatsTrack(new String(data)).toString());
-            } catch(KeeperException.NoNodeException ne) {
-                System.err.println("quota for " + path + " does not exist.");
-            }
-        } else if (cmd.equals("setquota") && args.length >= 4) {
-            String option = args[1];
-            String val = args[2];
-            path = args[3];
-            System.err.println("Comment: the parts are " +
-                               "option " + option +
-                               " val " + val +
-                               " path " + path);
-            if ("-b".equals(option)) {
-                // we are setting the bytes quota
-                createQuota(zk, path, Long.parseLong(val), -1);
-            } else if ("-n".equals(option)) {
-                // we are setting the num quota
-                createQuota(zk, path, -1L, Integer.parseInt(val));
+        // execute from commandMap
+        CliCommand cliCmd = commandMapCli.get(cmd);
+        if(cliCmd != null) {
+            cliCmd.setZk(zk);
+            watch = cliCmd.parse(args).exec();
             } else {
                 usage();
             }
-
-        } else if (cmd.equals("delquota") && args.length >= 2) {
-            //if neither option -n or -b is specified, we delete
-            // the quota node for thsi node.
-            if (args.length == 3) {
-                //this time we have an option
-                String option = args[1];
-                path = args[2];
-                if ("-b".equals(option)) {
-                    delQuota(zk, path, true, false);
-                } else if ("-n".equals(option)) {
-                    delQuota(zk, path, false, true);
-                }
-            } else if (args.length == 2) {
-                path = args[1];
-                // we dont have an option specified.
-                // just delete whole quota node
-                delQuota(zk, path, true, true);
-            } else if (cmd.equals("help")) {
+        } catch (ParseException ex) {
+            System.err.println(ex.getMessage());
                 usage();
+            return false;
             }
-        } else if (cmd.equals("close")) {
-                zk.close();            
-        } else if (cmd.equals("sync") && args.length >= 2) {
-            path = args[1];
-            zk.sync(path, new AsyncCallback.VoidCallback() { public void processResult(int rc, String path, Object ctx) { System.out.println("Sync returned " + rc); } }, null );
-        } else if (cmd.equals("addauth") && args.length >=2 ) {
-            byte[] b = null;
-            if (args.length >= 3)
-                b = args[2].getBytes();
-
-            zk.addAuthInfo(args[1], b);
-        } else {
-            usage();
-        }
         return watch;
     }
-
-    private static String getPermString(int perms) {
-        StringBuilder p = new StringBuilder();
-        if ((perms & ZooDefs.Perms.CREATE) != 0) {
-            p.append('c');
-        }
-        if ((perms & ZooDefs.Perms.DELETE) != 0) {
-            p.append('d');
-        }
-        if ((perms & ZooDefs.Perms.READ) != 0) {
-            p.append('r');
-        }
-        if ((perms & ZooDefs.Perms.WRITE) != 0) {
-            p.append('w');
-        }
-        if ((perms & ZooDefs.Perms.ADMIN) != 0) {
-            p.append('a');
-        }
-        return p.toString();
-    }
-
-    private static List<ACL> parseACLs(String aclString) {
-        List<ACL> acl;
-        String acls[] = aclString.split(",");
-        acl = new ArrayList<ACL>();
-        for (String a : acls) {
-            int firstColon = a.indexOf(':');
-            int lastColon = a.lastIndexOf(':');
-            if (firstColon == -1 || lastColon == -1 || firstColon == lastColon) {
-                System.err
-                .println(a + " does not have the form scheme:id:perm");
-                continue;
-            }
-            ACL newAcl = new ACL();
-            newAcl.setId(new Id(a.substring(0, firstColon), a.substring(
-                    firstColon + 1, lastColon)));
-            newAcl.setPerms(getPermFromString(a.substring(lastColon + 1)));
-            acl.add(newAcl);
         }
-        return acl;
-    }
-}

Added: zookeeper/trunk/src/java/main/org/apache/zookeeper/cli/AclParser.java
URL: http://svn.apache.org/viewvc/zookeeper/trunk/src/java/main/org/apache/zookeeper/cli/AclParser.java?rev=1302068&view=auto
==============================================================================
--- zookeeper/trunk/src/java/main/org/apache/zookeeper/cli/AclParser.java (added)
+++ zookeeper/trunk/src/java/main/org/apache/zookeeper/cli/AclParser.java Sun Mar 18 06:03:05 2012
@@ -0,0 +1,81 @@
+/**
+ * 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.cli;
+
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.zookeeper.ZooDefs;
+import org.apache.zookeeper.data.ACL;
+import org.apache.zookeeper.data.Id;
+
+/**
+ * a parser for ACL strings
+ */
+public class AclParser {
+
+    /**
+     * parse string into list of ACL
+     * @param aclString
+     * @return 
+     */
+    public static List<ACL> parse(String aclString) {
+        List<ACL> acl;
+        String acls[] = aclString.split(",");
+        acl = new ArrayList<ACL>();
+        for (String a : acls) {
+            int firstColon = a.indexOf(':');
+            int lastColon = a.lastIndexOf(':');
+            if (firstColon == -1 || lastColon == -1 || firstColon == lastColon) {
+                System.err.println(a + " does not have the form scheme:id:perm");
+                continue;
+            }
+            ACL newAcl = new ACL();
+            newAcl.setId(new Id(a.substring(0, firstColon), a.substring(
+                    firstColon + 1, lastColon)));
+            newAcl.setPerms(getPermFromString(a.substring(lastColon + 1)));
+            acl.add(newAcl);
+        }
+        return acl;
+    }
+
+    static private int getPermFromString(String permString) {
+        int perm = 0;
+        for (int i = 0; i < permString.length(); i++) {
+            switch (permString.charAt(i)) {
+                case 'r':
+                    perm |= ZooDefs.Perms.READ;
+                    break;
+                case 'w':
+                    perm |= ZooDefs.Perms.WRITE;
+                    break;
+                case 'c':
+                    perm |= ZooDefs.Perms.CREATE;
+                    break;
+                case 'd':
+                    perm |= ZooDefs.Perms.DELETE;
+                    break;
+                case 'a':
+                    perm |= ZooDefs.Perms.ADMIN;
+                    break;
+                default:
+                    System.err.println("Unknown perm type: " + permString.charAt(i));
+            }
+        }
+        return perm;
+    }
+}

Added: zookeeper/trunk/src/java/main/org/apache/zookeeper/cli/AddAuthCommand.java
URL: http://svn.apache.org/viewvc/zookeeper/trunk/src/java/main/org/apache/zookeeper/cli/AddAuthCommand.java?rev=1302068&view=auto
==============================================================================
--- zookeeper/trunk/src/java/main/org/apache/zookeeper/cli/AddAuthCommand.java (added)
+++ zookeeper/trunk/src/java/main/org/apache/zookeeper/cli/AddAuthCommand.java Sun Mar 18 06:03:05 2012
@@ -0,0 +1,64 @@
+/**
+ * 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.cli;
+
+import java.io.IOException;
+import org.apache.commons.cli.CommandLine;
+import org.apache.commons.cli.Options;
+import org.apache.commons.cli.ParseException;
+import org.apache.commons.cli.Parser;
+import org.apache.commons.cli.PosixParser;
+import org.apache.zookeeper.KeeperException;
+
+/**
+ * addAuth command for cli
+ */
+public class AddAuthCommand extends CliCommand {
+
+    private static Options options = new Options();
+    private String[] args;
+
+    public AddAuthCommand() {
+        super("addauth", "scheme auth");
+    }
+
+    @Override
+    public CliCommand parse(String[] cmdArgs) throws ParseException {
+        Parser parser = new PosixParser();
+        CommandLine cl = parser.parse(options, cmdArgs);
+        args = cl.getArgs();
+        if (args.length < 2) {
+            throw new ParseException(getUsageStr());
+        }
+
+        return this;
+    }
+
+    @Override
+    public boolean exec() throws KeeperException, IOException, 
+                                 InterruptedException {
+        byte[] b = null;
+        if (args.length >= 3) {
+            b = args[2].getBytes();
+        }
+
+        zk.addAuthInfo(args[1], b);
+
+        return false;
+    }
+}

Added: zookeeper/trunk/src/java/main/org/apache/zookeeper/cli/CliCommand.java
URL: http://svn.apache.org/viewvc/zookeeper/trunk/src/java/main/org/apache/zookeeper/cli/CliCommand.java?rev=1302068&view=auto
==============================================================================
--- zookeeper/trunk/src/java/main/org/apache/zookeeper/cli/CliCommand.java (added)
+++ zookeeper/trunk/src/java/main/org/apache/zookeeper/cli/CliCommand.java Sun Mar 18 06:03:05 2012
@@ -0,0 +1,119 @@
+/**
+ * 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.cli;
+
+import java.io.IOException;
+import java.io.PrintStream;
+import java.util.Map;
+import org.apache.commons.cli.ParseException;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.ZooKeeper;
+
+/**
+ * base class for all CLI commands
+ */
+abstract public class CliCommand {
+
+    protected ZooKeeper zk;
+    protected PrintStream out;
+    protected PrintStream err;
+    private String cmdStr;
+    private String optionStr;
+
+    /**
+     * a CLI command with command string and options.
+     * Using System.out and System.err for printing
+     * @param cmdStr the string used to call this command
+     * @param optionStr the string used to call this command 
+     */
+    public CliCommand(String cmdStr, String optionStr) {
+        this(cmdStr, optionStr, System.out, System.err);
+    }
+
+    /**
+     * a CLI command with command string and options.
+     * @param cmdStr the string used to call this command
+     * @param optionStr the string used to call this command
+     * @param out stream used for all normal printing
+     * @param err stream used for all error printing
+     */
+    public CliCommand(String cmdStr, String optionStr, PrintStream out, PrintStream err) {
+        this.out = out;
+        this.err = err;
+        this.cmdStr = cmdStr;
+        this.optionStr = optionStr;
+    }
+
+    /**
+     * set the zookeper instance
+     * @param zk the zookeper instance
+     */
+    public void setZk(ZooKeeper zk) {
+        this.zk = zk;
+    }
+
+    /**
+     * get the string used to call this command
+     * @return 
+     */
+    public String getCmdStr() {
+        return cmdStr;
+    }
+
+    /**
+     * get the option string
+     * @return 
+     */
+    public String getOptionStr() {
+        return optionStr;
+    }
+
+    /**
+     * get a usage string, contains the command and the options
+     * @return 
+     */
+    public String getUsageStr() {
+        return cmdStr + " " + optionStr;
+    }
+
+    /**
+     * add this command to a map. Use the command string as key.
+     * @param cmdMap 
+     */
+    public void addToMap(Map<String, CliCommand> cmdMap) {
+        cmdMap.put(cmdStr, this);
+    }
+    
+    /**
+     * parse the command arguments
+     * @param cmdArgs
+     * @return this CliCommand
+     * @throws ParseException 
+     */
+    abstract public CliCommand parse(String cmdArgs[]) throws ParseException;
+    
+    /**
+     * 
+     * @return
+     * @throws KeeperException
+     * @throws IOException
+     * @throws InterruptedException 
+     */
+    abstract public boolean exec() throws KeeperException,
+            IOException, InterruptedException;
+}

Added: zookeeper/trunk/src/java/main/org/apache/zookeeper/cli/CloseCommand.java
URL: http://svn.apache.org/viewvc/zookeeper/trunk/src/java/main/org/apache/zookeeper/cli/CloseCommand.java?rev=1302068&view=auto
==============================================================================
--- zookeeper/trunk/src/java/main/org/apache/zookeeper/cli/CloseCommand.java (added)
+++ zookeeper/trunk/src/java/main/org/apache/zookeeper/cli/CloseCommand.java Sun Mar 18 06:03:05 2012
@@ -0,0 +1,46 @@
+/**
+ * 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.cli;
+
+import java.io.IOException;
+import org.apache.commons.cli.ParseException;
+import org.apache.zookeeper.KeeperException;
+
+/**
+ * close command for cli
+ */
+public class CloseCommand extends CliCommand {
+
+    public CloseCommand() {
+        super("close", "");
+    }
+    
+    
+    @Override
+    public CliCommand parse(String[] cmdArgs) throws ParseException {
+        return this;
+    }
+
+    @Override
+    public boolean exec() throws KeeperException, IOException, InterruptedException {
+        zk.close();
+        
+        return false;
+    }
+    
+}

Added: zookeeper/trunk/src/java/main/org/apache/zookeeper/cli/CreateCommand.java
URL: http://svn.apache.org/viewvc/zookeeper/trunk/src/java/main/org/apache/zookeeper/cli/CreateCommand.java?rev=1302068&view=auto
==============================================================================
--- zookeeper/trunk/src/java/main/org/apache/zookeeper/cli/CreateCommand.java (added)
+++ zookeeper/trunk/src/java/main/org/apache/zookeeper/cli/CreateCommand.java Sun Mar 18 06:03:05 2012
@@ -0,0 +1,81 @@
+/**
+ * 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.cli;
+
+import java.util.List;
+import org.apache.commons.cli.*;
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.ZooDefs;
+import org.apache.zookeeper.data.ACL;
+
+/**
+ * create command for cli
+ */
+public class CreateCommand extends CliCommand {
+
+    private static Options options = new Options();
+    private String[] args;
+    private CommandLine cl;
+    
+    {
+        options.addOption(new Option("e", false, "ephemeral"));
+        options.addOption(new Option("s", false, "sequential"));
+    }
+    
+    public CreateCommand() {
+        super("create", "[-s] [-e] path [data] [acl]");
+    }
+    
+
+    @Override
+    public CliCommand parse(String[] cmdArgs) throws ParseException {
+        Parser parser = new PosixParser();
+        cl = parser.parse(options, cmdArgs);
+        args = cl.getArgs();
+        if(args.length < 2) {
+            throw new ParseException(getUsageStr());
+        }
+        return this;
+    }
+
+    
+    @Override
+    public boolean exec() throws KeeperException, InterruptedException {
+        CreateMode flags = CreateMode.PERSISTENT;
+        if(cl.hasOption("e") && cl.hasOption("s")) {
+            flags = CreateMode.EPHEMERAL_SEQUENTIAL;
+        } else if (cl.hasOption("e")) {
+            flags = CreateMode.EPHEMERAL;
+        } else if (cl.hasOption("s")) {
+            flags = CreateMode.PERSISTENT_SEQUENTIAL;
+        }
+        String path = args[1];
+        byte[] data = null;
+        if (args.length > 2) {
+            data = args[2].getBytes();
+        }
+        List<ACL> acl = ZooDefs.Ids.OPEN_ACL_UNSAFE;
+        if (args.length > 3) {
+            acl = AclParser.parse(args[3]);
+        }
+        String newPath = zk.create(path, data, acl, flags);
+        err.println("Created " + newPath);
+        return true;
+    }
+}

Added: zookeeper/trunk/src/java/main/org/apache/zookeeper/cli/DelQuotaCommand.java
URL: http://svn.apache.org/viewvc/zookeeper/trunk/src/java/main/org/apache/zookeeper/cli/DelQuotaCommand.java?rev=1302068&view=auto
==============================================================================
--- zookeeper/trunk/src/java/main/org/apache/zookeeper/cli/DelQuotaCommand.java (added)
+++ zookeeper/trunk/src/java/main/org/apache/zookeeper/cli/DelQuotaCommand.java Sun Mar 18 06:03:05 2012
@@ -0,0 +1,152 @@
+/**
+ * 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.cli;
+
+import java.io.IOException;
+import java.util.List;
+import org.apache.commons.cli.*;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.Quotas;
+import org.apache.zookeeper.StatsTrack;
+import org.apache.zookeeper.ZooKeeper;
+import org.apache.zookeeper.data.Stat;
+
+/**
+ * delQuota command for cli
+ */
+public class DelQuotaCommand extends CliCommand {
+
+    private Options options = new Options();
+    private String[] args;
+    private CommandLine cl;
+
+    public DelQuotaCommand() {
+        super("delquota", "[-n|-b] path");
+
+        OptionGroup og1 = new OptionGroup();
+        og1.addOption(new Option("b", false, "bytes quota"));
+        og1.addOption(new Option("n", false, "num quota"));
+        options.addOptionGroup(og1);
+    }
+
+    @Override
+    public CliCommand parse(String[] cmdArgs) throws ParseException {
+        Parser parser = new PosixParser();
+        cl = parser.parse(options, cmdArgs);
+        args = cl.getArgs();
+        if (args.length < 2) {
+            throw new ParseException(getUsageStr());
+        }
+
+        return this;
+    }
+
+    @Override
+    public boolean exec() throws KeeperException, IOException, 
+                                 InterruptedException {
+        //if neither option -n or -b is specified, we delete
+        // the quota node for thsi node.
+        String path = args[1];
+        if (cl.hasOption("b")) {
+            delQuota(zk, path, true, false);
+        } else if (cl.hasOption("n")) {
+            delQuota(zk, path, false, true);
+        } else if (args.length == 2) {
+            // we dont have an option specified.
+            // just delete whole quota node
+            delQuota(zk, path, true, true);
+        }
+        return false;
+    }
+
+    /**
+     * this method deletes quota for a node.
+     *
+     * @param zk the zookeeper client
+     * @param path the path to delete quota for
+     * @param bytes true if number of bytes needs to be unset
+     * @param numNodes true if number of nodes needs to be unset
+     * @return true if quota deletion is successful
+     * @throws KeeperException
+     * @throws IOException
+     * @throws InterruptedException
+     */
+    public static boolean delQuota(ZooKeeper zk, String path,
+            boolean bytes, boolean numNodes)
+            throws KeeperException, IOException, InterruptedException {
+        String parentPath = Quotas.quotaZookeeper + path;
+        String quotaPath = Quotas.quotaZookeeper + path + "/" + 
+                Quotas.limitNode;
+        if (zk.exists(quotaPath, false) == null) {
+            System.out.println("Quota does not exist for " + path);
+            return true;
+        }
+        byte[] data = null;
+        try {
+            data = zk.getData(quotaPath, false, new Stat());
+        } catch (KeeperException.NoNodeException ne) {
+            System.err.println("quota does not exist for " + path);
+            return true;
+        }
+        StatsTrack strack = new StatsTrack(new String(data));
+        if (bytes && !numNodes) {
+            strack.setBytes(-1L);
+            zk.setData(quotaPath, strack.toString().getBytes(), -1);
+        } else if (!bytes && numNodes) {
+            strack.setCount(-1);
+            zk.setData(quotaPath, strack.toString().getBytes(), -1);
+        } else if (bytes && numNodes) {
+            // delete till you can find a node with more than
+            // one child
+            List<String> children = zk.getChildren(parentPath, false);
+            /// delete the direct children first
+            for (String child : children) {
+                zk.delete(parentPath + "/" + child, -1);
+            }
+            // cut the tree till their is more than one child
+            trimProcQuotas(zk, parentPath);
+        }
+        return true;
+    }
+
+    /**
+     * trim the quota tree to recover unwanted tree elements in the quota's tree
+     *
+     * @param zk the zookeeper client
+     * @param path the path to start from and go up and see if their is any
+     * unwanted parent in the path.
+     * @return true if sucessful
+     * @throws KeeperException
+     * @throws IOException
+     * @throws InterruptedException
+     */
+    private static boolean trimProcQuotas(ZooKeeper zk, String path)
+            throws KeeperException, IOException, InterruptedException {
+        if (Quotas.quotaZookeeper.equals(path)) {
+            return true;
+        }
+        List<String> children = zk.getChildren(path, false);
+        if (children.size() == 0) {
+            zk.delete(path, -1);
+            String parent = path.substring(0, path.lastIndexOf('/'));
+            return trimProcQuotas(zk, parent);
+        } else {
+            return true;
+        }
+    }
+}

Added: zookeeper/trunk/src/java/main/org/apache/zookeeper/cli/DeleteAllCommand.java
URL: http://svn.apache.org/viewvc/zookeeper/trunk/src/java/main/org/apache/zookeeper/cli/DeleteAllCommand.java?rev=1302068&view=auto
==============================================================================
--- zookeeper/trunk/src/java/main/org/apache/zookeeper/cli/DeleteAllCommand.java (added)
+++ zookeeper/trunk/src/java/main/org/apache/zookeeper/cli/DeleteAllCommand.java Sun Mar 18 06:03:05 2012
@@ -0,0 +1,72 @@
+/**
+ * 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.cli;
+
+import org.apache.commons.cli.CommandLine;
+import org.apache.commons.cli.Options;
+import org.apache.commons.cli.ParseException;
+import org.apache.commons.cli.Parser;
+import org.apache.commons.cli.PosixParser;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.ZKUtil;
+
+/**
+ * deleteAll command for cli
+ */
+public class DeleteAllCommand extends CliCommand {
+
+    private static Options options = new Options();
+    private String[] args;
+
+    public DeleteAllCommand() {
+        this("deleteall");
+    }
+
+    public DeleteAllCommand(String cmdStr) {
+        super(cmdStr, "path");
+    }
+    
+    @Override
+    public CliCommand parse(String[] cmdArgs) throws ParseException {
+        Parser parser = new PosixParser();
+        CommandLine cl = parser.parse(options, cmdArgs);
+        args = cl.getArgs();
+        if (args.length < 2) {
+            throw new ParseException(getUsageStr());
+        }
+
+        return this;
+    }
+
+    @Override
+    public boolean exec() throws KeeperException,
+            InterruptedException {
+        printDeprecatedWarning();
+        
+        String path = args[1];
+        ZKUtil.deleteRecursive(zk, path);
+        return false;
+    }
+    
+    private void printDeprecatedWarning() {
+        if("rmr".equals(args[0])) {
+            err.println("The command 'rmr' has been deprecated. " +
+                  "Please use 'deleteall' instead.");
+        }
+    }
+}

Added: zookeeper/trunk/src/java/main/org/apache/zookeeper/cli/DeleteCommand.java
URL: http://svn.apache.org/viewvc/zookeeper/trunk/src/java/main/org/apache/zookeeper/cli/DeleteCommand.java?rev=1302068&view=auto
==============================================================================
--- zookeeper/trunk/src/java/main/org/apache/zookeeper/cli/DeleteCommand.java (added)
+++ zookeeper/trunk/src/java/main/org/apache/zookeeper/cli/DeleteCommand.java Sun Mar 18 06:03:05 2012
@@ -0,0 +1,88 @@
+/**
+ * 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.cli;
+
+import org.apache.commons.cli.*;
+import org.apache.zookeeper.KeeperException;
+
+/**
+ * delete command for cli
+ */
+public class DeleteCommand extends CliCommand {
+
+    private static Options options = new Options();
+    private String[] args;
+    private CommandLine cl;
+
+    {
+        options.addOption("v", true, "version");
+    }
+
+    public DeleteCommand() {
+        super("delete", "[-v version] path");
+    }
+
+    @Override
+    public CliCommand parse(String[] cmdArgs) throws ParseException {
+        Parser parser = new PosixParser();
+        cl = parser.parse(options, cmdArgs);
+        args = cl.getArgs();
+        if (args.length < 2) {
+            throw new ParseException(getUsageStr());
+        }
+        
+        retainCompatibility(cmdArgs);
+
+        return this;
+    }
+
+    private void retainCompatibility(String[] cmdArgs) throws ParseException {
+        // delete path [version]
+        if (args.length > 2) {
+            // rewrite to option
+            String [] newCmd = new String[4];
+            newCmd[0] = cmdArgs [0];
+            newCmd[1] = "-v";
+            newCmd[2] = cmdArgs[2]; // version
+            newCmd[3] = cmdArgs[1]; // path            
+            err.println("'delete path [version]' has been deprecated. "
+                    + "Please use 'delete [-v version] path' instead.");
+            Parser parser = new PosixParser();
+            cl = parser.parse(options, newCmd);
+            args = cl.getArgs();
+        }
+    }
+
+    @Override
+    public boolean exec() throws KeeperException, InterruptedException {
+        String path = args[1];
+        int version;
+        if (cl.hasOption("v")) {
+            version = Integer.parseInt(cl.getOptionValue("v"));
+        } else {
+            version = -1;
+        }
+        
+        try {
+        zk.delete(path, version);
+        } catch(KeeperException.BadVersionException ex) {
+            err.println(ex.getMessage());
+        }
+        return false;
+    }
+}

Added: zookeeper/trunk/src/java/main/org/apache/zookeeper/cli/GetAclCommand.java
URL: http://svn.apache.org/viewvc/zookeeper/trunk/src/java/main/org/apache/zookeeper/cli/GetAclCommand.java?rev=1302068&view=auto
==============================================================================
--- zookeeper/trunk/src/java/main/org/apache/zookeeper/cli/GetAclCommand.java (added)
+++ zookeeper/trunk/src/java/main/org/apache/zookeeper/cli/GetAclCommand.java Sun Mar 18 06:03:05 2012
@@ -0,0 +1,94 @@
+/**
+ * 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.cli;
+
+import java.util.List;
+import org.apache.commons.cli.CommandLine;
+import org.apache.commons.cli.Options;
+import org.apache.commons.cli.ParseException;
+import org.apache.commons.cli.Parser;
+import org.apache.commons.cli.PosixParser;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.ZooDefs;
+import org.apache.zookeeper.data.ACL;
+import org.apache.zookeeper.data.Stat;
+
+/**
+ * getAcl command for cli
+ */
+public class GetAclCommand extends CliCommand {
+
+    private static Options options = new Options();
+    private String args[];
+    private CommandLine cl;
+
+    {
+        options.addOption("s", false, "stats");
+    }
+
+    public GetAclCommand() {
+        super("getAcl", "[-s] path");
+    }
+
+    @Override
+    public CliCommand parse(String[] cmdArgs) throws ParseException {
+        Parser parser = new PosixParser();
+        cl = parser.parse(options, cmdArgs);
+        args = cl.getArgs();
+        if (args.length < 2) {
+            throw new ParseException(getUsageStr());
+        }
+
+        return this;
+    }
+
+    @Override
+    public boolean exec() throws KeeperException, InterruptedException {
+        String path = args[1];
+        Stat stat = new Stat();
+        List<ACL> acl = zk.getACL(path, stat);
+        for (ACL a : acl) {
+            out.println(a.getId() + ": "
+                        + getPermString(a.getPerms()));
+        }
+
+        if (cl.hasOption("s")) {
+            new StatPrinter(out).print(stat);
+        }
+        return false;
+    }
+
+    private static String getPermString(int perms) {
+        StringBuilder p = new StringBuilder();
+        if ((perms & ZooDefs.Perms.CREATE) != 0) {
+            p.append('c');
+        }
+        if ((perms & ZooDefs.Perms.DELETE) != 0) {
+            p.append('d');
+        }
+        if ((perms & ZooDefs.Perms.READ) != 0) {
+            p.append('r');
+        }
+        if ((perms & ZooDefs.Perms.WRITE) != 0) {
+            p.append('w');
+        }
+        if ((perms & ZooDefs.Perms.ADMIN) != 0) {
+            p.append('a');
+        }
+        return p.toString();
+    }
+}

Added: zookeeper/trunk/src/java/main/org/apache/zookeeper/cli/GetCommand.java
URL: http://svn.apache.org/viewvc/zookeeper/trunk/src/java/main/org/apache/zookeeper/cli/GetCommand.java?rev=1302068&view=auto
==============================================================================
--- zookeeper/trunk/src/java/main/org/apache/zookeeper/cli/GetCommand.java (added)
+++ zookeeper/trunk/src/java/main/org/apache/zookeeper/cli/GetCommand.java Sun Mar 18 06:03:05 2012
@@ -0,0 +1,83 @@
+/**
+ * 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.cli;
+
+import org.apache.commons.cli.*;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.data.Stat;
+
+/**
+ * get command for cli
+ */
+public class GetCommand extends CliCommand {
+
+    private static Options options = new Options();
+    private String args[];
+    private CommandLine cl;
+
+    {
+        options.addOption("s", false, "stats");
+        options.addOption("w", false, "watch");
+    }
+
+    public GetCommand() {
+        super("get", "[-s] [-w] path");
+    }
+
+    @Override
+    public CliCommand parse(String[] cmdArgs) throws ParseException {
+
+        Parser parser = new PosixParser();
+        cl = parser.parse(options, cmdArgs);
+        args = cl.getArgs();
+        if (args.length < 2) {
+            throw new ParseException(getUsageStr());
+        }
+
+        retainCompatibility(cmdArgs);
+
+        return this;
+    }
+
+    private void retainCompatibility(String[] cmdArgs) throws ParseException {
+        // get path [watch]
+        if (args.length > 2) {
+            // rewrite to option
+            cmdArgs[2] = "-w";
+            err.println("'get path [watch]' has been deprecated. "
+                    + "Please use 'get [-s] [-w] path' instead.");
+            Parser parser = new PosixParser();
+            cl = parser.parse(options, cmdArgs);
+            args = cl.getArgs();
+        }
+    }
+
+    @Override
+    public boolean exec() throws KeeperException, InterruptedException {
+        boolean watch = cl.hasOption("w");
+        String path = args[1];
+        Stat stat = new Stat();
+        byte data[] = zk.getData(path, watch, stat);
+        data = (data == null) ? "null".getBytes() : data;
+        out.println(new String(data));
+        if (cl.hasOption("s")) {
+            new StatPrinter(out).print(stat);
+        }
+        return watch;
+    }
+}

Added: zookeeper/trunk/src/java/main/org/apache/zookeeper/cli/ListQuotaCommand.java
URL: http://svn.apache.org/viewvc/zookeeper/trunk/src/java/main/org/apache/zookeeper/cli/ListQuotaCommand.java?rev=1302068&view=auto
==============================================================================
--- zookeeper/trunk/src/java/main/org/apache/zookeeper/cli/ListQuotaCommand.java (added)
+++ zookeeper/trunk/src/java/main/org/apache/zookeeper/cli/ListQuotaCommand.java Sun Mar 18 06:03:05 2012
@@ -0,0 +1,74 @@
+/**
+ * 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.cli;
+
+import org.apache.commons.cli.*;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.Quotas;
+import org.apache.zookeeper.StatsTrack;
+import org.apache.zookeeper.data.Stat;
+
+/**
+ * listQuta command for cli
+ */
+public class ListQuotaCommand extends CliCommand {
+
+    private static Options options = new Options();
+    private String[] args;
+    
+    public ListQuotaCommand() {
+        super("listquota", "path");
+    }
+
+    @Override
+    public CliCommand parse(String[] cmdArgs) throws ParseException {
+        Parser parser = new PosixParser();
+        CommandLine cl = parser.parse(options, cmdArgs);
+        args = cl.getArgs();
+        if(args.length < 2) {
+            throw new ParseException(getUsageStr());
+        }    
+        
+        return this;
+    }
+    
+    @Override
+    public boolean exec() throws KeeperException,
+            InterruptedException {
+        String path = args[1];
+        String absolutePath = Quotas.quotaZookeeper + path + "/"
+                + Quotas.limitNode;
+        try {
+            err.println("absolute path is " + absolutePath);
+            Stat stat = new Stat();
+            byte[] data = zk.getData(absolutePath, false, stat);
+            StatsTrack st = new StatsTrack(new String(data));
+            out.println("Output quota for " + path + " "
+                    + st.toString());
+
+            data = zk.getData(Quotas.quotaZookeeper + path + "/"
+                    + Quotas.statNode, false, stat);
+            out.println("Output stat for " + path + " "
+                    + new StatsTrack(new String(data)).toString());
+        } catch (KeeperException.NoNodeException ne) {
+            err.println("quota for " + path + " does not exist.");
+        }
+        
+        return false;
+    }
+}

Added: zookeeper/trunk/src/java/main/org/apache/zookeeper/cli/Ls2Command.java
URL: http://svn.apache.org/viewvc/zookeeper/trunk/src/java/main/org/apache/zookeeper/cli/Ls2Command.java?rev=1302068&view=auto
==============================================================================
--- zookeeper/trunk/src/java/main/org/apache/zookeeper/cli/Ls2Command.java (added)
+++ zookeeper/trunk/src/java/main/org/apache/zookeeper/cli/Ls2Command.java Sun Mar 18 06:03:05 2012
@@ -0,0 +1,62 @@
+/**
+ * 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.cli;
+
+import java.util.List;
+import org.apache.commons.cli.*;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.data.Stat;
+
+/**
+ * ls2 command for cli
+ */
+public class Ls2Command extends CliCommand {
+
+    private static Options options = new Options();
+   
+    private String args[];
+    
+    public Ls2Command() {
+        super("ls2", "path [watch]");
+    }
+    
+    @Override
+    public CliCommand parse(String[] cmdArgs) throws ParseException {
+        Parser parser = new PosixParser();
+        CommandLine cl = parser.parse(options, cmdArgs);
+        args = cl.getArgs();
+        if(args.length < 2) {
+            throw new ParseException(getUsageStr());
+        }    
+        
+        return this;
+    }
+
+    
+    
+    @Override
+    public boolean exec() throws KeeperException, InterruptedException {
+        String path = args[1];
+        boolean watch = args.length > 2;
+        Stat stat = new Stat();
+        List<String> children = zk.getChildren(path, watch, stat);
+        out.println(children);
+        new StatPrinter(out).print(stat);
+        return watch;
+    }
+}

Added: zookeeper/trunk/src/java/main/org/apache/zookeeper/cli/LsCommand.java
URL: http://svn.apache.org/viewvc/zookeeper/trunk/src/java/main/org/apache/zookeeper/cli/LsCommand.java?rev=1302068&view=auto
==============================================================================
--- zookeeper/trunk/src/java/main/org/apache/zookeeper/cli/LsCommand.java (added)
+++ zookeeper/trunk/src/java/main/org/apache/zookeeper/cli/LsCommand.java Sun Mar 18 06:03:05 2012
@@ -0,0 +1,85 @@
+/**
+ * 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.cli;
+
+import java.util.List;
+import org.apache.commons.cli.*;
+import org.apache.zookeeper.KeeperException;
+
+/**
+ * ls command for cli
+ */
+public class LsCommand extends CliCommand {
+
+    private static Options options = new Options();
+    private String args[];
+    private CommandLine cl;
+
+    {
+        options.addOption("?", false, "help");
+        options.addOption("w", false, "watch");
+    }
+
+    public LsCommand() {
+        super("ls", "[-w] path");
+    }
+
+    private void printHelp() {
+        HelpFormatter formatter = new HelpFormatter();
+        formatter.printHelp("ls [options] path", options);
+    }
+
+    @Override
+    public CliCommand parse(String[] cmdArgs) throws ParseException {
+        Parser parser = new PosixParser();
+        cl = parser.parse(options, cmdArgs);
+        args = cl.getArgs();
+        if (cl.hasOption("?")) {
+            printHelp();
+        }
+
+        if (args.length < 2) {
+            throw new ParseException(getUsageStr());
+        }
+
+        retainCompatibility(cmdArgs);
+        
+        return this;
+    }
+
+    private void retainCompatibility(String[] cmdArgs) throws ParseException {
+        // get path [watch]
+        if (args.length > 2) {
+            // rewrite to option
+            cmdArgs[2] = "-w";
+            err.println("'ls path [watch]' has been deprecated. "
+                    + "Please use 'ls [-w] path' instead.");
+            Parser parser = new PosixParser();
+            cl = parser.parse(options, cmdArgs);
+            args = cl.getArgs();
+        }
+    }
+
+    @Override
+    public boolean exec() throws KeeperException, InterruptedException {
+        String path = args[1];
+        boolean watch = cl.hasOption("w");
+        List<String> children = zk.getChildren(path, watch);
+        out.println(children);
+        return watch;
+    }
+}

Added: zookeeper/trunk/src/java/main/org/apache/zookeeper/cli/SetAclCommand.java
URL: http://svn.apache.org/viewvc/zookeeper/trunk/src/java/main/org/apache/zookeeper/cli/SetAclCommand.java?rev=1302068&view=auto
==============================================================================
--- zookeeper/trunk/src/java/main/org/apache/zookeeper/cli/SetAclCommand.java (added)
+++ zookeeper/trunk/src/java/main/org/apache/zookeeper/cli/SetAclCommand.java Sun Mar 18 06:03:05 2012
@@ -0,0 +1,82 @@
+/**
+ * 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.cli;
+
+import java.util.List;
+import org.apache.commons.cli.*;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.data.ACL;
+import org.apache.zookeeper.data.Stat;
+
+/**
+ * setAcl command for cli
+ */
+public class SetAclCommand extends CliCommand {
+
+    private static Options options = new Options();
+    private String[] args;
+    private CommandLine cl;
+
+    {
+        options.addOption("s", false, "stats");
+        options.addOption("v", true, "version");
+    }
+
+    public SetAclCommand() {
+        super("setAcl", "[-s] [-v version] path acl");
+    }
+
+    @Override
+    public CliCommand parse(String[] cmdArgs) throws ParseException {
+        Parser parser = new PosixParser();
+        cl = parser.parse(options, cmdArgs);
+        args = cl.getArgs();
+        if (args.length < 3) {
+            throw new ParseException(getUsageStr());
+        }
+
+        return this;
+    }
+
+    @Override
+    public boolean exec() throws KeeperException,
+            InterruptedException {
+        String path = args[1];
+        String aclStr = args[2];
+        List<ACL> acl = AclParser.parse(aclStr);
+        int version;
+        if (cl.hasOption("v")) {
+            version = Integer.parseInt(cl.getOptionValue("v"));
+        } else {
+            version = -1;
+        }
+        try {
+            Stat stat = zk.setACL(path, acl, version);
+            if (cl.hasOption("s")) {
+                new StatPrinter(out).print(stat);
+            }
+        } catch (KeeperException.InvalidACLException ex) {
+            err.println("Invalid ACL: " + aclStr);
+        } catch (KeeperException.NoAuthException ex) {
+            err.println(ex.getMessage());
+        }
+
+        return false;
+
+    }
+}

Added: zookeeper/trunk/src/java/main/org/apache/zookeeper/cli/SetCommand.java
URL: http://svn.apache.org/viewvc/zookeeper/trunk/src/java/main/org/apache/zookeeper/cli/SetCommand.java?rev=1302068&view=auto
==============================================================================
--- zookeeper/trunk/src/java/main/org/apache/zookeeper/cli/SetCommand.java (added)
+++ zookeeper/trunk/src/java/main/org/apache/zookeeper/cli/SetCommand.java Sun Mar 18 06:03:05 2012
@@ -0,0 +1,75 @@
+/**
+ * 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.cli;
+
+import org.apache.commons.cli.*;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.data.Stat;
+
+/**
+ * set command for cli
+ */
+public class SetCommand extends CliCommand {
+
+    private static Options options = new Options();
+    private String[] args;
+    private CommandLine cl;
+
+    {
+        options.addOption("s", false, "stats");
+        options.addOption("v", true, "version");
+    }
+
+    public SetCommand() {
+        super("set", "[-s] [-v version] path data");
+    }
+
+    @Override
+    public CliCommand parse(String[] cmdArgs) throws ParseException {
+        Parser parser = new PosixParser();
+        cl = parser.parse(options, cmdArgs);
+        args = cl.getArgs();
+        if (args.length < 3) {
+            throw new ParseException(getUsageStr());
+        }
+
+        return this;
+    }
+
+    @Override
+    public boolean exec() throws KeeperException, InterruptedException {
+        String path = args[1];
+        byte[] data = args[2].getBytes();
+        int version;
+        if (cl.hasOption("v")) {
+            version = Integer.parseInt(cl.getOptionValue("v"));
+        } else {
+            version = -1;
+        }
+
+        try {
+            Stat stat = zk.setData(path, data, version);
+            if (cl.hasOption("s")) {
+                new StatPrinter(out).print(stat);
+            }
+        } catch (KeeperException.BadVersionException ex) {
+            err.println(ex.getMessage());
+        }
+        return false;
+    }
+}

Added: zookeeper/trunk/src/java/main/org/apache/zookeeper/cli/SetQuotaCommand.java
URL: http://svn.apache.org/viewvc/zookeeper/trunk/src/java/main/org/apache/zookeeper/cli/SetQuotaCommand.java?rev=1302068&view=auto
==============================================================================
--- zookeeper/trunk/src/java/main/org/apache/zookeeper/cli/SetQuotaCommand.java (added)
+++ zookeeper/trunk/src/java/main/org/apache/zookeeper/cli/SetQuotaCommand.java Sun Mar 18 06:03:05 2012
@@ -0,0 +1,200 @@
+/**
+ * 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.cli;
+
+import java.io.IOException;
+import java.util.List;
+import org.apache.commons.cli.*;
+import org.apache.zookeeper.*;
+import org.apache.zookeeper.data.Stat;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * setQuota command for cli
+ */
+public class SetQuotaCommand extends CliCommand {
+
+    private static final Logger LOG = LoggerFactory.getLogger(SetQuotaCommand.class);
+    private Options options = new Options();
+    private String[] args;
+    private CommandLine cl;
+
+    public SetQuotaCommand() {
+        super("setquota", "-n|-b val path");
+        
+        OptionGroup og1 = new OptionGroup();
+        og1.addOption(new Option("b", true, "bytes quota"));
+        og1.addOption(new Option("n", true, "num quota"));
+        og1.setRequired(true);
+        options.addOptionGroup(og1);
+   }
+
+    @Override
+    public CliCommand parse(String[] cmdArgs) throws ParseException {
+        Parser parser = new PosixParser();
+        cl = parser.parse(options, cmdArgs);
+        args = cl.getArgs();
+        if (args.length < 2) {
+            throw new ParseException(getUsageStr());
+        }
+
+        return this;
+    }
+
+    @Override
+    public boolean exec() throws KeeperException, IOException,
+            InterruptedException {
+        // get the args
+        String path = args[1];
+
+        if (cl.hasOption("b")) {
+            // we are setting the bytes quota
+            long bytes = Long.parseLong(cl.getOptionValue("b"));
+            createQuota(zk, path, bytes, -1);
+        } else if (cl.hasOption("n")) {
+            // we are setting the num quota
+            int numNodes = Integer.parseInt(cl.getOptionValue("n"));
+            createQuota(zk, path, -1L, numNodes);
+        } else {
+            err.println(getUsageStr());
+        }
+
+        return false;
+    }
+
+    public static boolean createQuota(ZooKeeper zk, String path,
+            long bytes, int numNodes)
+            throws KeeperException, IOException, InterruptedException {
+        // check if the path exists. We cannot create
+        // quota for a path that already exists in zookeeper
+        // for now.
+        Stat initStat = zk.exists(path, false);
+        if (initStat == null) {
+            throw new IllegalArgumentException(path + " does not exist.");
+        }
+        // now check if their is already existing
+        // parent or child that has quota
+
+        String quotaPath = Quotas.quotaZookeeper;
+        // check for more than 2 children --
+        // if zookeeper_stats and zookeeper_qutoas
+        // are not the children then this path
+        // is an ancestor of some path that
+        // already has quota
+        String realPath = Quotas.quotaZookeeper + path;
+        try {
+            List<String> children = zk.getChildren(realPath, false);
+            for (String child : children) {
+                if (!child.startsWith("zookeeper_")) {
+                    throw new IllegalArgumentException(path + " has child "
+                            + child + " which has a quota");
+                }
+            }
+        } catch (KeeperException.NoNodeException ne) {
+            // this is fine
+        }
+
+        //check for any parent that has been quota
+        checkIfParentQuota(zk, path);
+
+        // this is valid node for quota
+        // start creating all the parents
+        if (zk.exists(quotaPath, false) == null) {
+            try {
+                zk.create(Quotas.procZookeeper, null, ZooDefs.Ids.OPEN_ACL_UNSAFE,
+                        CreateMode.PERSISTENT);
+                zk.create(Quotas.quotaZookeeper, null, ZooDefs.Ids.OPEN_ACL_UNSAFE,
+                        CreateMode.PERSISTENT);
+            } catch (KeeperException.NodeExistsException ne) {
+                // do nothing
+            }
+        }
+
+        // now create the direct children
+        // and the stat and quota nodes
+        String[] splits = path.split("/");
+        StringBuilder sb = new StringBuilder();
+        sb.append(quotaPath);
+        for (int i = 1; i < splits.length; i++) {
+            sb.append("/" + splits[i]);
+            quotaPath = sb.toString();
+            try {
+                zk.create(quotaPath, null, ZooDefs.Ids.OPEN_ACL_UNSAFE,
+                        CreateMode.PERSISTENT);
+            } catch (KeeperException.NodeExistsException ne) {
+                //do nothing
+            }
+        }
+        String statPath = quotaPath + "/" + Quotas.statNode;
+        quotaPath = quotaPath + "/" + Quotas.limitNode;
+        StatsTrack strack = new StatsTrack(null);
+        strack.setBytes(bytes);
+        strack.setCount(numNodes);
+        try {
+            zk.create(quotaPath, strack.toString().getBytes(),
+                    ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
+            StatsTrack stats = new StatsTrack(null);
+            stats.setBytes(0L);
+            stats.setCount(0);
+            zk.create(statPath, stats.toString().getBytes(),
+                    ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
+        } catch (KeeperException.NodeExistsException ne) {
+            byte[] data = zk.getData(quotaPath, false, new Stat());
+            StatsTrack strackC = new StatsTrack(new String(data));
+            if (bytes != -1L) {
+                strackC.setBytes(bytes);
+            }
+            if (numNodes != -1) {
+                strackC.setCount(numNodes);
+            }
+            zk.setData(quotaPath, strackC.toString().getBytes(), -1);
+        }
+        return true;
+    }
+
+    private static void checkIfParentQuota(ZooKeeper zk, String path)
+            throws InterruptedException, KeeperException {
+        final String[] splits = path.split("/");
+        String quotaPath = Quotas.quotaZookeeper;
+        for (String str : splits) {
+            if (str.length() == 0) {
+                // this should only be for the beginning of the path
+                // i.e. "/..." - split(path)[0] is empty string before first '/'
+                continue;
+            }
+            quotaPath += "/" + str;
+            List<String> children = null;
+            try {
+                children = zk.getChildren(quotaPath, false);
+            } catch (KeeperException.NoNodeException ne) {
+                LOG.debug("child removed during quota check", ne);
+                return;
+            }
+            if (children.size() == 0) {
+                return;
+            }
+            for (String child : children) {
+                if (Quotas.limitNode.equals(child)) {
+                    throw new IllegalArgumentException(path + " has a parent "
+                            + quotaPath + " which has a quota");
+                }
+            }
+        }
+    }
+}

Added: zookeeper/trunk/src/java/main/org/apache/zookeeper/cli/StatCommand.java
URL: http://svn.apache.org/viewvc/zookeeper/trunk/src/java/main/org/apache/zookeeper/cli/StatCommand.java?rev=1302068&view=auto
==============================================================================
--- zookeeper/trunk/src/java/main/org/apache/zookeeper/cli/StatCommand.java (added)
+++ zookeeper/trunk/src/java/main/org/apache/zookeeper/cli/StatCommand.java Sun Mar 18 06:03:05 2012
@@ -0,0 +1,81 @@
+/**
+ * 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.cli;
+
+import org.apache.commons.cli.*;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.data.Stat;
+
+/**
+ * stat command for cli
+ */
+public class StatCommand extends CliCommand {
+
+    private static final Options options = new Options();
+    private String[] args;
+    private CommandLine cl;
+
+    static {
+        options.addOption("w", false, "watch");
+    }
+    
+    public StatCommand() {
+        super("stat", "[-w] path");
+    }
+
+    
+    @Override
+    public CliCommand parse(String[] cmdArgs) throws ParseException {
+        Parser parser = new PosixParser();
+        cl = parser.parse(options, cmdArgs);
+        args = cl.getArgs();
+        if(args.length < 2) {
+            throw new ParseException(getUsageStr());
+        }    
+        
+        retainCompatibility(cmdArgs);
+
+        return this;
+    }
+
+    private void retainCompatibility(String[] cmdArgs) throws ParseException {
+        // stat path [watch]
+        if (args.length > 2) {
+            // rewrite to option
+            cmdArgs[2] = "-w";
+            err.println("'stat path [watch]' has been deprecated. "
+                    + "Please use 'stat [-w] path' instead.");
+            Parser parser = new PosixParser();
+            cl = parser.parse(options, cmdArgs);
+            args = cl.getArgs();
+        }
+    }
+    
+    @Override
+    public boolean exec() throws KeeperException,
+            InterruptedException {
+        String path = args[1];
+        boolean watch = cl.hasOption("w");
+        Stat stat = zk.exists(path, watch);
+        if (stat == null) {
+            throw new KeeperException.NoNodeException(path);
+        }
+        new StatPrinter(out).print(stat);
+        return watch;
+    }
+}

Added: zookeeper/trunk/src/java/main/org/apache/zookeeper/cli/StatPrinter.java
URL: http://svn.apache.org/viewvc/zookeeper/trunk/src/java/main/org/apache/zookeeper/cli/StatPrinter.java?rev=1302068&view=auto
==============================================================================
--- zookeeper/trunk/src/java/main/org/apache/zookeeper/cli/StatPrinter.java (added)
+++ zookeeper/trunk/src/java/main/org/apache/zookeeper/cli/StatPrinter.java Sun Mar 18 06:03:05 2012
@@ -0,0 +1,49 @@
+/**
+ * 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.cli;
+
+import java.io.PrintStream;
+import java.util.Date;
+import org.apache.zookeeper.data.Stat;
+
+/**
+ * utility for printing stat values s
+ */
+public class StatPrinter {
+
+    protected PrintStream out;
+
+    public StatPrinter(PrintStream out) {
+        this.out = out;
+    }
+
+    public void print(Stat stat) {
+        out.println("cZxid = 0x" + Long.toHexString(stat.getCzxid()));
+        out.println("ctime = " + new Date(stat.getCtime()).toString());
+        out.println("mZxid = 0x" + Long.toHexString(stat.getMzxid()));
+        out.println("mtime = " + new Date(stat.getMtime()).toString());
+        out.println("pZxid = 0x" + Long.toHexString(stat.getPzxid()));
+        out.println("cversion = " + stat.getCversion());
+        out.println("dataVersion = " + stat.getVersion());
+        out.println("aclVersion = " + stat.getAversion());
+        out.println("ephemeralOwner = 0x"
+                + Long.toHexString(stat.getEphemeralOwner()));
+        out.println("dataLength = " + stat.getDataLength());
+        out.println("numChildren = " + stat.getNumChildren());
+    }
+}

Added: zookeeper/trunk/src/java/main/org/apache/zookeeper/cli/SyncCommand.java
URL: http://svn.apache.org/viewvc/zookeeper/trunk/src/java/main/org/apache/zookeeper/cli/SyncCommand.java?rev=1302068&view=auto
==============================================================================
--- zookeeper/trunk/src/java/main/org/apache/zookeeper/cli/SyncCommand.java (added)
+++ zookeeper/trunk/src/java/main/org/apache/zookeeper/cli/SyncCommand.java Sun Mar 18 06:03:05 2012
@@ -0,0 +1,64 @@
+/**
+ * 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.cli;
+
+import java.io.IOException;
+import org.apache.commons.cli.CommandLine;
+import org.apache.commons.cli.Options;
+import org.apache.commons.cli.ParseException;
+import org.apache.commons.cli.Parser;
+import org.apache.commons.cli.PosixParser;
+import org.apache.zookeeper.AsyncCallback;
+import org.apache.zookeeper.KeeperException;
+
+/**
+ * sync command for cli
+ */
+public class SyncCommand extends CliCommand {
+
+    private static Options options = new Options();
+    private String[] args;
+
+    public SyncCommand() {
+        super("sync", "path");
+    }
+
+    @Override
+    public CliCommand parse(String[] cmdArgs) throws ParseException {
+        Parser parser = new PosixParser();
+        CommandLine cl = parser.parse(options, cmdArgs);
+        args = cl.getArgs();
+        if (args.length < 2) {
+            throw new ParseException(getUsageStr());
+        }
+
+        return this;
+    }
+
+    @Override
+    public boolean exec() throws KeeperException, IOException, InterruptedException {
+        String path = args[1];
+        zk.sync(path, new AsyncCallback.VoidCallback() {
+
+            public void processResult(int rc, String path, Object ctx) {
+                out.println("Sync returned " + rc);
+            }
+        }, null);
+
+        return false;
+    }
+}

Modified: zookeeper/trunk/src/java/test/org/apache/zookeeper/ZooKeeperTest.java
URL: http://svn.apache.org/viewvc/zookeeper/trunk/src/java/test/org/apache/zookeeper/ZooKeeperTest.java?rev=1302068&r1=1302067&r2=1302068&view=diff
==============================================================================
--- zookeeper/trunk/src/java/test/org/apache/zookeeper/ZooKeeperTest.java (original)
+++ zookeeper/trunk/src/java/test/org/apache/zookeeper/ZooKeeperTest.java Sun Mar 18 06:03:05 2012
@@ -310,7 +310,7 @@ public class ZooKeeperTest extends Clien
         stat = zk.exists("/node4", true);
         version = stat.getVersion();
         zkMain.cl.parseCommand(cmdstring2);
-        Assert.assertTrue(zkMain.processZKCmd(zkMain.cl));
+        Assert.assertFalse(zkMain.processZKCmd(zkMain.cl));
         stat = zk.exists("/node4", true);
         Assert.assertEquals(version + 1, stat.getVersion());
         zkMain.cl.parseCommand(cmdstring3);