You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@accumulo.apache.org by dl...@apache.org on 2014/09/08 04:32:39 UTC

[1/3] git commit: ACCUMULO-3093: Use ContextManager to load the Formatter classes

Repository: accumulo
Updated Branches:
  refs/heads/master 918e686fc -> 2533b7eff


ACCUMULO-3093: Use ContextManager to load the Formatter classes


Project: http://git-wip-us.apache.org/repos/asf/accumulo/repo
Commit: http://git-wip-us.apache.org/repos/asf/accumulo/commit/7eeb08ae
Tree: http://git-wip-us.apache.org/repos/asf/accumulo/tree/7eeb08ae
Diff: http://git-wip-us.apache.org/repos/asf/accumulo/diff/7eeb08ae

Branch: refs/heads/master
Commit: 7eeb08ae940407ee29ce8673ea77a98a87ecd311
Parents: 8b2cbe7
Author: Dave Marion <dl...@hotmail.com>
Authored: Sun Sep 7 21:57:45 2014 -0400
Committer: Dave Marion <dl...@hotmail.com>
Committed: Sun Sep 7 21:57:45 2014 -0400

----------------------------------------------------------------------
 .../apache/accumulo/core/util/shell/Shell.java  | 58 ++++++++++++++++++++
 .../core/util/shell/commands/ScanCommand.java   |  8 +--
 .../util/shell/commands/SetIterCommand.java     | 55 +------------------
 .../ShellPluginConfigurationCommand.java        | 23 +++++++-
 4 files changed, 83 insertions(+), 61 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/7eeb08ae/core/src/main/java/org/apache/accumulo/core/util/shell/Shell.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/util/shell/Shell.java b/core/src/main/java/org/apache/accumulo/core/util/shell/Shell.java
index ff6ba09..6043df3 100644
--- a/core/src/main/java/org/apache/accumulo/core/util/shell/Shell.java
+++ b/core/src/main/java/org/apache/accumulo/core/util/shell/Shell.java
@@ -49,6 +49,7 @@ import org.apache.accumulo.core.client.ClientConfiguration.ClientProperty;
 import org.apache.accumulo.core.client.Connector;
 import org.apache.accumulo.core.client.Instance;
 import org.apache.accumulo.core.client.IteratorSetting;
+import org.apache.accumulo.core.client.NamespaceNotFoundException;
 import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.client.ZooKeeperInstance;
 import org.apache.accumulo.core.client.impl.ServerConfigurationUtil;
@@ -130,6 +131,7 @@ import org.apache.accumulo.core.util.shell.commands.NamespacesCommand;
 import org.apache.accumulo.core.util.shell.commands.NoTableCommand;
 import org.apache.accumulo.core.util.shell.commands.OfflineCommand;
 import org.apache.accumulo.core.util.shell.commands.OnlineCommand;
+import org.apache.accumulo.core.util.shell.commands.OptUtil;
 import org.apache.accumulo.core.util.shell.commands.PasswdCommand;
 import org.apache.accumulo.core.util.shell.commands.PingCommand;
 import org.apache.accumulo.core.util.shell.commands.QuestionCommand;
@@ -158,6 +160,8 @@ import org.apache.accumulo.core.util.shell.commands.WhoAmICommand;
 import org.apache.accumulo.core.volume.VolumeConfiguration;
 import org.apache.accumulo.core.zookeeper.ZooUtil;
 import org.apache.accumulo.fate.zookeeper.ZooReader;
+import org.apache.accumulo.start.classloader.vfs.AccumuloVFSClassLoader;
+import org.apache.accumulo.start.classloader.vfs.ContextManager;
 import org.apache.commons.cli.BasicParser;
 import org.apache.commons.cli.CommandLine;
 import org.apache.commons.cli.HelpFormatter;
@@ -165,6 +169,7 @@ import org.apache.commons.cli.MissingOptionException;
 import org.apache.commons.cli.Option;
 import org.apache.commons.cli.Options;
 import org.apache.commons.cli.ParseException;
+import org.apache.commons.vfs2.FileSystemException;
 import org.apache.hadoop.fs.Path;
 import org.apache.log4j.Level;
 import org.apache.log4j.Logger;
@@ -454,6 +459,59 @@ public class Shell extends ShellOptions {
     return instance;
   }
 
