Author: mahadev
Date: Thu May 7 22:34:36 2009
New Revision: 772797
URL: http://svn.apache.org/viewvc?rev=772797&view=rev
Log:
ZOOKEEPER-386. improve java cli shell. (henry robinson via mahadev)
Added:
hadoop/zookeeper/trunk/src/java/lib/README.txt
hadoop/zookeeper/trunk/src/java/lib/jline-0.9.94.LICENSE.txt
hadoop/zookeeper/trunk/src/java/lib/jline-0.9.94.jar (with props)
hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/JLineZNodeCompletor.java
Modified:
hadoop/zookeeper/trunk/CHANGES.txt
hadoop/zookeeper/trunk/build.xml
hadoop/zookeeper/trunk/src/docs/src/documentation/content/xdocs/zookeeperAdmin.xml
hadoop/zookeeper/trunk/src/docs/src/documentation/content/xdocs/zookeeperQuotas.xml
hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/ZooKeeperMain.java
Modified: hadoop/zookeeper/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/CHANGES.txt?rev=772797&r1=772796&r2=772797&view=diff
==============================================================================
--- hadoop/zookeeper/trunk/CHANGES.txt (original)
+++ hadoop/zookeeper/trunk/CHANGES.txt Thu May 7 22:34:36 2009
@@ -131,6 +131,8 @@
ZOOKEEPER-378. perl binding for zookeeper (chris darroch via mahadev)
+ ZOOKEEPER-386. improve java cli shell. (henry robinson via mahadev)
+
Release 3.1.0 - 2009-02-06
Non-backward compatible changes:
Modified: hadoop/zookeeper/trunk/build.xml
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/build.xml?rev=772797&r1=772796&r2=772797&view=diff
==============================================================================
--- hadoop/zookeeper/trunk/build.xml (original)
+++ hadoop/zookeeper/trunk/build.xml Thu May 7 22:34:36 2009
@@ -169,6 +169,7 @@
<fileset dir="${build.classes}" />
<pathelement path="${lib.dir}/log4j-1.2.15.jar" />
<pathelement path="${clover.jar}" />
+ <pathelement path="${lib.dir}/jline-0.9.94.jar" />
</path>
</target>
Modified: hadoop/zookeeper/trunk/src/docs/src/documentation/content/xdocs/zookeeperAdmin.xml
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/docs/src/documentation/content/xdocs/zookeeperAdmin.xml?rev=772797&r1=772796&r2=772797&view=diff
==============================================================================
--- hadoop/zookeeper/trunk/src/docs/src/documentation/content/xdocs/zookeeperAdmin.xml (original)
+++ hadoop/zookeeper/trunk/src/docs/src/documentation/content/xdocs/zookeeperAdmin.xml Thu
May 7 22:34:36 2009
@@ -222,8 +222,8 @@
<para>In Java, you can run the following command to execute
simple operations:</para>
- <para><computeroutput>$ java -cp zookeeper.jar:src/java/lib/log4j-1.2.15.jar:conf
\
- org.apache.zookeeper.ZooKeeperMain 127.0.0.1:2181</computeroutput></para>
+ <para><computeroutput>$ java -cp zookeeper.jar:src/java/lib/log4j-1.2.15.jar:conf:src/java/lib/jline-0.9.94.jar
\
+ org.apache.zookeeper.ZooKeeperMain -server 127.0.0.1:2181</computeroutput></para>
</listitem>
<listitem>
Modified: hadoop/zookeeper/trunk/src/docs/src/documentation/content/xdocs/zookeeperQuotas.xml
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/docs/src/documentation/content/xdocs/zookeeperQuotas.xml?rev=772797&r1=772796&r2=772797&view=diff
==============================================================================
--- hadoop/zookeeper/trunk/src/docs/src/documentation/content/xdocs/zookeeperQuotas.xml (original)
+++ hadoop/zookeeper/trunk/src/docs/src/documentation/content/xdocs/zookeeperQuotas.xml Thu
May 7 22:34:36 2009
@@ -43,8 +43,8 @@
ZooKeeper prints <emphasis>WARN</emphasis> messages if users exceed the quota
assigned to them. The messages
are printed in the log of the ZooKeeper.
</para>
- <para><computeroutput>$java -cp zookeeper.jar:src/java/lib/log4j-1.2.15.jar/conf
\
- org.apache.zookeeper.ZooKeeperMain host:port</computeroutput></para>
+ <para><computeroutput>$java -cp zookeeper.jar:src/java/lib/log4j-1.2.15.jar/conf:src/java/lib/jline-0.9.94.jar
\
+ org.apache.zookeeper.ZooKeeperMain -server host:port</computeroutput></para>
<para> The above command gives you a command line option of using quotas.</para>
<section>
<title>Setting Quotas</title>
Added: hadoop/zookeeper/trunk/src/java/lib/README.txt
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/java/lib/README.txt?rev=772797&view=auto
==============================================================================
--- hadoop/zookeeper/trunk/src/java/lib/README.txt (added)
+++ hadoop/zookeeper/trunk/src/java/lib/README.txt Thu May 7 22:34:36 2009
@@ -0,0 +1,4 @@
+Folowing is the list of external jars contained in this directory and the sources from where
they were obtained:
+---------------------------------------------------------------------------------------------------------------
+
+* jline-0.9.94.jar - http://jline.sourceforge.net/
Added: hadoop/zookeeper/trunk/src/java/lib/jline-0.9.94.LICENSE.txt
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/java/lib/jline-0.9.94.LICENSE.txt?rev=772797&view=auto
==============================================================================
--- hadoop/zookeeper/trunk/src/java/lib/jline-0.9.94.LICENSE.txt (added)
+++ hadoop/zookeeper/trunk/src/java/lib/jline-0.9.94.LICENSE.txt Thu May 7 22:34:36 2009
@@ -0,0 +1,33 @@
+Copyright (c) 2002-2006, Marc Prud'hommeaux <mwp1@cornell.edu>
+All rights reserved.
+
+Redistribution and use in source and binary forms, with or
+without modification, are permitted provided that the following
+conditions are met:
+
+Redistributions of source code must retain the above copyright
+notice, this list of conditions and the following disclaimer.
+
+Redistributions in binary form must reproduce the above copyright
+notice, this list of conditions and the following disclaimer
+in the documentation and/or other materials provided with
+the distribution.
+
+Neither the name of JLine nor the names of its contributors
+may be used to endorse or promote products derived from this
+software without specific prior written permission.
+
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING,
+BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY
+AND FITNESS FOR A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO
+EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE LIABLE
+FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY,
+OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO,
+PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED
+AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT
+LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING
+IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED
+OF THE POSSIBILITY OF SUCH DAMAGE.
+
Added: hadoop/zookeeper/trunk/src/java/lib/jline-0.9.94.jar
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/java/lib/jline-0.9.94.jar?rev=772797&view=auto
==============================================================================
Binary file - no diff available.
Propchange: hadoop/zookeeper/trunk/src/java/lib/jline-0.9.94.jar
------------------------------------------------------------------------------
svn:executable = *
Propchange: hadoop/zookeeper/trunk/src/java/lib/jline-0.9.94.jar
------------------------------------------------------------------------------
svn:mime-type = application/octet-stream
Added: hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/JLineZNodeCompletor.java
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/JLineZNodeCompletor.java?rev=772797&view=auto
==============================================================================
--- hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/JLineZNodeCompletor.java (added)
+++ hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/JLineZNodeCompletor.java Thu
May 7 22:34:36 2009
@@ -0,0 +1,84 @@
+/**
+ * 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;
+
+import java.util.List;
+
+import jline.Completor;
+
+class JLineZNodeCompletor implements Completor {
+ private ZooKeeper zk;
+
+ public JLineZNodeCompletor(ZooKeeper zk) {
+ this.zk = zk;
+ }
+
+ @SuppressWarnings("unchecked")
+ public int complete(String buffer, int cursor, List candidates) {
+ // Guarantee that the final token is the one we're expanding
+ buffer = buffer.substring(0,cursor);
+ String token = "";
+ if (!buffer.endsWith(" ")) {
+ String[] tokens = buffer.split(" ");
+ if (tokens.length != 0) {
+ token = tokens[tokens.length-1] ;
+ }
+ }
+
+ if (token.startsWith("/")){
+ return completeZNode( buffer, token, candidates);
+ }
+ return completeCommand(buffer, token, candidates);
+ }
+
+ private int completeCommand(String buffer, String token,
+ List<String> candidates)
+ {
+ for (String cmd : ZooKeeperMain.getCommands()) {
+ if (cmd.startsWith( token )) {
+ candidates.add(cmd);
+ }
+ }
+ return buffer.lastIndexOf(" ")+1;
+ }
+
+ private int completeZNode( String buffer, String token,
+ List<String> candidates)
+ {
+ String path = token;
+ int idx = path.lastIndexOf("/") + 1;
+ String prefix = path.substring(idx);
+ try {
+ // Only the root path can end in a /, so strip it off every other prefix
+ String dir = idx == 1 ? "/" : path.substring(0,idx-1);
+ List<String> children = zk.getChildren(dir, false);
+ for (String child : children) {
+ if (child.startsWith(prefix)) {
+ candidates.add( child );
+ }
+ }
+ } catch( InterruptedException e) {
+ return 0;
+ }
+ catch( KeeperException e) {
+ return 0;
+ }
+ return candidates.size() == 0 ? buffer.length() : buffer.lastIndexOf("/") + 1;
+ }
+}
Modified: hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/ZooKeeperMain.java
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/ZooKeeperMain.java?rev=772797&r1=772796&r2=772797&view=diff
==============================================================================
--- hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/ZooKeeperMain.java (original)
+++ hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/ZooKeeperMain.java Thu May
7 22:34:36 2009
@@ -15,14 +15,23 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
+
package org.apache.zookeeper;
import java.io.BufferedReader;
import java.io.IOException;
import java.io.InputStreamReader;
+import java.lang.reflect.InvocationTargetException;
+import java.lang.reflect.Method;
import java.util.ArrayList;
+import java.util.Arrays;
import java.util.Date;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.LinkedList;
import java.util.List;
+import java.util.Map;
+import java.util.NoSuchElementException;
import org.apache.log4j.Logger;
import org.apache.zookeeper.AsyncCallback.DataCallback;
@@ -33,34 +42,54 @@
/**
* The command line client to ZooKeeper.
- *
+ *
*/
public class ZooKeeperMain {
- /**
- * the logger for this class
- */
private static final Logger LOG = Logger.getLogger(ZooKeeperMain.class);
-
+ protected static Map<String,String> commandMap = new HashMap<String,String>(
);
+
+ protected MyCommandOptions cl = new MyCommandOptions();
+ protected HashMap<Integer,String> history = new HashMap<Integer,String>(
);
+ protected int commandCount = 0;
+ protected boolean printWatches = true;
+
+ protected ZooKeeper zk;
+
+ public boolean getPrintWatches( ) {
+ return printWatches;
+ }
+
+ static void populateCommandMap() {
+ commandMap.put("create", "path data acl");
+ commandMap.put("delete"," path [version]");
+ commandMap.put("set"," path data [version]");
+ commandMap.put("get"," path [watch]");
+ commandMap.put("ls"," 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","");
+ }
+
static void usage() {
- LOG.info("message", new IOException("USAGE"));
System.err.println("ZooKeeper host:port cmd args");
- System.err.println("\tcreate path data acl");
- System.err.println("\tdelete path [version]");
- System.err.println("\tset path data [version]");
- System.err.println("\tget path [watch]");
- System.err.println("\tls path [watch]");
- System.err.println("\tgetAcl path");
- System.err.println("\tsetAcl path acl");
- System.err.println("\tstat path [watch]");
- System.err.println("\tsync path");
- System.err.println("\tsetquota -n|-b val path");
- System.err.println("\tlistquota path");
- System.err.println("\tdelquotsssa [-n|-b] path");
+ for (String cmd : commandMap.keySet()) {
+ System.err.println("\t"+cmd+commandMap.get(cmd));
}
+ }
- static private class MyWatcher implements Watcher {
+ private class MyWatcher implements Watcher {
public void process(WatchedEvent event) {
- System.err.println(event);
+ if (getPrintWatches()) {
+ ZooKeeperMain.printMessage(event.toString());
+ }
}
}
@@ -85,7 +114,7 @@
break;
default:
System.err
- .println("Unknown perm type: " + permString.charAt(i));
+ .println("Unknown perm type: " + permString.charAt(i));
}
}
return perm;
@@ -105,30 +134,213 @@
System.err.println("numChildren = " + stat.getNumChildren());
}
- public static void main(String args[]) throws NumberFormatException,
- KeeperException, IOException, InterruptedException {
- if (args.length == 1) {
- ZooKeeper zooKeeper = new ZooKeeper(args[0], 5000, new MyWatcher());
- BufferedReader br = new BufferedReader(new InputStreamReader(
- System.in));
- String line;
- while ((line = br.readLine()) != null) {
- line = "ignore " + line;
- args = line.split(" ");
- processCmd(args, zooKeeper);
+ /**
+ * A storage class for both command line options and shell commands.
+ *
+ */
+ static private class MyCommandOptions {
+
+ private Map<String,String> options = null;
+ private List<String> cmdArgs = null;
+ private String command = null;
+
+ public MyCommandOptions() {
+ options = null; command = null;
+ }
+
+ public String getOption(String opt) {
+ return options.get(opt);
+ }
+
+ public String getCommand( ) {
+ return command;
+ }
+
+ public String getCmdArgument( int index ) {
+ return cmdArgs.get(index);
+ }
+
+ public int getNumArguments( ) {
+ return cmdArgs.size();
+ }
+
+ public String[] getArgArray() {
+ return cmdArgs.toArray(new String[0]);
+ }
+
+ private Map<String,String> buildDefaults( ) {
+ options = new HashMap<String,String>( );
+ options.put("server", "localhost:2181");
+ options.put("timeout", "30000");
+ return options;
+ }
+
+ /**
+ * Parses a command line that may contain one or more flags
+ * before an optional command string
+ * @param args command line arguments
+ * @return true if parsing succeeded, false otherwise.
+ */
+ public boolean parseOptions(String[] args) {
+ Map<String, String> ret = buildDefaults();
+ List<String> argList = Arrays.asList(args);
+ Iterator<String> it = argList.iterator();
+
+ while (it.hasNext()) {
+ String opt = it.next();
+ try {
+ if (opt.equals("-server")) {
+ ret.put("server", it.next());
+ } else if (opt.equals("-timeout")) {
+ ret.put("timeout", it.next());
+ }
+ } catch (NoSuchElementException e){
+ System.err.println("Error: no argument found for option "
+ + opt);
+ return false;
+ }
+
+ if (!opt.startsWith("-")) {
+ command = opt;
+ cmdArgs = new ArrayList<String>( );
+ cmdArgs.add( command );
+ while (it.hasNext()) {
+ cmdArgs.add(it.next());
+ }
+ return true;
+ }
+ }
+ return true;
+ }
+
+ /**
+ * Breaks a string into command + arguments.
+ * @param cmdstring string of form "cmd arg1 arg2..etc"
+ * @return true if parsing succeeded.
+ */
+ public boolean parseCommand( String cmdstring ) {
+ String[] args = cmdstring.split(" ");
+ if (args.length == 0){
+ return false;
+ }
+ command = args[0];
+ cmdArgs = Arrays.asList(args);
+ return true;
+ }
+ }
+
+
+ /**
+ * Makes a list of possible completions, either for commands
+ * or for zk nodes if the token to complete begins with /
+ *
+ */
+
+
+ protected void addToHistory(int i,String cmd) {
+ history.put(i, cmd);
+ }
+
+ public static List<String> getCommands() {
+ return new LinkedList<String>(commandMap.keySet());
+ }
+
+ protected String getPrompt() {
+ return "[zkshell: " + commandCount + "] ";
+ }
+
+ public static void printMessage(String msg) {
+ System.out.println("\n"+msg);
+ }
+
+ public static void main(String args[])
+ throws KeeperException, IOException, InterruptedException
+ {
+ populateCommandMap();
+ ZooKeeperMain main = new ZooKeeperMain(args);
+ main.run();
+ }
+
+ public ZooKeeperMain(String args[]) throws IOException {
+ cl.parseOptions(args);
+ System.out.println("Connecting to " + cl.getOption("server"));
+ zk = new ZooKeeper(cl.getOption("server"),
+ Integer.parseInt(cl.getOption("timeout")), new MyWatcher());
+ }
+
+ @SuppressWarnings("unchecked")
+ void run() throws KeeperException, IOException, InterruptedException {
+ if (cl.getCommand() == null) {
+ System.out.println("Welcome to ZooKeeper!");
+
+ boolean jlinemissing = false;
+ // only use jline if it's in the classpath
+ try {
+ Class consoleC = Class.forName("jline.ConsoleReader");
+ Class completorC =
+ Class.forName("org.apache.zookeeper.JLineZNodeCompletor");
+
+ System.out.println("JLine support is enabled");
+
+ Object console =
+ consoleC.getConstructor().newInstance();
+
+ Object completor =
+ completorC.getConstructor(ZooKeeper.class).newInstance(zk);
+ Method addCompletor = consoleC.getMethod("addCompletor",
+ Class.forName("jline.Completor"));
+ addCompletor.invoke(console, completor);
+
+ String line;
+ Method readLine = consoleC.getMethod("readLine", String.class);
+ while ((line = (String)readLine.invoke(console, getPrompt())) != null) {
+ if (!line.equals("")) {
+ cl.parseCommand(line);
+ addToHistory(commandCount,line);
+ processCmd(cl);
+ commandCount++;
+ }
+ }
+ } catch (ClassNotFoundException e) {
+ LOG.debug("Unable to start jline", e);
+ jlinemissing = true;
+ } catch (NoSuchMethodException e) {
+ LOG.debug("Unable to start jline", e);
+ jlinemissing = true;
+ } catch (InvocationTargetException e) {
+ LOG.debug("Unable to start jline", e);
+ jlinemissing = true;
+ } catch (IllegalAccessException e) {
+ LOG.debug("Unable to start jline", e);
+ jlinemissing = true;
+ } catch (InstantiationException e) {
+ LOG.debug("Unable to start jline", e);
+ jlinemissing = true;
+ }
+
+ if (jlinemissing) {
+ System.out.println("JLine support is disabled");
+ BufferedReader br =
+ new BufferedReader(new InputStreamReader(System.in));
+
+ String line;
+ while ((line = br.readLine()) != null) {
+ if (!line.equals("")) {
+ cl.parseCommand(line);
+ addToHistory(commandCount,line);
+ processCmd(cl);
+ commandCount++;
+ }
+ }
}
- } else if (args.length < 3) {
- usage();
- System.exit(-1);
}
- ZooKeeper zooKeeper = new ZooKeeper(args[0], 5000, new MyWatcher());
- boolean watch = processCmd(args, zooKeeper);
+ boolean watch = processCmd(cl);
if (!watch) {
System.exit(0);
- }
+ }
}
-
+
private static DataCallback dataCallback = new DataCallback() {
public void processResult(int rc, String path, Object ctx, byte[] data,
@@ -139,7 +351,7 @@
}
};
-
+
/**
* trim the quota tree to recover unwanted tree elements
* in the quota's tree
@@ -151,8 +363,9 @@
* @throws IOException
* @throws InterruptedException
*/
- private static boolean trimProcQuotas(ZooKeeper zk, String path) throws
- KeeperException, IOException, InterruptedException {
+ private static boolean trimProcQuotas(ZooKeeper zk, String path)
+ throws KeeperException, IOException, InterruptedException
+ {
if (Quotas.quotaZookeeper.equals(path)) {
return true;
}
@@ -161,8 +374,7 @@
zk.delete(path, -1);
String parent = path.substring(0, path.lastIndexOf('/'));
return trimProcQuotas(zk, parent);
- }
- else {
+ } else {
return true;
}
}
@@ -173,7 +385,7 @@
* @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
+ * @param numNodes true if number of nodes needs
* to be unset
* @return true if quota deletion is successful
* @throws KeeperException
@@ -181,8 +393,9 @@
* @throws InterruptedException
*/
public static boolean delQuota(ZooKeeper zk, String path,
- boolean bytes, boolean numNodes) throws KeeperException,
- IOException, InterruptedException {
+ 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) {
@@ -191,7 +404,7 @@
}
byte[] data = null;
try {
- data = zk.getData(quotaPath, false, new Stat());
+ data = zk.getData(quotaPath, false, new Stat());
} catch(KeeperException.NoNodeException ne) {
System.err.println("quota does not exist for " + path);
return true;
@@ -200,12 +413,10 @@
if (bytes && !numNodes) {
strack.setBytes(-1L);
zk.setData(quotaPath, strack.toString().getBytes(), -1);
- }
- else if (!bytes && numNodes) {
+ } else if (!bytes && numNodes) {
strack.setCount(-1);
zk.setData(quotaPath, strack.toString().getBytes(), -1);
- }
- else if (bytes && numNodes) {
+ } else if (bytes && numNodes) {
// delete till you can find a node with more than
// one child
List<String> children = zk.getChildren(parentPath, false);
@@ -218,7 +429,7 @@
}
return true;
}
-
+
private static void checkIfParentQuota(ZooKeeper zk, String path)
throws InterruptedException, KeeperException
{
@@ -242,13 +453,13 @@
}
for (String child: children) {
if (Quotas.limitNode.equals(child)) {
- throw new IllegalArgumentException(path + " has a parent "
+ throw new IllegalArgumentException(path + " has a parent "
+ quotaPath + " which has a quota");
}
}
}
}
-
+
/**
* this method creates a quota node for the path
* @param zk the ZooKeeper client
@@ -257,62 +468,62 @@
* @param numNodes the limit of number of nodes on this path
* @return true if its successful and false if not.
*/
- public static boolean createQuota(ZooKeeper zk, String path,
- long bytes, int numNodes) throws KeeperException, IOException,
- InterruptedException {
- // check if the path exists. We cannot create
+ 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
+ // now check if their is already existing
// parent or child that has quota
-
+
String quotaPath = Quotas.quotaZookeeper;
- // check for more than 2 children --
+ // 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
+ // 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");
- }
- }
+ 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
- // we can proceed further
+ // 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, Ids.OPEN_ACL_UNSAFE,
- CreateMode.PERSISTENT);
- zk.create(Quotas.quotaZookeeper, null, Ids.OPEN_ACL_UNSAFE,
+ CreateMode.PERSISTENT);
+ zk.create(Quotas.quotaZookeeper, null, Ids.OPEN_ACL_UNSAFE,
CreateMode.PERSISTENT);
} catch(KeeperException.NodeExistsException ne) {
// do nothing
}
}
-
- // now create the direct children
+
+ // now create the direct children
// and the stat and quota nodes
String[] splits = path.split("/");
for (int i=1; i<splits.length; i++) {
quotaPath = quotaPath + "/" + splits[i];
try {
- zk.create(quotaPath, null, Ids.OPEN_ACL_UNSAFE ,
+ zk.create(quotaPath, null, Ids.OPEN_ACL_UNSAFE ,
CreateMode.PERSISTENT);
} catch(KeeperException.NodeExistsException ne) {
//do nothing
@@ -324,12 +535,12 @@
strack.setBytes(bytes);
strack.setCount(numNodes);
try {
- zk.create(quotaPath, strack.toString().getBytes(),
+ zk.create(quotaPath, strack.toString().getBytes(),
Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
StatsTrack stats = new StatsTrack(null);
stats.setBytes(0L);
stats.setCount(0);
- zk.create(statPath, stats.toString().getBytes(),
+ zk.create(statPath, stats.toString().getBytes(),
Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
} catch(KeeperException.NodeExistsException ne) {
byte[] data = zk.getData(quotaPath, false , new Stat());
@@ -344,117 +555,168 @@
}
return true;
}
-
- private static boolean processCmd(String[] args, ZooKeeper zooKeeper)
- throws KeeperException, IOException, InterruptedException {
+
+ protected boolean processCmd(MyCommandOptions co)
+ throws KeeperException, IOException, InterruptedException
+ {
+ try {
+ return processZKCmd(co);
+ } catch (IllegalArgumentException e) {
+ System.err.println("Command failed: " + e);
+ } catch (KeeperException.NoNodeException e) {
+ System.err.println("Node does not exist: " + e.getPath());
+ } catch (KeeperException.NoChildrenForEphemeralsException e) {
+ System.err.println("Ephemerals cannot have children: "
+ + e.getPath());
+ } catch (KeeperException.NodeExistsException e) {
+ System.err.println("Node already exists: " + e.getPath());
+ } catch (KeeperException.NotEmptyException e) {
+ System.err.println("Node not empty: " + e.getPath());
+ }
+ return false;
+ }
+
+ protected boolean processZKCmd(MyCommandOptions co)
+ throws KeeperException, IOException, InterruptedException
+ {
Stat stat = new Stat();
- if (args.length < 2) {
+ String[] args = co.getArgArray();
+ String cmd = co.getCommand();
+ if (args.length < 1) {
usage();
return false;
}
-
- String cmd = args[1];
- boolean watch = args.length > 3;
+
+ boolean watch = args.length > 2;
String path = null;
List<ACL> acl = Ids.OPEN_ACL_UNSAFE;
- System.out.println("Processing " + cmd);
- if (cmd.equals("create") && args.length >= 4) {
- if (args.length == 5) {
- acl = parseACLs(args[4]);
+ LOG.debug("Processing " + cmd);
+ if (cmd.equals("create") && args.length >= 3) {
+ if (args.length == 4) {
+ acl = parseACLs(args[3]);
}
- path = args[2];
- String newPath = zooKeeper.create(path, args[3].getBytes(), acl, CreateMode.PERSISTENT);
+ path = args[1];
+ String newPath = zk.create(path, args[2].getBytes(), acl,
+ CreateMode.PERSISTENT);
System.err.println("Created " + newPath);
- } else if (cmd.equals("delete") && args.length >= 3) {
- path = args[2];
- zooKeeper.delete(path, watch ? Integer.parseInt(args[3]) : -1);
- } else if (cmd.equals("set") && args.length >= 4) {
- path = args[2];
- stat = zooKeeper.setData(path, args[3].getBytes(),
- args.length > 4 ? Integer.parseInt(args[4]) : -1);
+ } else if (cmd.equals("delete") && args.length >= 2) {
+ path = args[1];
+ zk.delete(path, watch ? Integer.parseInt(args[2]) : -1);
+ } 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 >= 3) {
- path = args[2];
- zooKeeper.getData(path, watch, dataCallback, path);
- } else if (cmd.equals("get") && args.length >= 3) {
- path = args[2];
- byte data[] = zooKeeper.getData(path, watch, 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);
System.out.println(new String(data));
printStat(stat);
- } else if (cmd.equals("ls") && args.length >= 3) {
- path = args[2];
- List<String> children = zooKeeper.getChildren(path, watch);
+ } 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("getAcl") && args.length >= 2) {
- path = args[2];
- acl = zooKeeper.getACL(path, stat);
+ 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 >= 4) {
- path = args[2];
- stat = zooKeeper.setACL(path, parseACLs(args[3]),
- args.length > 4 ? Integer.parseInt(args[4]) : -1);
+ } 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 >= 3) {
- path = args[2];
- stat = zooKeeper.exists(path, watch);
+ } else if (cmd.equals("stat") && args.length >= 2) {
+ path = args[1];
+ stat = zk.exists(path, watch);
printStat(stat);
- } else if (cmd.equals("listquota") && args.length >= 3) {
- path = args[2];
+ } else if (cmd.equals("listquota") && args.length >= 2) {
+ path = args[1];
String absolutePath = Quotas.quotaZookeeper + path + "/" + Quotas.limitNode;
byte[] data = null;
- try {
+ try {
System.err.println("absolute path is " + absolutePath);
- data = zooKeeper.getData(absolutePath, false, stat);
+ data = zk.getData(absolutePath, false, stat);
StatsTrack st = new StatsTrack(new String(data));
- System.out.println("Output quota for " + path + " "
+ System.out.println("Output quota for " + path + " "
+ st.toString());
-
- data = zooKeeper.getData(Quotas.quotaZookeeper + path + "/" +
+
+ data = zk.getData(Quotas.quotaZookeeper + path + "/" +
Quotas.statNode, false, stat);
System.out.println("Output stat for " + path + " " +
- new StatsTrack(new String(data)).toString());
+ 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[2];
- path = args[4];
+ } else if (cmd.equals("quit")) {
+ System.out.println("Quitting...");
+ zk.close();
+ System.exit(0);
+ } else if (cmd.equals("setquota") && args.length >= 4) {
+ String option = args[1];
+ path = args[3];
System.err.println("Comment: the parts are " +
- "option " + option + " path " +
- args[4] + " val " + args[3]);
+ "option " + option + " path " +
+ args[3] + " val " + args[2]);
String val = args[3];
if ("-b".equals(option)) {
// we are setting the bytes quota
- createQuota(zooKeeper, path, Long.parseLong(val), -1);
+ createQuota(zk, path, Long.parseLong(val), -1);
} else if ("-n".equals(option)) {
// we are setting the num quota
- createQuota(zooKeeper, path, -1L, Integer.parseInt(val));
- }
- else {
+ createQuota(zk, path, -1L, Integer.parseInt(val));
+ } else {
usage();
}
- } else if (cmd.equals("delquota") && args.length >= 3) {
- //if neither option -n or -b is specified, we delete
+
+ } else if (cmd.equals("redo") && args.length >= 2) {
+ Integer i = Integer.decode(args[1]);
+ if (commandCount <= i){ // don't allow redoing this redo
+ System.out.println("Command index out of range");
+ return false;
+ }
+ cl.parseCommand(history.get(i));
+ if (cl.getCommand().equals( "redo" )){
+ System.out.println("No redoing redos");
+ return false;
+ }
+ history.put(commandCount, history.get(i));
+ processCmd( cl);
+ } else if (cmd.equals("history")) {
+ for (int i=commandCount - 10;i<=commandCount;++i) {
+ if (i < 0) continue;
+ System.out.println(i + " - " + history.get(i));
+ }
+ } 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 == 4) {
+ if (args.length == 3) {
//this time we have an option
- String option = args[2];
- path = args[3];
+ String option = args[1];
+ path = args[2];
if ("-b".equals(option)) {
- delQuota(zooKeeper, path, true, false);
+ delQuota(zk, path, true, false);
} else if ("-n".equals(option)) {
- delQuota(zooKeeper, path, false, true);
+ delQuota(zk, path, false, true);
}
- }
- else if (args.length == 3) {
- path = args[2];
+ } else if (args.length == 2) {
+ path = args[1];
// we dont have an option specified.
// just delete whole quota node
- delQuota(zooKeeper, path, true, true);
+ delQuota(zk, path, true, true);
+ } else if (cmd.equals("help")) {
+ usage();
+ }
+ } else if (cmd.equals("printwatches")) {
+ if (args.length == 1) {
+ System.out.println("printwatches is " + (printWatches ? "on" : "off"));
+ } else {
+ printWatches = args[1].equals("on");
}
} else {
usage();
@@ -491,7 +753,7 @@
int lastColon = a.lastIndexOf(':');
if (firstColon == -1 || lastColon == -1 || firstColon == lastColon) {
System.err
- .println(a + " does not have the form scheme:id:perm");
+ .println(a + " does not have the form scheme:id:perm");
continue;
}
ACL newAcl = new ACL();
@@ -502,5 +764,4 @@
}
return acl;
}
-
}
|