+  public ClassLoader getClassLoader(final CommandLine cl, final Shell shellState) throws AccumuloException, TableNotFoundException, AccumuloSecurityException,
+      IOException, FileSystemException {
+
+    boolean tables = cl.hasOption(OptUtil.tableOpt().getOpt()) || !shellState.getTableName().isEmpty();
+    boolean namespaces = cl.hasOption(OptUtil.namespaceOpt().getOpt());
+
+    String classpath = null;
+    Iterable<Entry<String,String>> tableProps;
+
+    if (namespaces) {
+      try {
+        tableProps = shellState.getConnector().namespaceOperations().getProperties(OptUtil.getNamespaceOpt(cl, shellState));
+      } catch (NamespaceNotFoundException e) {
+        throw new IllegalArgumentException(e);
+      }
+    } else if (tables) {
+      tableProps = shellState.getConnector().tableOperations().getProperties(OptUtil.getTableOpt(cl, shellState));
+    } else {
+      throw new IllegalArgumentException("No table or namespace specified");
+    }
+    for (Entry<String,String> entry : tableProps) {
+      if (entry.getKey().equals(Property.TABLE_CLASSPATH.getKey())) {
+        classpath = entry.getValue();
+      }
+    }
+
+    ClassLoader classloader;
+
+    if (classpath != null && !classpath.equals("")) {
+      shellState.getConnector().instanceOperations().getSystemConfiguration().get(Property.VFS_CONTEXT_CLASSPATH_PROPERTY.getKey() + classpath);
+
+      try {
+        AccumuloVFSClassLoader.getContextManager().setContextConfig(new ContextManager.DefaultContextsConfig(new Iterable<Map.Entry<String,String>>() {
+          @Override
+          public Iterator<Entry<String,String>> iterator() {
+            try {
+              return shellState.getConnector().instanceOperations().getSystemConfiguration().entrySet().iterator();
+            } catch (AccumuloException e) {
+              throw new RuntimeException(e);
+            } catch (AccumuloSecurityException e) {
+              throw new RuntimeException(e);
+            }
+          }
+        }));
+      } catch (IllegalStateException ise) {}
+
+      classloader = AccumuloVFSClassLoader.getContextManager().getClassLoader(classpath);
+    } else {
+      classloader = AccumuloVFSClassLoader.getClassLoader();
+    }
+    return classloader;
+  }
+
   public static void main(String args[]) throws IOException {
     Shell shell = new Shell();
     try {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/7eeb08ae/core/src/main/java/org/apache/accumulo/core/util/shell/commands/ScanCommand.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/util/shell/commands/ScanCommand.java b/core/src/main/java/org/apache/accumulo/core/util/shell/commands/ScanCommand.java
index a02a781..0270e6c 100644
--- a/core/src/main/java/org/apache/accumulo/core/util/shell/commands/ScanCommand.java
+++ b/core/src/main/java/org/apache/accumulo/core/util/shell/commands/ScanCommand.java
@@ -28,6 +28,7 @@ import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.IteratorSetting;
 import org.apache.accumulo.core.client.Scanner;
 import org.apache.accumulo.core.client.ScannerBase;
+import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Range;
@@ -199,12 +200,11 @@ public class ScanCommand extends Command {
     
     try {
       if (cl.hasOption(formatterOpt.getOpt())) {
-        return AccumuloVFSClassLoader.loadClass(cl.getOptionValue(formatterOpt.getOpt()), Formatter.class);
-        
+        return shellState.getClassLoader(cl, shellState).loadClass(cl.getOptionValue(formatterOpt.getOpt())).asSubclass(Formatter.class);
       } else if (cl.hasOption(formatterInterpeterOpt.getOpt())) {
-        return AccumuloVFSClassLoader.loadClass(cl.getOptionValue(formatterInterpeterOpt.getOpt()), Formatter.class);
+        return shellState.getClassLoader(cl, shellState).loadClass(cl.getOptionValue(formatterInterpeterOpt.getOpt())).asSubclass(Formatter.class);
       }
-    } catch (ClassNotFoundException e) {
+    } catch (Exception e) {
       shellState.getReader().println("Formatter class could not be loaded.\n" + e.getMessage());
     }
     

http://git-wip-us.apache.org/repos/asf/accumulo/blob/7eeb08ae/core/src/main/java/org/apache/accumulo/core/util/shell/commands/SetIterCommand.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/util/shell/commands/SetIterCommand.java b/core/src/main/java/org/apache/accumulo/core/util/shell/commands/SetIterCommand.java
index 3d8a5a7..4be178a 100644
--- a/core/src/main/java/org/apache/accumulo/core/util/shell/commands/SetIterCommand.java
+++ b/core/src/main/java/org/apache/accumulo/core/util/shell/commands/SetIterCommand.java
@@ -85,7 +85,7 @@ public class SetIterCommand extends Command {
       classname = ReqVisFilter.class.getName();
     }
 
-    ClassLoader classloader = getClassLoader(cl, shellState);
+    ClassLoader classloader = shellState.getClassLoader(cl, shellState);
 
     // Get the iterator options, with potentially a name provided by the OptionDescriber impl or through user input
     String configuredName = setUpOptions(classloader, shellState.getReader(), classname, options);
@@ -115,59 +115,6 @@ public class SetIterCommand extends Command {
     return 0;
   }
 
-  private ClassLoader getClassLoader(final CommandLine cl, final Shell shellState) throws AccumuloException, TableNotFoundException, AccumuloSecurityException,
-      IOException, FileSystemException {
-
-    boolean tables = cl.hasOption(OptUtil.tableOpt().getOpt()) || !shellState.getTableName().isEmpty();
-    boolean namespaces = cl.hasOption(OptUtil.namespaceOpt().getOpt());
-
-    String classpath = null;
-    Iterable<Entry<String,String>> tableProps;
-
-    if (namespaces) {
-      try {
-        tableProps = shellState.getConnector().namespaceOperations().getProperties(OptUtil.getNamespaceOpt(cl, shellState));
-      } catch (NamespaceNotFoundException e) {
-        throw new IllegalArgumentException(e);
-      }
-    } else if (tables) {
-      tableProps = shellState.getConnector().tableOperations().getProperties(OptUtil.getTableOpt(cl, shellState));
-    } else {
-      throw new IllegalArgumentException("No table or namespace specified");
-    }
-    for (Entry<String,String> entry : tableProps) {
-      if (entry.getKey().equals(Property.TABLE_CLASSPATH.getKey())) {
-        classpath = entry.getValue();
-      }
-    }
-
-    ClassLoader classloader;
-
-    if (classpath != null && !classpath.equals("")) {
-      shellState.getConnector().instanceOperations().getSystemConfiguration().get(Property.VFS_CONTEXT_CLASSPATH_PROPERTY.getKey() + classpath);
-
-      try {
-        AccumuloVFSClassLoader.getContextManager().setContextConfig(new ContextManager.DefaultContextsConfig(new Iterable<Map.Entry<String,String>>() {
-          @Override
-          public Iterator<Entry<String,String>> iterator() {
-            try {
-              return shellState.getConnector().instanceOperations().getSystemConfiguration().entrySet().iterator();
-            } catch (AccumuloException e) {
-              throw new RuntimeException(e);
-            } catch (AccumuloSecurityException e) {
-              throw new RuntimeException(e);
-            }
-          }
-        }));
-      } catch (IllegalStateException ise) {}
-
-      classloader = AccumuloVFSClassLoader.getContextManager().getClassLoader(classpath);
-    } else {
-      classloader = AccumuloVFSClassLoader.getClassLoader();
-    }
-    return classloader;
-  }
-
   protected void setTableProperties(final CommandLine cl, final Shell shellState, final int priority, final Map<String,String> options, final String classname,
       final String name) throws AccumuloException, AccumuloSecurityException, ShellCommandException, TableNotFoundException {
     // remove empty values

http://git-wip-us.apache.org/repos/asf/accumulo/blob/7eeb08ae/core/src/main/java/org/apache/accumulo/core/util/shell/commands/ShellPluginConfigurationCommand.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/util/shell/commands/ShellPluginConfigurationCommand.java b/core/src/main/java/org/apache/accumulo/core/util/shell/commands/ShellPluginConfigurationCommand.java
index d4c9739..39bba5c 100644
--- a/core/src/main/java/org/apache/accumulo/core/util/shell/commands/ShellPluginConfigurationCommand.java
+++ b/core/src/main/java/org/apache/accumulo/core/util/shell/commands/ShellPluginConfigurationCommand.java
@@ -16,6 +16,7 @@
  */
 package org.apache.accumulo.core.util.shell.commands;
 
+import java.util.Arrays;
 import java.util.Iterator;
 import java.util.Map.Entry;
 
@@ -25,11 +26,12 @@ import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.util.shell.Shell;
 import org.apache.accumulo.core.util.shell.Shell.Command;
-import org.apache.accumulo.start.classloader.vfs.AccumuloVFSClassLoader;
+import org.apache.commons.cli.BasicParser;
 import org.apache.commons.cli.CommandLine;
 import org.apache.commons.cli.Option;
 import org.apache.commons.cli.OptionGroup;
 import org.apache.commons.cli.Options;
+import org.apache.commons.cli.ParseException;
 import org.apache.log4j.Logger;
 
 public abstract class ShellPluginConfigurationCommand extends Command {
@@ -101,10 +103,25 @@ public abstract class ShellPluginConfigurationCommand extends Command {
       final Entry<String,String> ent = props.next();
       if (ent.getKey().equals(pluginProp.toString())) {
         Class<? extends T> pluginClazz;
+        String[] args = new String[2];
         try {
-          pluginClazz = AccumuloVFSClassLoader.loadClass(ent.getValue(), clazz);
+          Options o = new Options();
+          o.addOption(OptUtil.tableOpt());
+          args[0] = "-t";
+          args[1] = tableName;
+          CommandLine cl = new BasicParser().parse(o, args);
+          pluginClazz = shellState.getClassLoader(cl, shellState).loadClass(ent.getValue()).asSubclass(clazz);
         } catch (ClassNotFoundException e) {
-          Logger.getLogger(ShellPluginConfigurationCommand.class).warn("Class not found" + e.getMessage());
+          Logger.getLogger(ShellPluginConfigurationCommand.class).error("Class not found" + e.getMessage());
+          return null;
+        } catch (ParseException e) {
+          Logger.getLogger(ShellPluginConfigurationCommand.class).error("Error parsing table: " + Arrays.toString(args) + e.getMessage());
+          return null;
+        } catch (TableNotFoundException e) {
+          Logger.getLogger(ShellPluginConfigurationCommand.class).error("Table not found: " + tableName + e.getMessage());
+          return null;
+        } catch (Exception e) {
+          Logger.getLogger(ShellPluginConfigurationCommand.class).error("Error: " + e.getMessage());
           return null;
         }
         


[2/3] Merge branch '1.6.1-SNAPSHOT'

Posted by dl...@apache.org.
http://git-wip-us.apache.org/repos/asf/accumulo/blob/2533b7ef/shell/src/main/java/org/apache/accumulo/shell/commands/SetIterCommand.java
----------------------------------------------------------------------
diff --cc shell/src/main/java/org/apache/accumulo/shell/commands/SetIterCommand.java
index a398b3a,0000000..7505c90
mode 100644,000000..100644
--- a/shell/src/main/java/org/apache/accumulo/shell/commands/SetIterCommand.java
+++ b/shell/src/main/java/org/apache/accumulo/shell/commands/SetIterCommand.java
@@@ -1,453 -1,0 +1,400 @@@
 +/*
 + * 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.accumulo.shell.commands;
 +
 +import java.io.IOException;
 +import java.util.EnumSet;
 +import java.util.HashMap;
 +import java.util.Iterator;
 +import java.util.Map;
 +import java.util.Map.Entry;
 +
 +import jline.console.ConsoleReader;
 +
 +import org.apache.accumulo.core.client.AccumuloException;
 +import org.apache.accumulo.core.client.AccumuloSecurityException;
 +import org.apache.accumulo.core.client.IteratorSetting;
 +import org.apache.accumulo.core.client.NamespaceNotFoundException;
 +import org.apache.accumulo.core.client.TableNotFoundException;
 +import org.apache.accumulo.core.conf.Property;
 +import org.apache.accumulo.core.data.Key;
 +import org.apache.accumulo.core.data.Value;
 +import org.apache.accumulo.core.iterators.IteratorUtil.IteratorScope;
 +import org.apache.accumulo.core.iterators.OptionDescriber;
 +import org.apache.accumulo.core.iterators.OptionDescriber.IteratorOptions;
 +import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
 +import org.apache.accumulo.core.iterators.user.AgeOffFilter;
 +import org.apache.accumulo.core.iterators.user.RegExFilter;
 +import org.apache.accumulo.core.iterators.user.ReqVisFilter;
 +import org.apache.accumulo.core.iterators.user.VersioningIterator;
 +import org.apache.accumulo.shell.Shell;
 +import org.apache.accumulo.shell.ShellCommandException;
 +import org.apache.accumulo.shell.Shell.Command;
 +import org.apache.accumulo.shell.ShellCommandException.ErrorCode;
 +import org.apache.accumulo.start.classloader.vfs.AccumuloVFSClassLoader;
 +import org.apache.accumulo.start.classloader.vfs.ContextManager;
 +import org.apache.commons.cli.CommandLine;
 +import org.apache.commons.cli.Option;
 +import org.apache.commons.cli.OptionGroup;
 +import org.apache.commons.cli.Options;
 +import org.apache.commons.lang.StringUtils;
 +import org.apache.commons.vfs2.FileSystemException;
 +
 +public class SetIterCommand extends Command {
 +
 +  private Option allScopeOpt, mincScopeOpt, majcScopeOpt, scanScopeOpt, nameOpt, priorityOpt;
 +  private Option aggTypeOpt, ageoffTypeOpt, regexTypeOpt, versionTypeOpt, reqvisTypeOpt, classnameTypeOpt;
 +
 +  @Override
 +  public int execute(final String fullCommand, final CommandLine cl, final Shell shellState) throws AccumuloException, AccumuloSecurityException,
 +      TableNotFoundException, IOException, ShellCommandException {
 +
 +    boolean tables = cl.hasOption(OptUtil.tableOpt().getOpt()) || !shellState.getTableName().isEmpty();
 +    boolean namespaces = cl.hasOption(OptUtil.namespaceOpt().getOpt());
 +
 +    final int priority = Integer.parseInt(cl.getOptionValue(priorityOpt.getOpt()));
 +
 +    final Map<String,String> options = new HashMap<String,String>();
 +    String classname = cl.getOptionValue(classnameTypeOpt.getOpt());
 +    if (cl.hasOption(aggTypeOpt.getOpt())) {
 +      Shell.log.warn("aggregators are deprecated");
 +      @SuppressWarnings("deprecation")
 +      String deprecatedClassName = org.apache.accumulo.core.iterators.AggregatingIterator.class.getName();
 +      classname = deprecatedClassName;
 +    } else if (cl.hasOption(regexTypeOpt.getOpt())) {
 +      classname = RegExFilter.class.getName();
 +    } else if (cl.hasOption(ageoffTypeOpt.getOpt())) {
 +      classname = AgeOffFilter.class.getName();
 +    } else if (cl.hasOption(versionTypeOpt.getOpt())) {
 +      classname = VersioningIterator.class.getName();
 +    } else if (cl.hasOption(reqvisTypeOpt.getOpt())) {
 +      classname = ReqVisFilter.class.getName();
 +    }
 +
-     ClassLoader classloader = getClassLoader(cl, shellState);
++    ClassLoader classloader = shellState.getClassLoader(cl, shellState);
 +
 +    // Get the iterator options, with potentially a name provided by the OptionDescriber impl or through user input
 +    String configuredName = setUpOptions(classloader, shellState.getReader(), classname, options);
 +
 +    // Try to get the name provided by the setiter command
 +    String name = cl.getOptionValue(nameOpt.getOpt(), null);
 +    
 +    // Cannot continue if no name is provided
 +    if (null == name && null == configuredName) {
 +      throw new IllegalArgumentException("No provided or default name for iterator");
 +    } else if (null == name) {
 +      // Fall back to the name from OptionDescriber or user input if none is provided on setiter option
 +      name = configuredName;
 +    }
 +
 +    if (namespaces) {
 +      try {
 +        setNamespaceProperties(cl, shellState, priority, options, classname, name);
 +      } catch (NamespaceNotFoundException e) {
 +        throw new IllegalArgumentException(e);
 +      }
 +    } else if (tables) {
 +      setTableProperties(cl, shellState, priority, options, classname, name);
 +    } else {
 +      throw new IllegalArgumentException("No table or namespace specified");
 +    }
 +    return 0;
 +  }
 +
-   private ClassLoader getClassLoader(final CommandLine cl, final Shell shellState) throws AccumuloException, TableNotFoundException, AccumuloSecurityException,
-       IOException, FileSystemException {
- 
-     boolean tables = cl.hasOption(OptUtil.tableOpt().getOpt()) || !shellState.getTableName().isEmpty();
-     boolean namespaces = cl.hasOption(OptUtil.namespaceOpt().getOpt());
- 
-     String classpath = null;
-     Iterable<Entry<String,String>> tableProps;
- 
-     if (namespaces) {
-       try {
-         tableProps = shellState.getConnector().namespaceOperations().getProperties(OptUtil.getNamespaceOpt(cl, shellState));
-       } catch (NamespaceNotFoundException e) {
-         throw new IllegalArgumentException(e);
-       }
-     } else if (tables) {
-       tableProps = shellState.getConnector().tableOperations().getProperties(OptUtil.getTableOpt(cl, shellState));
-     } else {
-       throw new IllegalArgumentException("No table or namespace specified");
-     }
-     for (Entry<String,String> entry : tableProps) {
-       if (entry.getKey().equals(Property.TABLE_CLASSPATH.getKey())) {
-         classpath = entry.getValue();
-       }
-     }
- 
-     ClassLoader classloader;
- 
-     if (classpath != null && !classpath.equals("")) {
-       shellState.getConnector().instanceOperations().getSystemConfiguration().get(Property.VFS_CONTEXT_CLASSPATH_PROPERTY.getKey() + classpath);
- 
-       try {
-         AccumuloVFSClassLoader.getContextManager().setContextConfig(new ContextManager.DefaultContextsConfig(new Iterable<Map.Entry<String,String>>() {
-           @Override
-           public Iterator<Entry<String,String>> iterator() {
-             try {
-               return shellState.getConnector().instanceOperations().getSystemConfiguration().entrySet().iterator();
-             } catch (AccumuloException e) {
-               throw new RuntimeException(e);
-             } catch (AccumuloSecurityException e) {
-               throw new RuntimeException(e);
-             }
-           }
-         }));
-       } catch (IllegalStateException ise) {}
- 
-       classloader = AccumuloVFSClassLoader.getContextManager().getClassLoader(classpath);
-     } else {
-       classloader = AccumuloVFSClassLoader.getClassLoader();
-     }
-     return classloader;
-   }
- 
 +  protected void setTableProperties(final CommandLine cl, final Shell shellState, final int priority, final Map<String,String> options, final String classname,
 +      final String name) throws AccumuloException, AccumuloSecurityException, ShellCommandException, TableNotFoundException {
 +    // remove empty values
 +
 +    final String tableName = OptUtil.getTableOpt(cl, shellState);
 +
 +    if (!shellState.getConnector().tableOperations().testClassLoad(tableName, classname, SortedKeyValueIterator.class.getName())) {
 +      throw new ShellCommandException(ErrorCode.INITIALIZATION_FAILURE, "Servers are unable to load " + classname + " as type "
 +          + SortedKeyValueIterator.class.getName());
 +    }
 +
 +    final String aggregatorClass = options.get("aggregatorClass");
 +    @SuppressWarnings("deprecation")
 +    String deprecatedAggregatorClassName = org.apache.accumulo.core.iterators.aggregation.Aggregator.class.getName();
 +    if (aggregatorClass != null && !shellState.getConnector().tableOperations().testClassLoad(tableName, aggregatorClass, deprecatedAggregatorClassName)) {
 +      throw new ShellCommandException(ErrorCode.INITIALIZATION_FAILURE, "Servers are unable to load " + aggregatorClass + " as type "
 +          + deprecatedAggregatorClassName);
 +    }
 +
 +    for (Iterator<Entry<String,String>> i = options.entrySet().iterator(); i.hasNext();) {
 +      final Entry<String,String> entry = i.next();
 +      if (entry.getValue() == null || entry.getValue().isEmpty()) {
 +        i.remove();
 +      }
 +    }
 +    final EnumSet<IteratorScope> scopes = EnumSet.noneOf(IteratorScope.class);
 +    if (cl.hasOption(allScopeOpt.getOpt()) || cl.hasOption(mincScopeOpt.getOpt())) {
 +      scopes.add(IteratorScope.minc);
 +    }
 +    if (cl.hasOption(allScopeOpt.getOpt()) || cl.hasOption(majcScopeOpt.getOpt())) {
 +      scopes.add(IteratorScope.majc);
 +    }
 +    if (cl.hasOption(allScopeOpt.getOpt()) || cl.hasOption(scanScopeOpt.getOpt())) {
 +      scopes.add(IteratorScope.scan);
 +    }
 +    if (scopes.isEmpty()) {
 +      throw new IllegalArgumentException("You must select at least one scope to configure");
 +    }
 +    final IteratorSetting setting = new IteratorSetting(priority, name, classname, options);
 +    shellState.getConnector().tableOperations().attachIterator(tableName, setting, scopes);
 +  }
 +
 +  protected void setNamespaceProperties(final CommandLine cl, final Shell shellState, final int priority, final Map<String,String> options,
 +      final String classname, final String name) throws AccumuloException, AccumuloSecurityException, ShellCommandException, NamespaceNotFoundException {
 +    // remove empty values
 +
 +    final String namespace = OptUtil.getNamespaceOpt(cl, shellState);
 +
 +    if (!shellState.getConnector().namespaceOperations().testClassLoad(namespace, classname, SortedKeyValueIterator.class.getName())) {
 +      throw new ShellCommandException(ErrorCode.INITIALIZATION_FAILURE, "Servers are unable to load " + classname + " as type "
 +          + SortedKeyValueIterator.class.getName());
 +    }
 +
 +    final String aggregatorClass = options.get("aggregatorClass");
 +    @SuppressWarnings("deprecation")
 +    String deprecatedAggregatorClassName = org.apache.accumulo.core.iterators.aggregation.Aggregator.class.getName();
 +    if (aggregatorClass != null && !shellState.getConnector().namespaceOperations().testClassLoad(namespace, aggregatorClass, deprecatedAggregatorClassName)) {
 +      throw new ShellCommandException(ErrorCode.INITIALIZATION_FAILURE, "Servers are unable to load " + aggregatorClass + " as type "
 +          + deprecatedAggregatorClassName);
 +    }
 +
 +    for (Iterator<Entry<String,String>> i = options.entrySet().iterator(); i.hasNext();) {
 +      final Entry<String,String> entry = i.next();
 +      if (entry.getValue() == null || entry.getValue().isEmpty()) {
 +        i.remove();
 +      }
 +    }
 +    final EnumSet<IteratorScope> scopes = EnumSet.noneOf(IteratorScope.class);
 +    if (cl.hasOption(allScopeOpt.getOpt()) || cl.hasOption(mincScopeOpt.getOpt())) {
 +      scopes.add(IteratorScope.minc);
 +    }
 +    if (cl.hasOption(allScopeOpt.getOpt()) || cl.hasOption(majcScopeOpt.getOpt())) {
 +      scopes.add(IteratorScope.majc);
 +    }
 +    if (cl.hasOption(allScopeOpt.getOpt()) || cl.hasOption(scanScopeOpt.getOpt())) {
 +      scopes.add(IteratorScope.scan);
 +    }
 +    if (scopes.isEmpty()) {
 +      throw new IllegalArgumentException("You must select at least one scope to configure");
 +    }
 +    final IteratorSetting setting = new IteratorSetting(priority, name, classname, options);
 +    shellState.getConnector().namespaceOperations().attachIterator(namespace, setting, scopes);
 +  }
 +
 +  private static String setUpOptions(ClassLoader classloader, final ConsoleReader reader, final String className, final Map<String,String> options)
 +      throws IOException, ShellCommandException {
 +    String input;
 +    @SuppressWarnings("rawtypes")
 +    SortedKeyValueIterator untypedInstance;
 +    @SuppressWarnings("rawtypes")
 +    Class<? extends SortedKeyValueIterator> clazz;
 +    try {
 +      clazz = classloader.loadClass(className).asSubclass(SortedKeyValueIterator.class);
 +      untypedInstance = clazz.newInstance();
 +    } catch (ClassNotFoundException e) {
 +      StringBuilder msg = new StringBuilder("Unable to load ").append(className);
 +      if (className.indexOf('.') < 0) {
 +        msg.append("; did you use a fully qualified package name?");
 +      } else {
 +        msg.append("; class not found.");
 +      }
 +      throw new ShellCommandException(ErrorCode.INITIALIZATION_FAILURE, msg.toString());
 +    } catch (InstantiationException e) {
 +      throw new IllegalArgumentException(e.getMessage());
 +    } catch (IllegalAccessException e) {
 +      throw new IllegalArgumentException(e.getMessage());
 +    } catch (ClassCastException e) {
 +      StringBuilder msg = new StringBuilder(50);
 +      msg.append(className).append(" loaded successfully but does not implement SortedKeyValueIterator.");
 +      msg.append(" This class cannot be used with this command.");
 +      throw new ShellCommandException(ErrorCode.INITIALIZATION_FAILURE, msg.toString());
 +    }
 +
 +    @SuppressWarnings("unchecked")
 +    SortedKeyValueIterator<Key,Value> skvi = (SortedKeyValueIterator<Key,Value>) untypedInstance;
 +    OptionDescriber iterOptions = null;
 +    if (OptionDescriber.class.isAssignableFrom(skvi.getClass())) {
 +      iterOptions = (OptionDescriber) skvi;
 +    }
 +
 +    String iteratorName;
 +    if (null != iterOptions) {
 +      final IteratorOptions itopts = iterOptions.describeOptions();
 +      iteratorName = itopts.getName();
 +      
 +      if (iteratorName == null) {
 +        throw new IllegalArgumentException(className + " described its default distinguishing name as null");
 +      }
 +      String shortClassName = className;
 +      if (className.contains(".")) {
 +        shortClassName = className.substring(className.lastIndexOf('.') + 1);
 +      }
 +      final Map<String,String> localOptions = new HashMap<String,String>();
 +      do {
 +        // clean up the overall options that caused things to fail
 +        for (String key : localOptions.keySet()) {
 +          options.remove(key);
 +        }
 +        localOptions.clear();
 +  
 +        reader.println(itopts.getDescription());
 +  
 +        String prompt;
 +        if (itopts.getNamedOptions() != null) {
 +          for (Entry<String,String> e : itopts.getNamedOptions().entrySet()) {
 +            prompt = Shell.repeat("-", 10) + "> set " + shortClassName + " parameter " + e.getKey() + ", " + e.getValue() + ": ";
 +            reader.flush();
 +            input = reader.readLine(prompt);
 +            if (input == null) {
 +              reader.println();
 +              throw new IOException("Input stream closed");
 +            }
 +            // Places all Parameters and Values into the LocalOptions, even if the value is "".
 +            // This allows us to check for "" values when setting the iterators and allows us to remove
 +            // the parameter and value from the table property.
 +            localOptions.put(e.getKey(), input);
 +          }
 +        }
 +  
 +        if (itopts.getUnnamedOptionDescriptions() != null) {
 +          for (String desc : itopts.getUnnamedOptionDescriptions()) {
 +            reader.println(Shell.repeat("-", 10) + "> entering options: " + desc);
 +            input = "start";
 +            prompt = Shell.repeat("-", 10) + "> set " + shortClassName + " option (<name> <value>, hit enter to skip): ";
 +            while (true) {
 +              reader.flush();
 +              input = reader.readLine(prompt);
 +              if (input == null) {
 +                reader.println();
 +                throw new IOException("Input stream closed");
 +              } else {
 +                input = new String(input);
 +              }
 +  
 +              if (input.length() == 0)
 +                break;
 +  
 +              String[] sa = input.split(" ", 2);
 +              localOptions.put(sa[0], sa[1]);
 +            }
 +          }
 +        }
 +  
 +        options.putAll(localOptions);
 +        if (!iterOptions.validateOptions(options))
 +          reader.println("invalid options for " + clazz.getName());
 +  
 +      } while (!iterOptions.validateOptions(options));
 +    } else {
 +      reader.flush();
 +      reader.println("The iterator class does not implement OptionDescriber. Consider this for better iterator configuration using this setiter command.");
 +      iteratorName = reader.readLine("Name for iterator (enter to skip): ");
 +      if (null == iteratorName) {
 +        reader.println();
 +        throw new IOException("Input stream closed");
 +      } else if (StringUtils.isWhitespace(iteratorName)) {
 +        // Treat whitespace or empty string as no name provided
 +        iteratorName = null;
 +      }
 +      
 +      reader.flush();
 +      reader.println("Optional, configure name-value options for iterator:");
 +      String prompt = Shell.repeat("-", 10) + "> set option (<name> <value>, hit enter to skip): ";
 +      final HashMap<String,String> localOptions = new HashMap<String,String>();
 +      
 +      while (true) {
 +        reader.flush();
 +        input = reader.readLine(prompt);
 +        if (input == null) {
 +          reader.println();
 +          throw new IOException("Input stream closed");
 +        } else if (StringUtils.isWhitespace(input)) {
 +          break;
 +        } 
 +
 +        String[] sa = input.split(" ", 2);
 +        localOptions.put(sa[0], sa[1]);
 +      }
 +      
 +      options.putAll(localOptions);
 +    }
 +    
 +    return iteratorName;
 +  }
 +
 +  @Override
 +  public String description() {
 +    return "sets a table-specific or namespace-specific iterator";
 +  }
 +
 +  @Override
 +  public Options getOptions() {
 +    final Options o = new Options();
 +
 +    priorityOpt = new Option("p", "priority", true, "the order in which the iterator is applied");
 +    priorityOpt.setArgName("pri");
 +    priorityOpt.setRequired(true);
 +
 +    nameOpt = new Option("n", "name", true, "iterator to set");
 +    nameOpt.setArgName("itername");
 +
 +    allScopeOpt = new Option("all", "all-scopes", false, "applied at scan time, minor and major compactions");
 +    mincScopeOpt = new Option(IteratorScope.minc.name(), "minor-compaction", false, "applied at minor compaction");
 +    majcScopeOpt = new Option(IteratorScope.majc.name(), "major-compaction", false, "applied at major compaction");
 +    scanScopeOpt = new Option(IteratorScope.scan.name(), "scan-time", false, "applied at scan time");
 +
 +    final OptionGroup typeGroup = new OptionGroup();
 +    classnameTypeOpt = new Option("class", "class-name", true, "a java class that implements SortedKeyValueIterator");
 +    classnameTypeOpt.setArgName("name");
 +    aggTypeOpt = new Option("agg", "aggregator", false, "an aggregating type");
 +    regexTypeOpt = new Option("regex", "regular-expression", false, "a regex matching iterator");
 +    versionTypeOpt = new Option("vers", "version", false, "a versioning iterator");
 +    reqvisTypeOpt = new Option("reqvis", "require-visibility", false, "an iterator that omits entries with empty visibilities");
 +    ageoffTypeOpt = new Option("ageoff", "ageoff", false, "an aging off iterator");
 +
 +    typeGroup.addOption(classnameTypeOpt);
 +    typeGroup.addOption(aggTypeOpt);
 +    typeGroup.addOption(regexTypeOpt);
 +    typeGroup.addOption(versionTypeOpt);
 +    typeGroup.addOption(reqvisTypeOpt);
 +    typeGroup.addOption(ageoffTypeOpt);
 +    typeGroup.setRequired(true);
 +
 +    final OptionGroup tableGroup = new OptionGroup();
 +    tableGroup.addOption(OptUtil.tableOpt("table to configure iterators on"));
 +    tableGroup.addOption(OptUtil.namespaceOpt("namespace to configure iterators on"));
 +
 +    o.addOption(priorityOpt);
 +    o.addOption(nameOpt);
 +    o.addOption(allScopeOpt);
 +    o.addOption(mincScopeOpt);
 +    o.addOption(majcScopeOpt);
 +    o.addOption(scanScopeOpt);
 +    o.addOptionGroup(typeGroup);
 +    o.addOptionGroup(tableGroup);
 +    return o;
 +  }
 +
 +  @Override
 +  public int numArgs() {
 +    return 0;
 +  }
 +}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/2533b7ef/shell/src/main/java/org/apache/accumulo/shell/commands/ShellPluginConfigurationCommand.java
----------------------------------------------------------------------
diff --cc shell/src/main/java/org/apache/accumulo/shell/commands/ShellPluginConfigurationCommand.java
index 45d48e7,0000000..8f99eec
mode 100644,000000..100644
--- a/shell/src/main/java/org/apache/accumulo/shell/commands/ShellPluginConfigurationCommand.java
+++ b/shell/src/main/java/org/apache/accumulo/shell/commands/ShellPluginConfigurationCommand.java
@@@ -1,146 -1,0 +1,163 @@@
 +/*
 + * 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.accumulo.shell.commands;
 +
++import java.util.Arrays;
 +import java.util.Iterator;
 +import java.util.Map.Entry;
 +
 +import org.apache.accumulo.core.client.AccumuloException;
 +import org.apache.accumulo.core.client.AccumuloSecurityException;
 +import org.apache.accumulo.core.client.TableNotFoundException;
 +import org.apache.accumulo.core.conf.Property;
 +import org.apache.accumulo.shell.Shell;
 +import org.apache.accumulo.shell.Shell.Command;
- import org.apache.accumulo.start.classloader.vfs.AccumuloVFSClassLoader;
++import org.apache.commons.cli.BasicParser;
 +import org.apache.commons.cli.CommandLine;
 +import org.apache.commons.cli.Option;
 +import org.apache.commons.cli.OptionGroup;
 +import org.apache.commons.cli.Options;
++import org.apache.commons.cli.ParseException;
 +import org.apache.log4j.Logger;
 +
 +public abstract class ShellPluginConfigurationCommand extends Command {
 +  private Option removePluginOption, pluginClassOption, listPluginOption;
 +  
 +  private String pluginType;
 +  
 +  private Property tableProp;
 +  
 +  private String classOpt;
 +  
 +  ShellPluginConfigurationCommand(final String typeName, final Property tableProp, final String classOpt) {
 +    this.pluginType = typeName;
 +    this.tableProp = tableProp;
 +    this.classOpt = classOpt;
 +  }
 +
 +  @Override
 +  public int execute(final String fullCommand, final CommandLine cl, final Shell shellState) throws Exception {
 +    final String tableName = OptUtil.getTableOpt(cl, shellState);
 +    
 +    if (cl.hasOption(removePluginOption.getOpt())) {
 +      // Remove the property
 +      removePlugin(cl, shellState, tableName);
 +      
 +      shellState.getReader().println("Removed "+pluginType+" on " + tableName);
 +    } else if (cl.hasOption(listPluginOption.getOpt())) {
 +      // Get the options for this table
 +      final Iterator<Entry<String,String>> iter = shellState.getConnector().tableOperations().getProperties(tableName).iterator();
 +      
 +      while (iter.hasNext()) {
 +        Entry<String,String> ent = iter.next();
 +        
 +        // List all parameters with the property name
 +        if (ent.getKey().startsWith(tableProp.toString())) {
 +          shellState.getReader().println(ent.getKey() + ": " + ent.getValue());
 +        }
 +      }
 +    } else {
 +      // Set the plugin with the provided options
 +      String className = cl.getOptionValue(pluginClassOption.getOpt());
 +      
 +      // Set the plugin property on the table
 +      setPlugin(cl, shellState, tableName, className);
 +    }
 +    
 +    return 0;
 +  }
 +
 +  protected void setPlugin(final CommandLine cl, final Shell shellState, final String tableName, final String className) throws AccumuloException, AccumuloSecurityException {
 +    shellState.getConnector().tableOperations().setProperty(tableName, tableProp.toString(), className);
 +  }
 +  
 +  protected void removePlugin(final CommandLine cl, final Shell shellState, final String tableName) throws AccumuloException, AccumuloSecurityException {
 +    shellState.getConnector().tableOperations().removeProperty(tableName, tableProp.toString());
 +  }
 +  
 +  public static <T> Class<? extends T> getPluginClass(final String tableName, final Shell shellState, final Class<T> clazz, final Property pluginProp) {
 +    Iterator<Entry<String,String>> props;
 +    try {
 +      props = shellState.getConnector().tableOperations().getProperties(tableName).iterator();
 +    } catch (AccumuloException e) {
 +      return null;
 +    } catch (TableNotFoundException e) {
 +      return null;
 +    }
 +    
 +    while (props.hasNext()) {
 +      final Entry<String,String> ent = props.next();
 +      if (ent.getKey().equals(pluginProp.toString())) {
 +        Class<? extends T> pluginClazz;
++        String[] args = new String[2];
 +        try {
-           pluginClazz = AccumuloVFSClassLoader.loadClass(ent.getValue(), clazz);
++          Options o = new Options();
++          o.addOption(OptUtil.tableOpt());
++          args[0] = "-t";
++          args[1] = tableName;
++          CommandLine cl = new BasicParser().parse(o, args);
++          pluginClazz = shellState.getClassLoader(cl, shellState).loadClass(ent.getValue()).asSubclass(clazz);
 +        } catch (ClassNotFoundException e) {
-           Logger.getLogger(ShellPluginConfigurationCommand.class).warn("Class not found" + e.getMessage());
++          Logger.getLogger(ShellPluginConfigurationCommand.class).error("Class not found" + e.getMessage());
++          return null;
++        } catch (ParseException e) {
++          Logger.getLogger(ShellPluginConfigurationCommand.class).error("Error parsing table: " + Arrays.toString(args) + e.getMessage());
++          return null;
++        } catch (TableNotFoundException e) {
++          Logger.getLogger(ShellPluginConfigurationCommand.class).error("Table not found: " + tableName + e.getMessage());
++          return null;
++        } catch (Exception e) {
++          Logger.getLogger(ShellPluginConfigurationCommand.class).error("Error: " + e.getMessage());
 +          return null;
 +        }
 +        
 +        return pluginClazz;
 +      }
 +    }
 +    
 +    return null;
 +  }
 +  
 +  @Override
 +  public Options getOptions() {
 +    final Options o = new Options();
 +    final OptionGroup actionGroup = new OptionGroup();
 +    
 +    pluginClassOption = new Option(classOpt, pluginType, true, "fully qualified name of the " + pluginType + " class to use");
 +    pluginClassOption.setArgName("className");
 +    
 +    // Action to take: apply (default), remove, list
 +    removePluginOption = new Option("r", "remove", false, "remove the current "+pluginType+"");
 +    listPluginOption = new Option("l", "list", false, "display the current "+pluginType+"");
 +    
 +    actionGroup.addOption(pluginClassOption);
 +    actionGroup.addOption(removePluginOption);
 +    actionGroup.addOption(listPluginOption);
 +    actionGroup.setRequired(true);
 +    
 +    o.addOptionGroup(actionGroup);
 +    o.addOption(OptUtil.tableOpt("table to set the "+pluginType+" on"));
 +    
 +    return o;
 +  }
 +  
 +  @Override
 +  public int numArgs() {
 +    return 0;
 +  }
 +  
 +}


[3/3] git commit: Merge branch '1.6.1-SNAPSHOT'

Posted by dl...@apache.org.
Merge branch '1.6.1-SNAPSHOT'

Conflicts:
	shell/src/main/java/org/apache/accumulo/shell/Shell.java
	shell/src/main/java/org/apache/accumulo/shell/commands/ShellPluginConfigurationCommand.java


Project: http://git-wip-us.apache.org/repos/asf/accumulo/repo
Commit: http://git-wip-us.apache.org/repos/asf/accumulo/commit/2533b7ef
Tree: http://git-wip-us.apache.org/repos/asf/accumulo/tree/2533b7ef
Diff: http://git-wip-us.apache.org/repos/asf/accumulo/diff/2533b7ef

Branch: refs/heads/master
Commit: 2533b7eff8de15504841eae5ca2b7988db709786
Parents: 918e686 7eeb08a
Author: Dave Marion <dl...@hotmail.com>
Authored: Sun Sep 7 22:32:21 2014 -0400
Committer: Dave Marion <dl...@hotmail.com>
Committed: Sun Sep 7 22:32:21 2014 -0400

----------------------------------------------------------------------
 .../java/org/apache/accumulo/shell/Shell.java   | 58 ++++++++++++++++++++
 .../accumulo/shell/commands/ScanCommand.java    |  8 +--
 .../accumulo/shell/commands/SetIterCommand.java | 55 +------------------
 .../ShellPluginConfigurationCommand.java        | 23 +++++++-
 4 files changed, 83 insertions(+), 61 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/2533b7ef/shell/src/main/java/org/apache/accumulo/shell/Shell.java
----------------------------------------------------------------------
diff --cc shell/src/main/java/org/apache/accumulo/shell/Shell.java
index 48d8bd0,0000000..4cb7b53
mode 100644,000000..100644
--- a/shell/src/main/java/org/apache/accumulo/shell/Shell.java
+++ b/shell/src/main/java/org/apache/accumulo/shell/Shell.java
@@@ -1,1168 -1,0 +1,1226 @@@
 +/*
 + * 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.accumulo.shell;
 +
 +import java.io.BufferedWriter;
 +import java.io.File;
 +import java.io.FileNotFoundException;
 +import java.io.FileOutputStream;
 +import java.io.IOException;
 +import java.io.OutputStreamWriter;
 +import java.io.PrintWriter;
 +import java.net.InetAddress;
 +import java.nio.charset.StandardCharsets;
 +import java.util.Arrays;
 +import java.util.Collections;
 +import java.util.HashMap;
 +import java.util.HashSet;
 +import java.util.Iterator;
 +import java.util.List;
 +import java.util.Locale;
 +import java.util.Map;
 +import java.util.Map.Entry;
 +import java.util.Set;
 +import java.util.TreeMap;
 +import java.util.UUID;
 +
 +import jline.console.ConsoleReader;
 +import jline.console.UserInterruptException;
 +import jline.console.history.FileHistory;
 +
 +import org.apache.accumulo.core.Constants;
 +import org.apache.accumulo.core.client.AccumuloException;
 +import org.apache.accumulo.core.client.AccumuloSecurityException;
 +import org.apache.accumulo.core.client.ClientConfiguration;
 +import org.apache.accumulo.core.client.ClientConfiguration.ClientProperty;
 +import org.apache.accumulo.core.client.Connector;
 +import org.apache.accumulo.core.client.Instance;
 +import org.apache.accumulo.core.client.IteratorSetting;
++import org.apache.accumulo.core.client.NamespaceNotFoundException;
 +import org.apache.accumulo.core.client.TableNotFoundException;
 +import org.apache.accumulo.core.client.ZooKeeperInstance;
 +import org.apache.accumulo.core.client.impl.ServerConfigurationUtil;
 +import org.apache.accumulo.core.client.impl.Tables;
 +import org.apache.accumulo.core.client.mock.MockInstance;
 +import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
 +import org.apache.accumulo.core.client.security.tokens.PasswordToken;
 +import org.apache.accumulo.core.conf.AccumuloConfiguration;
 +import org.apache.accumulo.core.conf.DefaultConfiguration;
 +import org.apache.accumulo.core.conf.Property;
 +import org.apache.accumulo.core.conf.SiteConfiguration;
 +import org.apache.accumulo.core.data.Key;
 +import org.apache.accumulo.core.data.Value;
 +import org.apache.accumulo.core.data.thrift.TConstraintViolationSummary;
 +import org.apache.accumulo.core.tabletserver.thrift.ConstraintViolationException;
 +import org.apache.accumulo.core.trace.DistributedTrace;
 +import org.apache.accumulo.core.util.BadArgumentException;
 +import org.apache.accumulo.core.util.format.BinaryFormatter;
 +import org.apache.accumulo.core.util.format.DefaultFormatter;
 +import org.apache.accumulo.core.util.format.Formatter;
 +import org.apache.accumulo.core.util.format.FormatterFactory;
 +import org.apache.accumulo.core.volume.VolumeConfiguration;
 +import org.apache.accumulo.core.zookeeper.ZooUtil;
 +import org.apache.accumulo.fate.zookeeper.ZooReader;
++import org.apache.accumulo.start.classloader.vfs.AccumuloVFSClassLoader;
++import org.apache.accumulo.start.classloader.vfs.ContextManager;
 +import org.apache.accumulo.shell.commands.AboutCommand;
 +import org.apache.accumulo.shell.commands.AddAuthsCommand;
 +import org.apache.accumulo.shell.commands.AddSplitsCommand;
 +import org.apache.accumulo.shell.commands.AuthenticateCommand;
 +import org.apache.accumulo.shell.commands.ByeCommand;
 +import org.apache.accumulo.shell.commands.ClasspathCommand;
 +import org.apache.accumulo.shell.commands.ClearCommand;
 +import org.apache.accumulo.shell.commands.CloneTableCommand;
 +import org.apache.accumulo.shell.commands.ClsCommand;
 +import org.apache.accumulo.shell.commands.CompactCommand;
 +import org.apache.accumulo.shell.commands.ConfigCommand;
 +import org.apache.accumulo.shell.commands.ConstraintCommand;
 +import org.apache.accumulo.shell.commands.CreateNamespaceCommand;
 +import org.apache.accumulo.shell.commands.CreateTableCommand;
 +import org.apache.accumulo.shell.commands.CreateUserCommand;
 +import org.apache.accumulo.shell.commands.DUCommand;
 +import org.apache.accumulo.shell.commands.DebugCommand;
 +import org.apache.accumulo.shell.commands.DeleteCommand;
 +import org.apache.accumulo.shell.commands.DeleteIterCommand;
 +import org.apache.accumulo.shell.commands.DeleteManyCommand;
 +import org.apache.accumulo.shell.commands.DeleteNamespaceCommand;
 +import org.apache.accumulo.shell.commands.DeleteRowsCommand;
 +import org.apache.accumulo.shell.commands.DeleteScanIterCommand;
 +import org.apache.accumulo.shell.commands.DeleteShellIterCommand;
 +import org.apache.accumulo.shell.commands.DeleteTableCommand;
 +import org.apache.accumulo.shell.commands.DeleteUserCommand;
 +import org.apache.accumulo.shell.commands.DropTableCommand;
 +import org.apache.accumulo.shell.commands.DropUserCommand;
 +import org.apache.accumulo.shell.commands.EGrepCommand;
 +import org.apache.accumulo.shell.commands.ExecfileCommand;
 +import org.apache.accumulo.shell.commands.ExitCommand;
 +import org.apache.accumulo.shell.commands.ExportTableCommand;
 +import org.apache.accumulo.shell.commands.ExtensionCommand;
 +import org.apache.accumulo.shell.commands.FateCommand;
 +import org.apache.accumulo.shell.commands.FlushCommand;
 +import org.apache.accumulo.shell.commands.FormatterCommand;
 +import org.apache.accumulo.shell.commands.GetAuthsCommand;
 +import org.apache.accumulo.shell.commands.GetGroupsCommand;
 +import org.apache.accumulo.shell.commands.GetSplitsCommand;
 +import org.apache.accumulo.shell.commands.GrantCommand;
 +import org.apache.accumulo.shell.commands.GrepCommand;
 +import org.apache.accumulo.shell.commands.HelpCommand;
 +import org.apache.accumulo.shell.commands.HiddenCommand;
 +import org.apache.accumulo.shell.commands.HistoryCommand;
 +import org.apache.accumulo.shell.commands.ImportDirectoryCommand;
 +import org.apache.accumulo.shell.commands.ImportTableCommand;
 +import org.apache.accumulo.shell.commands.InfoCommand;
 +import org.apache.accumulo.shell.commands.InsertCommand;
 +import org.apache.accumulo.shell.commands.InterpreterCommand;
 +import org.apache.accumulo.shell.commands.ListCompactionsCommand;
 +import org.apache.accumulo.shell.commands.ListIterCommand;
 +import org.apache.accumulo.shell.commands.ListScansCommand;
 +import org.apache.accumulo.shell.commands.ListShellIterCommand;
 +import org.apache.accumulo.shell.commands.MaxRowCommand;
 +import org.apache.accumulo.shell.commands.MergeCommand;
 +import org.apache.accumulo.shell.commands.NamespacePermissionsCommand;
 +import org.apache.accumulo.shell.commands.NamespacesCommand;
 +import org.apache.accumulo.shell.commands.NoTableCommand;
 +import org.apache.accumulo.shell.commands.OfflineCommand;
 +import org.apache.accumulo.shell.commands.OnlineCommand;
++import org.apache.accumulo.shell.commands.OptUtil;
 +import org.apache.accumulo.shell.commands.PasswdCommand;
 +import org.apache.accumulo.shell.commands.PingCommand;
 +import org.apache.accumulo.shell.commands.QuestionCommand;
 +import org.apache.accumulo.shell.commands.QuitCommand;
 +import org.apache.accumulo.shell.commands.QuotedStringTokenizer;
 +import org.apache.accumulo.shell.commands.RenameNamespaceCommand;
 +import org.apache.accumulo.shell.commands.RenameTableCommand;
 +import org.apache.accumulo.shell.commands.RevokeCommand;
 +import org.apache.accumulo.shell.commands.ScanCommand;
 +import org.apache.accumulo.shell.commands.ScriptCommand;
 +import org.apache.accumulo.shell.commands.SetAuthsCommand;
 +import org.apache.accumulo.shell.commands.SetGroupsCommand;
 +import org.apache.accumulo.shell.commands.SetIterCommand;
 +import org.apache.accumulo.shell.commands.SetScanIterCommand;
 +import org.apache.accumulo.shell.commands.SetShellIterCommand;
 +import org.apache.accumulo.shell.commands.SleepCommand;
 +import org.apache.accumulo.shell.commands.SystemPermissionsCommand;
 +import org.apache.accumulo.shell.commands.TableCommand;
 +import org.apache.accumulo.shell.commands.TablePermissionsCommand;
 +import org.apache.accumulo.shell.commands.TablesCommand;
 +import org.apache.accumulo.shell.commands.TraceCommand;
 +import org.apache.accumulo.shell.commands.UserCommand;
 +import org.apache.accumulo.shell.commands.UserPermissionsCommand;
 +import org.apache.accumulo.shell.commands.UsersCommand;
 +import org.apache.accumulo.shell.commands.WhoAmICommand;
 +import org.apache.commons.cli.BasicParser;
 +import org.apache.commons.cli.CommandLine;
 +import org.apache.commons.cli.HelpFormatter;
 +import org.apache.commons.cli.MissingOptionException;
 +import org.apache.commons.cli.Option;
 +import org.apache.commons.cli.Options;
 +import org.apache.commons.cli.ParseException;
++import org.apache.commons.vfs2.FileSystemException;
 +import org.apache.hadoop.fs.Path;
 +import org.apache.log4j.Level;
 +import org.apache.log4j.Logger;
 +
 +import com.beust.jcommander.JCommander;
 +import com.beust.jcommander.ParameterException;
 +
 +/**
 + * A convenient console interface to perform basic accumulo functions Includes auto-complete, help, and quoted strings with escape sequences
 + */
 +public class Shell extends ShellOptions {
 +  public static final Logger log = Logger.getLogger(Shell.class);
 +  private static final Logger audit = Logger.getLogger(Shell.class.getName() + ".audit");
 +
 +  public static final String CHARSET = "ISO-8859-1";
 +  public static final int NO_FIXED_ARG_LENGTH_CHECK = -1;
 +  public static final String COMMENT_PREFIX = "#";
 +  public static final String HISTORY_DIR_NAME = ".accumulo";
 +  public static final String HISTORY_FILE_NAME = "shell_history.txt";
 +  private static final String SHELL_DESCRIPTION = "Shell - Apache Accumulo Interactive Shell";
 +
 +  protected int exitCode = 0;
 +  private String tableName;
 +  protected Instance instance;
 +  private Connector connector;
 +  protected ConsoleReader reader;
 +  private String principal;
 +  private AuthenticationToken token;
 +  private final Class<? extends Formatter> defaultFormatterClass = DefaultFormatter.class;
 +  private final Class<? extends Formatter> binaryFormatterClass = BinaryFormatter.class;
 +  public Map<String,List<IteratorSetting>> scanIteratorOptions = new HashMap<String,List<IteratorSetting>>();
 +  public Map<String,List<IteratorSetting>> iteratorProfiles = new HashMap<String,List<IteratorSetting>>();
 +
 +  private Token rootToken;
 +  public final Map<String,Command> commandFactory = new TreeMap<String,Command>();
 +  public final Map<String,Command[]> commandGrouping = new TreeMap<String,Command[]>();
 +  protected boolean configError = false;
 +
 +  // exit if true
 +  private boolean exit = false;
 +
 +  // file to execute commands from
 +  protected File execFile = null;
 +  // single command to execute from the command line
 +  protected String execCommand = null;
 +  protected boolean verbose = true;
 +
 +  private boolean tabCompletion;
 +  private boolean disableAuthTimeout;
 +  private long authTimeout;
 +  private long lastUserActivity = System.currentTimeMillis();
 +  private boolean logErrorsToConsole = false;
 +  private PrintWriter writer = null;
 +  private boolean masking = false;
 +
 +  public Shell() throws IOException {
 +    this(new ConsoleReader(), new PrintWriter(
 +        new OutputStreamWriter(System.out,
 +        System.getProperty("jline.WindowsTerminal.output.encoding", System.getProperty("file.encoding")))));
 +  }
 +
 +  public Shell(ConsoleReader reader, PrintWriter writer) {
 +    super();
 +    this.reader = reader;
 +    this.writer = writer;
 +  }
 +
 +  // Not for client use
 +  public boolean config(String... args) {
 +    ShellOptionsJC options = new ShellOptionsJC();
 +    JCommander jc = new JCommander();
 +
 +    jc.setProgramName("accumulo shell");
 +    jc.addObject(options);
 +    try {
 +      jc.parse(args);
 +    } catch (ParameterException e) {
 +      configError = true;
 +    }
 +
 +    if (options.isHelpEnabled()) {
 +      configError = true;
 +    }
 +
 +    if (!configError && options.getUnrecognizedOptions() != null) {
 +      configError = true;
 +      logError("Unrecognized Options: " + options.getUnrecognizedOptions().toString());
 +    }
 +
 +    if (configError) {
 +      jc.usage();
 +      return true;
 +    }
 +
 +    setDebugging(options.isDebugEnabled());
 +    authTimeout = options.getAuthTimeout() * 60 * 1000; // convert minutes to milliseconds
 +    disableAuthTimeout = options.isAuthTimeoutDisabled();
 +
 +    // get the options that were parsed
 +    String user = options.getUsername();
 +    String password = options.getPassword();
 +
 +    tabCompletion = !options.isTabCompletionDisabled();
 +
 +    // Use a fake (Mock), ZK, or HdfsZK Accumulo instance
 +    setInstance(options);
 +
 +    // AuthenticationToken options
 +    token = options.getAuthenticationToken();
 +    Map<String,String> loginOptions = options.getTokenProperties();
 +
 +    // process default parameters if unspecified
 +    try {
 +      boolean hasToken = (token != null);
 +      boolean hasTokenOptions = !loginOptions.isEmpty();
 +
 +      if (hasToken && password != null) {
 +        throw new ParameterException("Can not supply '--pass' option with '--tokenClass' option");
 +      }
 +
 +      Runtime.getRuntime().addShutdownHook(new Thread() {
 +        @Override
 +        public void run() {
 +          reader.getTerminal().setEchoEnabled(true);
 +        }
 +      });
 +
 +      // Need either both a token and options, or neither, but not just one.
 +      if (hasToken != hasTokenOptions) {
 +        throw new ParameterException("Must supply either both or neither of '--tokenClass' and '--tokenProperty'");
 +      } else if (hasToken) { // implied hasTokenOptions
 +        // Fully qualified name so we don't shadow java.util.Properties
 +        org.apache.accumulo.core.client.security.tokens.AuthenticationToken.Properties props;
 +        // and line wrap it because the package name is so long
 +        props = new org.apache.accumulo.core.client.security.tokens.AuthenticationToken.Properties();
 +
 +        props.putAllStrings(loginOptions);
 +        token.init(props);
 +      } else {
 +        // Read password if the user explicitly asked for it, or didn't specify anything at all
 +        if ("stdin".equals(password) || password == null) {
 +          password = reader.readLine("Password: ", '*');
 +        }
 +
 +        if (password == null) {
 +          // User cancel, e.g. Ctrl-D pressed
 +          throw new ParameterException("No password or token option supplied");
 +        } else {
 +          this.token = new PasswordToken(password);
 +        }
 +      }
 +
 +      if (!options.isFake()) {
 +        ZooReader zr = new ZooReader(instance.getZooKeepers(), instance.getZooKeepersSessionTimeOut());
 +        DistributedTrace.enable(instance, zr, "shell", InetAddress.getLocalHost().getHostName());
 +      }
 +
 +      this.setTableName("");
 +      this.principal = user;
 +      connector = instance.getConnector(this.principal, token);
 +
 +    } catch (Exception e) {
 +      printException(e);
 +      configError = true;
 +    }
 +
 +    // decide whether to execute commands from a file and quit
 +    if (options.getExecFile() != null) {
 +      execFile = options.getExecFile();
 +      verbose = false;
 +    } else if (options.getExecFileVerbose() != null) {
 +      execFile = options.getExecFileVerbose();
 +      verbose = true;
 +    }
 +    execCommand = options.getExecCommand();
 +    if (execCommand != null) {
 +      verbose = false;
 +    }
 +
 +    rootToken = new Token();
 +
 +    Command[] dataCommands = {new DeleteCommand(), new DeleteManyCommand(), new DeleteRowsCommand(), new EGrepCommand(), new FormatterCommand(),
 +        new InterpreterCommand(), new GrepCommand(), new ImportDirectoryCommand(), new InsertCommand(), new MaxRowCommand(), new ScanCommand()};
 +    Command[] debuggingCommands = {new ClasspathCommand(), new DebugCommand(), new ListScansCommand(), new ListCompactionsCommand(), new TraceCommand(),
 +        new PingCommand()};
 +    Command[] execCommands = {new ExecfileCommand(), new HistoryCommand(), new ExtensionCommand(), new ScriptCommand()};
 +    Command[] exitCommands = {new ByeCommand(), new ExitCommand(), new QuitCommand()};
 +    Command[] helpCommands = {new AboutCommand(), new HelpCommand(), new InfoCommand(), new QuestionCommand()};
 +    Command[] iteratorCommands = {new DeleteIterCommand(), new DeleteScanIterCommand(), new ListIterCommand(), new SetIterCommand(), new SetScanIterCommand(),
 +        new SetShellIterCommand(), new ListShellIterCommand(), new DeleteShellIterCommand()};
 +    Command[] otherCommands = {new HiddenCommand()};
 +    Command[] permissionsCommands = {new GrantCommand(), new RevokeCommand(), new SystemPermissionsCommand(), new TablePermissionsCommand(),
 +        new UserPermissionsCommand(), new NamespacePermissionsCommand()};
 +    Command[] stateCommands = {new AuthenticateCommand(), new ClsCommand(), new ClearCommand(), new FateCommand(), new NoTableCommand(), new SleepCommand(),
 +        new TableCommand(), new UserCommand(), new WhoAmICommand()};
 +    Command[] tableCommands = {new CloneTableCommand(), new ConfigCommand(), new CreateTableCommand(), new DeleteTableCommand(), new DropTableCommand(),
 +        new DUCommand(), new ExportTableCommand(), new ImportTableCommand(), new OfflineCommand(), new OnlineCommand(), new RenameTableCommand(),
 +        new TablesCommand(), new NamespacesCommand(), new CreateNamespaceCommand(), new DeleteNamespaceCommand(), new RenameNamespaceCommand()};
 +    Command[] tableControlCommands = {new AddSplitsCommand(), new CompactCommand(), new ConstraintCommand(), new FlushCommand(), new GetGroupsCommand(),
 +        new GetSplitsCommand(), new MergeCommand(), new SetGroupsCommand()};
 +    Command[] userCommands = {new AddAuthsCommand(), new CreateUserCommand(), new DeleteUserCommand(), new DropUserCommand(), new GetAuthsCommand(),
 +        new PasswdCommand(), new SetAuthsCommand(), new UsersCommand()};
 +    commandGrouping.put("-- Writing, Reading, and Removing Data --", dataCommands);
 +    commandGrouping.put("-- Debugging Commands -------------------", debuggingCommands);
 +    commandGrouping.put("-- Shell Execution Commands -------------", execCommands);
 +    commandGrouping.put("-- Exiting Commands ---------------------", exitCommands);
 +    commandGrouping.put("-- Help Commands ------------------------", helpCommands);
 +    commandGrouping.put("-- Iterator Configuration ---------------", iteratorCommands);
 +    commandGrouping.put("-- Permissions Administration Commands --", permissionsCommands);
 +    commandGrouping.put("-- Shell State Commands -----------------", stateCommands);
 +    commandGrouping.put("-- Table Administration Commands --------", tableCommands);
 +    commandGrouping.put("-- Table Control Commands ---------------", tableControlCommands);
 +    commandGrouping.put("-- User Administration Commands ---------", userCommands);
 +
 +    for (Command[] cmds : commandGrouping.values()) {
 +      for (Command cmd : cmds)
 +        commandFactory.put(cmd.getName(), cmd);
 +    }
 +    for (Command cmd : otherCommands) {
 +      commandFactory.put(cmd.getName(), cmd);
 +    }
 +    return configError;
 +  }
 +
 +  /**
 +   * Sets the instance used by the shell based on the given options.
 +   * 
 +   * @param options
 +   *          shell options
 +   */
 +  protected void setInstance(ShellOptionsJC options) {
 +    // should only be one set of instance options set
 +    instance = null;
 +    if (options.isFake()) {
 +      instance = new MockInstance("fake");
 +    } else {
 +      String instanceName, hosts;
 +      if (options.isHdfsZooInstance()) {
 +        instanceName = hosts = null;
 +      } else if (options.getZooKeeperInstance().size() > 0) {
 +        List<String> zkOpts = options.getZooKeeperInstance();
 +        instanceName = zkOpts.get(0);
 +        hosts = zkOpts.get(1);
 +      } else {
 +        instanceName = options.getZooKeeperInstanceName();
 +        hosts = options.getZooKeeperHosts();
 +      }
 +      try {
 +        instance = getZooInstance(instanceName, hosts, options.getClientConfiguration());
 +      } catch (Exception e) {
 +        throw new IllegalArgumentException("Unable to load client config from " + options.getClientConfigFile(), e);
 +      }
 +    }
 +  }
 +
 +  /*
 +   * Takes instanceName and keepers as separate arguments, rather than just packaged into the clientConfig, so that we can fail over to accumulo-site.xml or
 +   * HDFS config if they're unspecified.
 +   */
 +  private static Instance getZooInstance(String instanceName, String keepers, ClientConfiguration clientConfig) {
 +    UUID instanceId = null;
 +    if (instanceName == null) {
 +      instanceName = clientConfig.get(ClientProperty.INSTANCE_NAME);
 +    }
 +    if (instanceName == null || keepers == null) {
 +      AccumuloConfiguration conf = SiteConfiguration.getInstance(ServerConfigurationUtil.convertClientConfig(DefaultConfiguration.getInstance(), clientConfig));
 +      if (instanceName == null) {
 +        Path instanceDir = new Path(VolumeConfiguration.getVolumeUris(conf)[0], "instance_id");
 +        instanceId = UUID.fromString(ZooUtil.getInstanceIDFromHdfs(instanceDir, conf));
 +      }
 +      if (keepers == null) {
 +        keepers = conf.get(Property.INSTANCE_ZK_HOST);
 +      }
 +    }
 +    if (instanceId != null) {
 +      return new ZooKeeperInstance(clientConfig.withInstance(instanceId).withZkHosts(keepers));
 +    } else {
 +      return new ZooKeeperInstance(clientConfig.withInstance(instanceName).withZkHosts(keepers));
 +    }
 +  }
 +
 +  public Connector getConnector() {
 +    return connector;
 +  }
 +
 +  public Instance getInstance() {
 +    return instance;
 +  }
 +
++  public ClassLoader getClassLoader(final CommandLine cl, final Shell shellState) throws AccumuloException, TableNotFoundException, AccumuloSecurityException,
++      IOException, FileSystemException {
++
++    boolean tables = cl.hasOption(OptUtil.tableOpt().getOpt()) || !shellState.getTableName().isEmpty();
++    boolean namespaces = cl.hasOption(OptUtil.namespaceOpt().getOpt());
++
++    String classpath = null;
++    Iterable<Entry<String,String>> tableProps;
++
++    if (namespaces) {
++      try {
++        tableProps = shellState.getConnector().namespaceOperations().getProperties(OptUtil.getNamespaceOpt(cl, shellState));
++      } catch (NamespaceNotFoundException e) {
++        throw new IllegalArgumentException(e);
++      }
++    } else if (tables) {
++      tableProps = shellState.getConnector().tableOperations().getProperties(OptUtil.getTableOpt(cl, shellState));
++    } else {
++      throw new IllegalArgumentException("No table or namespace specified");
++    }
++    for (Entry<String,String> entry : tableProps) {
++      if (entry.getKey().equals(Property.TABLE_CLASSPATH.getKey())) {
++        classpath = entry.getValue();
++      }
++    }
++
++    ClassLoader classloader;
++
++    if (classpath != null && !classpath.equals("")) {
++      shellState.getConnector().instanceOperations().getSystemConfiguration().get(Property.VFS_CONTEXT_CLASSPATH_PROPERTY.getKey() + classpath);
++
++      try {
++        AccumuloVFSClassLoader.getContextManager().setContextConfig(new ContextManager.DefaultContextsConfig(new Iterable<Map.Entry<String,String>>() {
++          @Override
++          public Iterator<Entry<String,String>> iterator() {
++            try {
++              return shellState.getConnector().instanceOperations().getSystemConfiguration().entrySet().iterator();
++            } catch (AccumuloException e) {
++              throw new RuntimeException(e);
++            } catch (AccumuloSecurityException e) {
++              throw new RuntimeException(e);
++            }
++          }
++        }));
++      } catch (IllegalStateException ise) {}
++
++      classloader = AccumuloVFSClassLoader.getContextManager().getClassLoader(classpath);
++    } else {
++      classloader = AccumuloVFSClassLoader.getClassLoader();
++    }
++    return classloader;
++  }
++
 +  public static void main(String args[]) throws IOException {
 +    Shell shell = new Shell();
 +    try {
 +      shell.config(args);
 +
 +      System.exit(shell.start());
 +    } finally {
 +      shell.shutdown();
 +    }
 +  }
 +
 +  public int start() throws IOException {
 +    if (configError)
 +      return 1;
 +
 +    String input;
 +    if (isVerbose())
 +      printInfo();
 +
 +    String home = System.getProperty("HOME");
 +    if (home == null)
 +      home = System.getenv("HOME");
 +    String configDir = home + "/" + HISTORY_DIR_NAME;
 +    String historyPath = configDir + "/" + HISTORY_FILE_NAME;
 +    File accumuloDir = new File(configDir);
 +    if (!accumuloDir.exists() && !accumuloDir.mkdirs())
 +      log.warn("Unable to make directory for history at " + accumuloDir);
 +    try {
 +      final FileHistory history = new FileHistory(new File(historyPath));
 +      reader.setHistory(history);
 +      // Add shutdown hook to flush file history, per jline javadocs
 +      Runtime.getRuntime().addShutdownHook(new Thread() {
 +        @Override
 +        public void run() {
 +          try {
 +            history.flush();
 +          } catch (IOException e) {
 +            log.warn("Could not flush history to file.");
 +          }
 +        }
 +      });
 +    } catch (IOException e) {
 +      log.warn("Unable to load history file at " + historyPath);
 +    }
 +
 +    // Turn Ctrl+C into Exception instead of JVM exit
 +    reader.setHandleUserInterrupt(true);
 +
 +    ShellCompletor userCompletor = null;
 +
 +    if (execFile != null) {
 +      java.util.Scanner scanner = new java.util.Scanner(execFile, StandardCharsets.UTF_8.name());
 +      try {
 +        while (scanner.hasNextLine() && !hasExited()) {
 +          execCommand(scanner.nextLine(), true, isVerbose());
 +        }
 +      } finally {
 +        scanner.close();
 +      }
 +    } else if (execCommand != null) {
 +      for (String command : execCommand.split("\n")) {
 +        execCommand(command, true, isVerbose());
 +      }
 +      return exitCode;
 +    }
 +
 +    while (true) {
 +      try {
 +        if (hasExited())
 +          return exitCode;
 +
 +        // If tab completion is true we need to reset
 +        if (tabCompletion) {
 +          if (userCompletor != null)
 +            reader.removeCompleter(userCompletor);
 +
 +          userCompletor = setupCompletion();
 +          reader.addCompleter(userCompletor);
 +        }
 +
 +        reader.setPrompt(getDefaultPrompt());
 +        input = reader.readLine();
 +        if (input == null) {
 +          reader.println();
 +          return exitCode;
 +        } // User Canceled (Ctrl+D)
 +
 +        execCommand(input, disableAuthTimeout, false);
 +      } catch (UserInterruptException uie) {
 +        // User Cancelled (Ctrl+C)
 +        reader.println();
 +
 +        String partialLine = uie.getPartialLine();
 +        if (partialLine == null || "".equals(uie.getPartialLine().trim())) {
 +          // No content, actually exit
 +          return exitCode;
 +        }
 +      } finally {
 +        reader.flush();
 +      }
 +    }
 +  }
 +
 +  public void shutdown() {
 +    if (reader != null) {
 +      reader.shutdown();
 +    }
 +  }
 +
 +  public void printInfo() throws IOException {
 +    reader.print("\n" + SHELL_DESCRIPTION + "\n" + "- \n" + "- version: " + Constants.VERSION + "\n" + "- instance name: "
 +        + connector.getInstance().getInstanceName() + "\n" + "- instance id: " + connector.getInstance().getInstanceID() + "\n" + "- \n"
 +        + "- type 'help' for a list of available commands\n" + "- \n");
 +    reader.flush();
 +  }
 +
 +  public void printVerboseInfo() throws IOException {
 +    StringBuilder sb = new StringBuilder("-\n");
 +    sb.append("- Current user: ").append(connector.whoami()).append("\n");
 +    if (execFile != null)
 +      sb.append("- Executing commands from: ").append(execFile).append("\n");
 +    if (disableAuthTimeout)
 +      sb.append("- Authorization timeout: disabled\n");
 +    else
 +      sb.append("- Authorization timeout: ").append(String.format("%.2fs%n", authTimeout / 1000.0));
 +    sb.append("- Debug: ").append(isDebuggingEnabled() ? "on" : "off").append("\n");
 +    if (!scanIteratorOptions.isEmpty()) {
 +      for (Entry<String,List<IteratorSetting>> entry : scanIteratorOptions.entrySet()) {
 +        sb.append("- Session scan iterators for table ").append(entry.getKey()).append(":\n");
 +        for (IteratorSetting setting : entry.getValue()) {
 +          sb.append("-    Iterator ").append(setting.getName()).append(" options:\n");
 +          sb.append("-        ").append("iteratorPriority").append(" = ").append(setting.getPriority()).append("\n");
 +          sb.append("-        ").append("iteratorClassName").append(" = ").append(setting.getIteratorClass()).append("\n");
 +          for (Entry<String,String> optEntry : setting.getOptions().entrySet()) {
 +            sb.append("-        ").append(optEntry.getKey()).append(" = ").append(optEntry.getValue()).append("\n");
 +          }
 +        }
 +      }
 +    }
 +    sb.append("-\n");
 +    reader.print(sb.toString());
 +  }
 +
 +  public String getDefaultPrompt() {
 +    return connector.whoami() + "@" + connector.getInstance().getInstanceName() + (getTableName().isEmpty() ? "" : " ") + getTableName() + "> ";
 +  }
 +
 +  public void execCommand(String input, boolean ignoreAuthTimeout, boolean echoPrompt) throws IOException {
 +    audit.log(Level.INFO, getDefaultPrompt() + input);
 +    if (echoPrompt) {
 +      reader.print(getDefaultPrompt());
 +      reader.println(input);
 +    }
 +
 +    if (input.startsWith(COMMENT_PREFIX)) {
 +      return;
 +    }
 +
 +    String fields[];
 +    try {
 +      fields = new QuotedStringTokenizer(input).getTokens();
 +    } catch (BadArgumentException e) {
 +      printException(e);
 +      ++exitCode;
 +      return;
 +    }
 +    if (fields.length == 0)
 +      return;
 +
 +    String command = fields[0];
 +    fields = fields.length > 1 ? Arrays.copyOfRange(fields, 1, fields.length) : new String[] {};
 +
 +    Command sc = null;
 +    if (command.length() > 0) {
 +      try {
 +        // Obtain the command from the command table
 +        sc = commandFactory.get(command);
 +        if (sc == null) {
 +          reader.println(String.format("Unknown command \"%s\".  Enter \"help\" for a list possible commands.", command));
 +          reader.flush();
 +          return;
 +        }
 +
 +        if (!(sc instanceof ExitCommand) && !ignoreAuthTimeout && System.currentTimeMillis() - lastUserActivity > authTimeout) {
 +          reader.println("Shell has been idle for too long. Please re-authenticate.");
 +          boolean authFailed = true;
 +          do {
 +            String pwd = readMaskedLine("Enter current password for '" + connector.whoami() + "': ", '*');
 +            if (pwd == null) {
 +              reader.println();
 +              return;
 +            } // user canceled
 +
 +            try {
 +              authFailed = !connector.securityOperations().authenticateUser(connector.whoami(), new PasswordToken(pwd));
 +            } catch (Exception e) {
 +              ++exitCode;
 +              printException(e);
 +            }
 +
 +            if (authFailed)
 +              reader.print("Invalid password. ");
 +          } while (authFailed);
 +          lastUserActivity = System.currentTimeMillis();
 +        }
 +
 +        // Get the options from the command on how to parse the string
 +        Options parseOpts = sc.getOptionsWithHelp();
 +
 +        // Parse the string using the given options
 +        CommandLine cl = new BasicParser().parse(parseOpts, fields);
 +
 +        int actualArgLen = cl.getArgs().length;
 +        int expectedArgLen = sc.numArgs();
 +        if (cl.hasOption(helpOption)) {
 +          // Display help if asked to; otherwise execute the command
 +          sc.printHelp(this);
 +        } else if (expectedArgLen != NO_FIXED_ARG_LENGTH_CHECK && actualArgLen != expectedArgLen) {
 +          ++exitCode;
 +          // Check for valid number of fixed arguments (if not
 +          // negative; negative means it is not checked, for
 +          // vararg-like commands)
 +          printException(new IllegalArgumentException(String.format("Expected %d argument%s. There %s %d.", expectedArgLen, expectedArgLen == 1 ? "" : "s",
 +              actualArgLen == 1 ? "was" : "were", actualArgLen)));
 +          sc.printHelp(this);
 +        } else {
 +          int tmpCode = sc.execute(input, cl, this);
 +          exitCode += tmpCode;
 +          reader.flush();
 +        }
 +
 +      } catch (ConstraintViolationException e) {
 +        ++exitCode;
 +        printConstraintViolationException(e);
 +      } catch (TableNotFoundException e) {
 +        ++exitCode;
 +        if (getTableName().equals(e.getTableName()))
 +          setTableName("");
 +        printException(e);
 +      } catch (ParseException e) {
 +        // not really an error if the exception is a missing required
 +        // option when the user is asking for help
 +        if (!(e instanceof MissingOptionException && (Arrays.asList(fields).contains("-" + helpOption) || Arrays.asList(fields).contains("--" + helpLongOption)))) {
 +          ++exitCode;
 +          printException(e);
 +        }
 +        if (sc != null)
 +          sc.printHelp(this);
 +      } catch (UserInterruptException e) {
 +        ++exitCode;
 +      } catch (Exception e) {
 +        ++exitCode;
 +        printException(e);
 +      }
 +    } else {
 +      ++exitCode;
 +      printException(new BadArgumentException("Unrecognized empty command", command, -1));
 +    }
 +    reader.flush();
 +  }
 +
 +  /**
 +   * The command tree is built in reverse so that the references are more easily linked up. There is some code in token to allow forward building of the command
 +   * tree.
 +   */
 +  private ShellCompletor setupCompletion() {
 +    rootToken = new Token();
 +
 +    Set<String> tableNames = null;
 +    try {
 +      tableNames = connector.tableOperations().list();
 +    } catch (Exception e) {
 +      log.debug("Unable to obtain list of tables", e);
 +      tableNames = Collections.emptySet();
 +    }
 +
 +    Set<String> userlist = null;
 +    try {
 +      userlist = connector.securityOperations().listLocalUsers();
 +    } catch (Exception e) {
 +      log.debug("Unable to obtain list of users", e);
 +      userlist = Collections.emptySet();
 +    }
 +
 +    Set<String> namespaces = null;
 +    try {
 +      namespaces = connector.namespaceOperations().list();
 +    } catch (Exception e) {
 +      log.debug("Unable to obtain list of namespaces", e);
 +      namespaces = Collections.emptySet();
 +    }
 +
 +    Map<Command.CompletionSet,Set<String>> options = new HashMap<Command.CompletionSet,Set<String>>();
 +
 +    Set<String> commands = new HashSet<String>();
 +    for (String a : commandFactory.keySet())
 +      commands.add(a);
 +
 +    Set<String> modifiedUserlist = new HashSet<String>();
 +    Set<String> modifiedTablenames = new HashSet<String>();
 +    Set<String> modifiedNamespaces = new HashSet<String>();
 +
 +    for (String a : tableNames)
 +      modifiedTablenames.add(a.replaceAll("([\\s'\"])", "\\\\$1"));
 +    for (String a : userlist)
 +      modifiedUserlist.add(a.replaceAll("([\\s'\"])", "\\\\$1"));
 +    for (String a : namespaces) {
 +      String b = a.replaceAll("([\\s'\"])", "\\\\$1");
 +      modifiedNamespaces.add(b.isEmpty() ? "\"\"" : b);
 +    }
 +
 +    options.put(Command.CompletionSet.USERNAMES, modifiedUserlist);
 +    options.put(Command.CompletionSet.TABLENAMES, modifiedTablenames);
 +    options.put(Command.CompletionSet.NAMESPACES, modifiedNamespaces);
 +    options.put(Command.CompletionSet.COMMANDS, commands);
 +
 +    for (Command[] cmdGroup : commandGrouping.values()) {
 +      for (Command c : cmdGroup) {
 +        c.getOptionsWithHelp(); // prep the options for the command
 +        // so that the completion can
 +        // include them
 +        c.registerCompletion(rootToken, options);
 +      }
 +    }
 +    return new ShellCompletor(rootToken, options);
 +  }
 +
 +  /**
 +   * The Command class represents a command to be run in the shell. It contains the methods to execute along with some methods to help tab completion, and
 +   * return the command name, help, and usage.
 +   */
 +  public static abstract class Command {
 +    // Helper methods for completion
 +    public enum CompletionSet {
 +      TABLENAMES, USERNAMES, COMMANDS, NAMESPACES
 +    }
 +
 +    static Set<String> getCommandNames(Map<CompletionSet,Set<String>> objects) {
 +      return objects.get(CompletionSet.COMMANDS);
 +    }
 +
 +    static Set<String> getTableNames(Map<CompletionSet,Set<String>> objects) {
 +      return objects.get(CompletionSet.TABLENAMES);
 +    }
 +
 +    static Set<String> getUserNames(Map<CompletionSet,Set<String>> objects) {
 +      return objects.get(CompletionSet.USERNAMES);
 +    }
 +
 +    static Set<String> getNamespaces(Map<CompletionSet,Set<String>> objects) {
 +      return objects.get(CompletionSet.NAMESPACES);
 +    }
 +
 +    public void registerCompletionGeneral(Token root, Set<String> args, boolean caseSens) {
 +      Token t = new Token(args);
 +      t.setCaseSensitive(caseSens);
 +
 +      Token command = new Token(getName());
 +      command.addSubcommand(t);
 +
 +      root.addSubcommand(command);
 +    }
 +
 +    public void registerCompletionForTables(Token root, Map<CompletionSet,Set<String>> completionSet) {
 +      registerCompletionGeneral(root, completionSet.get(CompletionSet.TABLENAMES), true);
 +    }
 +
 +    public void registerCompletionForUsers(Token root, Map<CompletionSet,Set<String>> completionSet) {
 +      registerCompletionGeneral(root, completionSet.get(CompletionSet.USERNAMES), true);
 +    }
 +
 +    public void registerCompletionForCommands(Token root, Map<CompletionSet,Set<String>> completionSet) {
 +      registerCompletionGeneral(root, completionSet.get(CompletionSet.COMMANDS), false);
 +    }
 +
 +    public void registerCompletionForNamespaces(Token root, Map<CompletionSet,Set<String>> completionSet) {
 +      registerCompletionGeneral(root, completionSet.get(CompletionSet.NAMESPACES), true);
 +    }
 +
 +    // abstract methods to override
 +    public abstract int execute(String fullCommand, CommandLine cl, Shell shellState) throws Exception;
 +
 +    public abstract String description();
 +
 +    /**
 +     * If the number of arguments is not always zero (not including those arguments handled through Options), make sure to override the {@link #usage()} method.
 +     * Otherwise, {@link #usage()} does need to be overridden.
 +     */
 +    public abstract int numArgs();
 +
 +    // OPTIONAL methods to override:
 +
 +    // the general version of getname uses reflection to get the class name
 +    // and then cuts off the suffix -Command to get the name of the command
 +    public String getName() {
 +      String s = this.getClass().getName();
 +      int st = Math.max(s.lastIndexOf('$'), s.lastIndexOf('.'));
 +      int i = s.indexOf("Command");
 +      return i > 0 ? s.substring(st + 1, i).toLowerCase(Locale.ENGLISH) : null;
 +    }
 +
 +    // The general version of this method adds the name
 +    // of the command to the completion tree
 +    public void registerCompletion(Token root, Map<CompletionSet,Set<String>> completion_set) {
 +      root.addSubcommand(new Token(getName()));
 +    }
 +
 +    // The general version of this method uses the HelpFormatter
 +    // that comes with the apache Options package to print out the help
 +    public final void printHelp(Shell shellState) {
 +      shellState.printHelp(usage(), "description: " + this.description(), getOptionsWithHelp());
 +    }
 +
 +    public final void printHelp(Shell shellState, int width) {
 +      shellState.printHelp(usage(), "description: " + this.description(), getOptionsWithHelp(), width);
 +    }
 +
 +    // Get options with help
 +    public final Options getOptionsWithHelp() {
 +      Options opts = getOptions();
 +      opts.addOption(new Option(helpOption, helpLongOption, false, "display this help"));
 +      return opts;
 +    }
 +
 +    // General usage is just the command
 +    public String usage() {
 +      return getName();
 +    }
 +
 +    // General Options are empty
 +    public Options getOptions() {
 +      return new Options();
 +    }
 +  }
 +
 +  public interface PrintLine {
 +    void print(String s);
 +
 +    void close();
 +  }
 +
 +  public static class PrintShell implements PrintLine {
 +    ConsoleReader reader;
 +
 +    public PrintShell(ConsoleReader reader) {
 +      this.reader = reader;
 +    }
 +
 +    @Override
 +    public void print(String s) {
 +      try {
 +        reader.println(s);
 +      } catch (Exception ex) {
 +        throw new RuntimeException(ex);
 +      }
 +    }
 +
 +    @Override
 +    public void close() {}
 +  };
 +
 +  public static class PrintFile implements PrintLine {
 +    PrintWriter writer;
 +
 +    public PrintFile(String filename) throws FileNotFoundException {
 +      writer = new PrintWriter(new BufferedWriter(new OutputStreamWriter(new FileOutputStream(filename), StandardCharsets.UTF_8)));
 +    }
 +
 +    @Override
 +    public void print(String s) {
 +      writer.println(s);
 +    }
 +
 +    @Override
 +    public void close() {
 +      writer.close();
 +    }
 +  };
 +
 +  public final void printLines(Iterator<String> lines, boolean paginate) throws IOException {
 +    printLines(lines, paginate, null);
 +  }
 +
 +  public final void printLines(Iterator<String> lines, boolean paginate, PrintLine out) throws IOException {
 +    int linesPrinted = 0;
 +    String prompt = "-- hit any key to continue or 'q' to quit --";
 +    int lastPromptLength = prompt.length();
 +    int termWidth = reader.getTerminal().getWidth();
 +    int maxLines = reader.getTerminal().getHeight();
 +
 +    String peek = null;
 +    while (lines.hasNext()) {
 +      String nextLine = lines.next();
 +      if (nextLine == null)
 +        continue;
 +      for (String line : nextLine.split("\\n")) {
 +        if (out == null) {
 +          if (peek != null) {
 +            reader.println(peek);
 +            if (paginate) {
 +              linesPrinted += peek.length() == 0 ? 0 : Math.ceil(peek.length() * 1.0 / termWidth);
 +
 +              // check if displaying the next line would result in
 +              // scrolling off the screen
 +              if (linesPrinted + Math.ceil(lastPromptLength * 1.0 / termWidth) + Math.ceil(prompt.length() * 1.0 / termWidth)
 +                  + Math.ceil(line.length() * 1.0 / termWidth) > maxLines) {
 +                linesPrinted = 0;
 +                int numdashes = (termWidth - prompt.length()) / 2;
 +                String nextPrompt = repeat("-", numdashes) + prompt + repeat("-", numdashes);
 +                lastPromptLength = nextPrompt.length();
 +                reader.print(nextPrompt);
 +                reader.flush();
 +
 +                if (Character.toUpperCase((char) reader.readCharacter()) == 'Q') {
 +                  reader.println();
 +                  return;
 +                }
 +                reader.println();
 +                termWidth = reader.getTerminal().getWidth();
 +                maxLines = reader.getTerminal().getHeight();
 +              }
 +            }
 +          }
 +          peek = line;
 +        } else {
 +          out.print(line);
 +        }
 +      }
 +    }
 +    if (out == null && peek != null) {
 +      reader.println(peek);
 +    }
 +  }
 +
 +  public final void printRecords(Iterable<Entry<Key,Value>> scanner, boolean printTimestamps, boolean paginate, Class<? extends Formatter> formatterClass,
 +      PrintLine outFile) throws IOException {
 +    printLines(FormatterFactory.getFormatter(formatterClass, scanner, printTimestamps), paginate, outFile);
 +  }
 +
 +  public final void printRecords(Iterable<Entry<Key,Value>> scanner, boolean printTimestamps, boolean paginate, Class<? extends Formatter> formatterClass)
 +      throws IOException {
 +    printLines(FormatterFactory.getFormatter(formatterClass, scanner, printTimestamps), paginate);
 +  }
 +
 +  public final void printBinaryRecords(Iterable<Entry<Key,Value>> scanner, boolean printTimestamps, boolean paginate, PrintLine outFile) throws IOException {
 +    printLines(FormatterFactory.getFormatter(binaryFormatterClass, scanner, printTimestamps), paginate, outFile);
 +  }
 +
 +  public final void printBinaryRecords(Iterable<Entry<Key,Value>> scanner, boolean printTimestamps, boolean paginate) throws IOException {
 +    printLines(FormatterFactory.getFormatter(binaryFormatterClass, scanner, printTimestamps), paginate);
 +  }
 +
 +  public static String repeat(String s, int c) {
 +    StringBuilder sb = new StringBuilder();
 +    for (int i = 0; i < c; i++)
 +      sb.append(s);
 +    return sb.toString();
 +  }
 +
 +  public void checkTableState() {
 +    if (getTableName().isEmpty())
 +      throw new IllegalStateException(
 +          "Not in a table context. Please use 'table <tableName>' to switch to a table, or use '-t' to specify a table if option is available.");
 +  }
 +
 +  private final void printConstraintViolationException(ConstraintViolationException cve) {
 +    printException(cve, "");
 +    int COL1 = 50, COL2 = 14;
 +    int col3 = Math.max(1, Math.min(Integer.MAX_VALUE, reader.getTerminal().getWidth() - COL1 - COL2 - 6));
 +    logError(String.format("%" + COL1 + "s-+-%" + COL2 + "s-+-%" + col3 + "s%n", repeat("-", COL1), repeat("-", COL2), repeat("-", col3)));
 +    logError(String.format("%-" + COL1 + "s | %" + COL2 + "s | %-" + col3 + "s%n", "Constraint class", "Violation code", "Violation Description"));
 +    logError(String.format("%" + COL1 + "s-+-%" + COL2 + "s-+-%" + col3 + "s%n", repeat("-", COL1), repeat("-", COL2), repeat("-", col3)));
 +    for (TConstraintViolationSummary cvs : cve.violationSummaries)
 +      logError(String.format("%-" + COL1 + "s | %" + COL2 + "d | %-" + col3 + "s%n", cvs.constrainClass, cvs.violationCode, cvs.violationDescription));
 +    logError(String.format("%" + COL1 + "s-+-%" + COL2 + "s-+-%" + col3 + "s%n", repeat("-", COL1), repeat("-", COL2), repeat("-", col3)));
 +  }
 +
 +  public final void printException(Exception e) {
 +    printException(e, e.getMessage());
 +  }
 +
 +  private final void printException(Exception e, String msg) {
 +    logError(e.getClass().getName() + (msg != null ? ": " + msg : ""));
 +    log.debug(e.getClass().getName() + (msg != null ? ": " + msg : ""), e);
 +  }
 +
 +  public static final void setDebugging(boolean debuggingEnabled) {
 +    Logger.getLogger(Constants.CORE_PACKAGE_NAME).setLevel(debuggingEnabled ? Level.TRACE : Level.INFO);
 +  }
 +
 +  public static final boolean isDebuggingEnabled() {
 +    return Logger.getLogger(Constants.CORE_PACKAGE_NAME).isTraceEnabled();
 +  }
 +
 +  private final void printHelp(String usage, String description, Options opts) {
 +    printHelp(usage, description, opts, Integer.MAX_VALUE);
 +  }
 +
 +  private final void printHelp(String usage, String description, Options opts, int width) {
 +    // TODO Use the OutputStream from the JLine ConsoleReader if we can ever get access to it
 +    new HelpFormatter().printHelp(writer, width, usage, description, opts, 2, 5, null, true);
 +    writer.flush();
 +  }
 +
 +  public int getExitCode() {
 +    return exitCode;
 +  }
 +
 +  public void resetExitCode() {
 +    exitCode = 0;
 +  }
 +
 +  public void setExit(boolean exit) {
 +    this.exit = exit;
 +  }
 +
 +  public boolean getExit() {
 +    return this.exit;
 +  }
 +
 +  public boolean isVerbose() {
 +    return verbose;
 +  }
 +
 +  public void setTableName(String tableName) {
 +    this.tableName = (tableName == null || tableName.isEmpty()) ? "" : Tables.qualified(tableName);
 +  }
 +
 +  public String getTableName() {
 +    return tableName;
 +  }
 +
 +  public ConsoleReader getReader() {
 +    return reader;
 +  }
 +
 +  public void updateUser(String principal, AuthenticationToken token) throws AccumuloException, AccumuloSecurityException {
 +    connector = instance.getConnector(principal, token);
 +    this.principal = principal;
 +    this.token = token;
 +  }
 +
 +  public String getPrincipal() {
 +    return principal;
 +  }
 +
 +  public AuthenticationToken getToken() {
 +    return token;
 +  }
 +
 +  /**
 +   * Return the formatter for the current table.
 +   * 
 +   * @return the formatter class for the current table
 +   */
 +  public Class<? extends Formatter> getFormatter() {
 +    return getFormatter(this.tableName);
 +  }
 +
 +  /**
 +   * Return the formatter for the given table.
 +   * 
 +   * @param tableName
 +   *          the table name
 +   * @return the formatter class for the given table
 +   */
 +  public Class<? extends Formatter> getFormatter(String tableName) {
 +    Class<? extends Formatter> formatter = FormatterCommand.getCurrentFormatter(tableName, this);
 +
 +    if (null == formatter) {
 +      logError("Could not load the specified formatter. Using the DefaultFormatter");
 +      return this.defaultFormatterClass;
 +    } else {
 +      return formatter;
 +    }
 +  }
 +
 +  public void setLogErrorsToConsole() {
 +    this.logErrorsToConsole = true;
 +  }
 +
 +  private void logError(String s) {
 +    log.error(s);
 +    if (logErrorsToConsole) {
 +      try {
 +        reader.println("ERROR: " + s);
 +        reader.flush();
 +      } catch (IOException e) {}
 +    }
 +  }
 +
 +  public String readMaskedLine(String prompt, Character mask) throws IOException {
 +    this.masking = true;
 +    String s = reader.readLine(prompt, mask);
 +    this.masking = false;
 +    return s;
 +  }
 +
 +  public boolean isMasking() {
 +    return masking;
 +  }
 +
 +  public boolean hasExited() {
 +    return exit;
 +  }
 +
 +  public boolean isTabCompletion() {
 +    return tabCompletion;
 +  }
 +
 +}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/2533b7ef/shell/src/main/java/org/apache/accumulo/shell/commands/ScanCommand.java
----------------------------------------------------------------------
diff --cc shell/src/main/java/org/apache/accumulo/shell/commands/ScanCommand.java
index aac16e3,0000000..ede0756
mode 100644,000000..100644
--- a/shell/src/main/java/org/apache/accumulo/shell/commands/ScanCommand.java
+++ b/shell/src/main/java/org/apache/accumulo/shell/commands/ScanCommand.java
@@@ -1,334 -1,0 +1,334 @@@
 +/*
 + * 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.accumulo.shell.commands;
 +
 +import java.io.FileNotFoundException;
 +import java.io.IOException;
 +import java.io.UnsupportedEncodingException;
 +import java.util.List;
 +import java.util.Map.Entry;
 +import java.util.concurrent.TimeUnit;
 +
 +import org.apache.accumulo.core.client.AccumuloException;
 +import org.apache.accumulo.core.client.AccumuloSecurityException;
 +import org.apache.accumulo.core.client.IteratorSetting;
 +import org.apache.accumulo.core.client.Scanner;
 +import org.apache.accumulo.core.client.ScannerBase;
++import org.apache.accumulo.core.client.TableNotFoundException;
 +import org.apache.accumulo.core.conf.AccumuloConfiguration;
 +import org.apache.accumulo.core.data.Key;
 +import org.apache.accumulo.core.data.Range;
 +import org.apache.accumulo.core.data.Value;
 +import org.apache.accumulo.core.security.Authorizations;
 +import org.apache.accumulo.core.util.format.BinaryFormatter;
 +import org.apache.accumulo.core.util.format.Formatter;
 +import org.apache.accumulo.core.util.interpret.DefaultScanInterpreter;
 +import org.apache.accumulo.core.util.interpret.ScanInterpreter;
 +import org.apache.accumulo.shell.Shell;
 +import org.apache.accumulo.shell.Shell.Command;
 +import org.apache.accumulo.shell.Shell.PrintFile;
 +import org.apache.accumulo.start.classloader.vfs.AccumuloVFSClassLoader;
 +import org.apache.commons.cli.CommandLine;
 +import org.apache.commons.cli.Option;
 +import org.apache.commons.cli.Options;
 +import org.apache.hadoop.io.Text;
 +
 +public class ScanCommand extends Command {
 +  
 +  private Option scanOptAuths, scanOptRow, scanOptColumns, disablePaginationOpt, showFewOpt, formatterOpt, interpreterOpt, formatterInterpeterOpt,
 +      outputFileOpt;
 +  
 +  protected Option timestampOpt;
 +  private Option optStartRowExclusive;
 +  private Option optEndRowExclusive;
 +  private Option timeoutOption;
 +  private Option profileOpt;
 +  
 +  @Override
 +  public int execute(final String fullCommand, final CommandLine cl, final Shell shellState) throws Exception {
 +    final PrintFile printFile = getOutputFile(cl);
 +    final String tableName = OptUtil.getTableOpt(cl, shellState);
 +    
 +    final Class<? extends Formatter> formatter = getFormatter(cl, tableName, shellState);
 +    final ScanInterpreter interpeter = getInterpreter(cl, tableName, shellState);
 +    
 +    // handle first argument, if present, the authorizations list to
 +    // scan with
 +    final Authorizations auths = getAuths(cl, shellState);
 +    final Scanner scanner = shellState.getConnector().createScanner(tableName, auths);
 +    
 +    // handle session-specific scan iterators
 +    addScanIterators(shellState, cl, scanner, tableName);
 +    
 +    // handle remaining optional arguments
 +    scanner.setRange(getRange(cl, interpeter));
 +    
 +    // handle columns
 +    fetchColumns(cl, scanner, interpeter);
 +    
 +    // set timeout
 +    scanner.setTimeout(getTimeout(cl), TimeUnit.MILLISECONDS);
 +    
 +    // output the records
 +    if (cl.hasOption(showFewOpt.getOpt())) {
 +      final String showLength = cl.getOptionValue(showFewOpt.getOpt());
 +      try {
 +        final int length = Integer.parseInt(showLength);
 +        if (length < 1) {
 +          throw new IllegalArgumentException();
 +        }
 +        BinaryFormatter.getlength(length);
 +        printBinaryRecords(cl, shellState, scanner, printFile);
 +      } catch (NumberFormatException nfe) {
 +        shellState.getReader().println("Arg must be an integer.");
 +      } catch (IllegalArgumentException iae) {
 +        shellState.getReader().println("Arg must be greater than one.");
 +      }
 +      
 +    } else {
 +      printRecords(cl, shellState, scanner, formatter, printFile);
 +    }
 +    if (printFile != null) {
 +      printFile.close();
 +    }
 +    
 +    return 0;
 +  }
 +  
 +  protected long getTimeout(final CommandLine cl) {
 +    if (cl.hasOption(timeoutOption.getLongOpt())) {
 +      return AccumuloConfiguration.getTimeInMillis(cl.getOptionValue(timeoutOption.getLongOpt()));
 +    }
 +    
 +    return Long.MAX_VALUE;
 +  }
 +  
 +  protected void addScanIterators(final Shell shellState, CommandLine cl, final Scanner scanner, final String tableName) {
 +    
 +    List<IteratorSetting> tableScanIterators;
 +    if (cl.hasOption(profileOpt.getOpt())) {
 +      String profile = cl.getOptionValue(profileOpt.getOpt());
 +      tableScanIterators = shellState.iteratorProfiles.get(profile);
 +      
 +      if (tableScanIterators == null) {
 +        throw new IllegalArgumentException("Profile " + profile + " does not exist");
 +      }
 +    } else {
 +      tableScanIterators = shellState.scanIteratorOptions.get(tableName);
 +      if (tableScanIterators == null) {
 +        Shell.log.debug("Found no scan iterators to set");
 +        return;
 +      }
 +    }
 +    
 +    Shell.log.debug("Found " + tableScanIterators.size() + " scan iterators to set");
 +    
 +    for (IteratorSetting setting : tableScanIterators) {
 +      Shell.log.debug("Setting scan iterator " + setting.getName() + " at priority " + setting.getPriority() + " using class name "
 +          + setting.getIteratorClass());
 +      for (Entry<String,String> option : setting.getOptions().entrySet()) {
 +        Shell.log.debug("Setting option for " + setting.getName() + ": " + option.getKey() + "=" + option.getValue());
 +      }
 +      scanner.addScanIterator(setting);
 +    }
 +  }
 +  
 +  protected void printRecords(final CommandLine cl, final Shell shellState, final Iterable<Entry<Key,Value>> scanner, final Class<? extends Formatter> formatter)
 +      throws IOException {
 +    printRecords(cl, shellState, scanner, formatter, null);
 +  }
 +  
 +  protected void printRecords(final CommandLine cl, final Shell shellState, final Iterable<Entry<Key,Value>> scanner,
 +      final Class<? extends Formatter> formatter, PrintFile outFile) throws IOException {
 +    if (outFile == null) {
 +      shellState.printRecords(scanner, cl.hasOption(timestampOpt.getOpt()), !cl.hasOption(disablePaginationOpt.getOpt()), formatter);
 +    } else {
 +      shellState.printRecords(scanner, cl.hasOption(timestampOpt.getOpt()), !cl.hasOption(disablePaginationOpt.getOpt()), formatter, outFile);
 +    }
 +  }
 +  
 +  protected void printBinaryRecords(final CommandLine cl, final Shell shellState, final Iterable<Entry<Key,Value>> scanner) throws IOException {
 +    printBinaryRecords(cl, shellState, scanner, null);
 +  }
 +  
 +  protected void printBinaryRecords(final CommandLine cl, final Shell shellState, final Iterable<Entry<Key,Value>> scanner, PrintFile outFile)
 +      throws IOException {
 +    if (outFile == null) {
 +      shellState.printBinaryRecords(scanner, cl.hasOption(timestampOpt.getOpt()), !cl.hasOption(disablePaginationOpt.getOpt()));
 +    } else {
 +      shellState.printBinaryRecords(scanner, cl.hasOption(timestampOpt.getOpt()), !cl.hasOption(disablePaginationOpt.getOpt()), outFile);
 +    }
 +  }
 +  
 +  protected ScanInterpreter getInterpreter(final CommandLine cl, final String tableName, final Shell shellState) throws Exception {
 +    
 +    Class<? extends ScanInterpreter> clazz = null;
 +    try {
 +      if (cl.hasOption(interpreterOpt.getOpt())) {
 +        clazz = AccumuloVFSClassLoader.loadClass(cl.getOptionValue(interpreterOpt.getOpt()), ScanInterpreter.class);
 +      } else if (cl.hasOption(formatterInterpeterOpt.getOpt())) {
 +        clazz = AccumuloVFSClassLoader.loadClass(cl.getOptionValue(formatterInterpeterOpt.getOpt()), ScanInterpreter.class);
 +      }
 +    } catch (ClassNotFoundException e) {
 +      shellState.getReader().println("Interpreter class could not be loaded.\n" + e.getMessage());
 +    }
 +    
 +    if (clazz == null)
 +      clazz = InterpreterCommand.getCurrentInterpreter(tableName, shellState);
 +    
 +    if (clazz == null)
 +      clazz = DefaultScanInterpreter.class;
 +    
 +    return clazz.newInstance();
 +  }
 +  
 +  protected Class<? extends Formatter> getFormatter(final CommandLine cl, final String tableName, final Shell shellState) throws IOException {
 +    
 +    try {
 +      if (cl.hasOption(formatterOpt.getOpt())) {
-         return AccumuloVFSClassLoader.loadClass(cl.getOptionValue(formatterOpt.getOpt()), Formatter.class);
-         
++        return shellState.getClassLoader(cl, shellState).loadClass(cl.getOptionValue(formatterOpt.getOpt())).asSubclass(Formatter.class);
 +      } else if (cl.hasOption(formatterInterpeterOpt.getOpt())) {
-         return AccumuloVFSClassLoader.loadClass(cl.getOptionValue(formatterInterpeterOpt.getOpt()), Formatter.class);
++        return shellState.getClassLoader(cl, shellState).loadClass(cl.getOptionValue(formatterInterpeterOpt.getOpt())).asSubclass(Formatter.class);
 +      }
-     } catch (ClassNotFoundException e) {
++    } catch (Exception e) {
 +      shellState.getReader().println("Formatter class could not be loaded.\n" + e.getMessage());
 +    }
 +    
 +    return shellState.getFormatter(tableName);
 +  }
 +  
 +  protected void fetchColumns(final CommandLine cl, final ScannerBase scanner, final ScanInterpreter formatter) throws UnsupportedEncodingException {
 +    if (cl.hasOption(scanOptColumns.getOpt())) {
 +      for (String a : cl.getOptionValue(scanOptColumns.getOpt()).split(",")) {
 +        final String sa[] = a.split(":", 2);
 +        if (sa.length == 1) {
 +          scanner.fetchColumnFamily(formatter.interpretColumnFamily(new Text(a.getBytes(Shell.CHARSET))));
 +        } else {
 +          scanner.fetchColumn(formatter.interpretColumnFamily(new Text(sa[0].getBytes(Shell.CHARSET))),
 +              formatter.interpretColumnQualifier(new Text(sa[1].getBytes(Shell.CHARSET))));
 +        }
 +      }
 +    }
 +  }
 +  
 +  protected Range getRange(final CommandLine cl, final ScanInterpreter formatter) throws UnsupportedEncodingException {
 +    if ((cl.hasOption(OptUtil.START_ROW_OPT) || cl.hasOption(OptUtil.END_ROW_OPT)) && cl.hasOption(scanOptRow.getOpt())) {
 +      // did not see a way to make commons cli do this check... it has mutually exclusive options but does not support the or
 +      throw new IllegalArgumentException("Options -" + scanOptRow.getOpt() + " AND (-" + OptUtil.START_ROW_OPT + " OR -" + OptUtil.END_ROW_OPT
 +          + ") are mutally exclusive ");
 +    }
 +    
 +    if (cl.hasOption(scanOptRow.getOpt())) {
 +      return new Range(formatter.interpretRow(new Text(cl.getOptionValue(scanOptRow.getOpt()).getBytes(Shell.CHARSET))));
 +    } else {
 +      Text startRow = OptUtil.getStartRow(cl);
 +      if (startRow != null)
 +        startRow = formatter.interpretBeginRow(startRow);
 +      Text endRow = OptUtil.getEndRow(cl);
 +      if (endRow != null)
 +        endRow = formatter.interpretEndRow(endRow);
 +      final boolean startInclusive = !cl.hasOption(optStartRowExclusive.getOpt());
 +      final boolean endInclusive = !cl.hasOption(optEndRowExclusive.getOpt());
 +      return new Range(startRow, startInclusive, endRow, endInclusive);
 +    }
 +  }
 +  
 +  protected Authorizations getAuths(final CommandLine cl, final Shell shellState) throws AccumuloSecurityException, AccumuloException {
 +    final String user = shellState.getConnector().whoami();
 +    Authorizations auths = shellState.getConnector().securityOperations().getUserAuthorizations(user);
 +    if (cl.hasOption(scanOptAuths.getOpt())) {
 +      auths = ScanCommand.parseAuthorizations(cl.getOptionValue(scanOptAuths.getOpt()));
 +    }
 +    return auths;
 +  }
 +  
 +  static Authorizations parseAuthorizations(final String field) {
 +    if (field == null || field.isEmpty()) {
 +      return Authorizations.EMPTY;
 +    }
 +    return new Authorizations(field.split(","));
 +  }
 +  
 +  @Override
 +  public String description() {
 +    return "scans the table, and displays the resulting records";
 +  }
 +  
 +  @Override
 +  public Options getOptions() {
 +    final Options o = new Options();
 +    
 +    scanOptAuths = new Option("s", "scan-authorizations", true, "scan authorizations (all user auths are used if this argument is not specified)");
 +    optStartRowExclusive = new Option("be", "begin-exclusive", false, "make start row exclusive (by default it's inclusive)");
 +    optStartRowExclusive.setArgName("begin-exclusive");
 +    optEndRowExclusive = new Option("ee", "end-exclusive", false, "make end row exclusive (by default it's inclusive)");
 +    optEndRowExclusive.setArgName("end-exclusive");
 +    scanOptRow = new Option("r", "row", true, "row to scan");
 +    scanOptColumns = new Option("c", "columns", true, "comma-separated columns");
 +    timestampOpt = new Option("st", "show-timestamps", false, "display timestamps");
 +    disablePaginationOpt = new Option("np", "no-pagination", false, "disable pagination of output");
 +    showFewOpt = new Option("f", "show-few", true, "show only a specified number of characters");
 +    formatterOpt = new Option("fm", "formatter", true, "fully qualified name of the formatter class to use");
 +    interpreterOpt = new Option("i", "interpreter", true, "fully qualified name of the interpreter class to use");
 +    formatterInterpeterOpt = new Option("fi", "fmt-interpreter", true, "fully qualified name of a class that is a formatter and interpreter");
 +    timeoutOption = new Option(null, "timeout", true,
 +        "time before scan should fail if no data is returned. If no unit is given assumes seconds.  Units d,h,m,s,and ms are supported.  e.g. 30s or 100ms");
 +    outputFileOpt = new Option("o", "output", true, "local file to write the scan output to");
 +    
 +    scanOptAuths.setArgName("comma-separated-authorizations");
 +    scanOptRow.setArgName("row");
 +    scanOptColumns.setArgName("<columnfamily>[:<columnqualifier>]{,<columnfamily>[:<columnqualifier>]}");
 +    showFewOpt.setRequired(false);
 +    showFewOpt.setArgName("int");
 +    formatterOpt.setArgName("className");
 +    timeoutOption.setArgName("timeout");
 +    outputFileOpt.setArgName("file");
 +    
 +    profileOpt = new Option("pn", "profile", true, "iterator profile name");
 +    profileOpt.setArgName("profile");
 +    
 +    o.addOption(scanOptAuths);
 +    o.addOption(scanOptRow);
 +    o.addOption(OptUtil.startRowOpt());
 +    o.addOption(OptUtil.endRowOpt());
 +    o.addOption(optStartRowExclusive);
 +    o.addOption(optEndRowExclusive);
 +    o.addOption(scanOptColumns);
 +    o.addOption(timestampOpt);
 +    o.addOption(disablePaginationOpt);
 +    o.addOption(OptUtil.tableOpt("table to be scanned"));
 +    o.addOption(showFewOpt);
 +    o.addOption(formatterOpt);
 +    o.addOption(interpreterOpt);
 +    o.addOption(formatterInterpeterOpt);
 +    o.addOption(timeoutOption);
 +    o.addOption(outputFileOpt);
 +    o.addOption(profileOpt);
 +    
 +    return o;
 +  }
 +  
 +  @Override
 +  public int numArgs() {
 +    return 0;
 +  }
 +  
 +  protected PrintFile getOutputFile(final CommandLine cl) throws FileNotFoundException {
 +    final String outputFile = cl.getOptionValue(outputFileOpt.getOpt());
 +    return (outputFile == null ? null : new PrintFile(outputFile));
 +  }
 +}