You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@accumulo.apache.org by bu...@apache.org on 2014/04/26 01:12:30 UTC

[01/17] ACCUMULO-2726 Adds back missing methods for binary backwards compat with non-Deprecated methods from 1.5.0. Marks things that we're for sure removing with @Deprecated; mostly things that shouldn't have been in public to begin with.

Repository: accumulo
Updated Branches:
  refs/heads/master 6ac5c5f35 -> eae953e6a


http://git-wip-us.apache.org/repos/asf/accumulo/blob/f4454a06/core/src/main/java/org/apache/accumulo/core/client/mock/MockSecurityOperationsImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mock/MockSecurityOperationsImpl.java b/core/src/main/java/org/apache/accumulo/core/client/mock/MockSecurityOperationsImpl.java
new file mode 100644
index 0000000..f97069a
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/client/mock/MockSecurityOperationsImpl.java
@@ -0,0 +1,225 @@
+/*
+ * 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.core.client.mock;
+
+import java.util.EnumSet;
+import java.util.Set;
+
+import org.apache.accumulo.core.client.AccumuloException;
+import org.apache.accumulo.core.client.AccumuloSecurityException;
+import org.apache.accumulo.core.client.admin.SecurityOperations;
+import org.apache.accumulo.core.client.impl.thrift.SecurityErrorCode;
+import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
+import org.apache.accumulo.core.client.security.tokens.PasswordToken;
+import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.core.security.NamespacePermission;
+import org.apache.accumulo.core.security.SystemPermission;
+import org.apache.accumulo.core.security.TablePermission;
+
+class MockSecurityOperationsImpl implements SecurityOperations {
+
+  final private MockAccumulo acu;
+
+  MockSecurityOperationsImpl(MockAccumulo acu) {
+    this.acu = acu;
+  }
+
+  @Deprecated
+  @Override
+  public void createUser(String user, byte[] password, Authorizations authorizations) throws AccumuloException, AccumuloSecurityException {
+    createLocalUser(user, new PasswordToken(password));
+    changeUserAuthorizations(user, authorizations);
+  }
+
+  @Override
+  public void createLocalUser(String principal, PasswordToken password) throws AccumuloException, AccumuloSecurityException {
+    this.acu.users.put(principal, new MockUser(principal, password, new Authorizations()));
+  }
+
+  @Deprecated
+  @Override
+  public void dropUser(String user) throws AccumuloException, AccumuloSecurityException {
+    dropLocalUser(user);
+  }
+
+  @Override
+  public void dropLocalUser(String principal) throws AccumuloException, AccumuloSecurityException {
+    this.acu.users.remove(principal);
+  }
+
+  @Deprecated
+  @Override
+  public boolean authenticateUser(String user, byte[] password) throws AccumuloException, AccumuloSecurityException {
+    return authenticateUser(user, new PasswordToken(password));
+  }
+
+  @Override
+  public boolean authenticateUser(String principal, AuthenticationToken token) throws AccumuloException, AccumuloSecurityException {
+    MockUser user = acu.users.get(principal);
+    if (user == null)
+      return false;
+    return user.token.equals(token);
+  }
+
+  @Deprecated
+  @Override
+  public void changeUserPassword(String user, byte[] password) throws AccumuloException, AccumuloSecurityException {
+    changeLocalUserPassword(user, new PasswordToken(password));
+  }
+
+  @Override
+  public void changeLocalUserPassword(String principal, PasswordToken token) throws AccumuloException, AccumuloSecurityException {
+    MockUser user = acu.users.get(principal);
+    if (user != null)
+      user.token = token.clone();
+    else
+      throw new AccumuloSecurityException(principal, SecurityErrorCode.USER_DOESNT_EXIST);
+  }
+
+  @Override
+  public void changeUserAuthorizations(String principal, Authorizations authorizations) throws AccumuloException, AccumuloSecurityException {
+    MockUser user = acu.users.get(principal);
+    if (user != null)
+      user.authorizations = authorizations;
+    else
+      throw new AccumuloSecurityException(principal, SecurityErrorCode.USER_DOESNT_EXIST);
+  }
+
+  @Override
+  public Authorizations getUserAuthorizations(String principal) throws AccumuloException, AccumuloSecurityException {
+    MockUser user = acu.users.get(principal);
+    if (user != null)
+      return user.authorizations;
+    else
+      throw new AccumuloSecurityException(principal, SecurityErrorCode.USER_DOESNT_EXIST);
+  }
+
+  @Override
+  public boolean hasSystemPermission(String principal, SystemPermission perm) throws AccumuloException, AccumuloSecurityException {
+    MockUser user = acu.users.get(principal);
+    if (user != null)
+      return user.permissions.contains(perm);
+    else
+      throw new AccumuloSecurityException(principal, SecurityErrorCode.USER_DOESNT_EXIST);
+  }
+
+  @Override
+  public boolean hasTablePermission(String principal, String tableName, TablePermission perm) throws AccumuloException, AccumuloSecurityException {
+    MockTable table = acu.tables.get(tableName);
+    if (table == null)
+      throw new AccumuloSecurityException(tableName, SecurityErrorCode.TABLE_DOESNT_EXIST);
+    EnumSet<TablePermission> perms = table.userPermissions.get(principal);
+    if (perms == null)
+      return false;
+    return perms.contains(perm);
+  }
+
+  @Override
+  public boolean hasNamespacePermission(String principal, String namespace, NamespacePermission permission) throws AccumuloException, AccumuloSecurityException {
+    MockNamespace mockNamespace = acu.namespaces.get(namespace);
+    if (mockNamespace == null)
+      throw new AccumuloSecurityException(namespace, SecurityErrorCode.NAMESPACE_DOESNT_EXIST);
+    EnumSet<NamespacePermission> perms = mockNamespace.userPermissions.get(principal);
+    if (perms == null)
+      return false;
+    return perms.contains(permission);
+  }
+
+  @Override
+  public void grantSystemPermission(String principal, SystemPermission permission) throws AccumuloException, AccumuloSecurityException {
+    MockUser user = acu.users.get(principal);
+    if (user != null)
+      user.permissions.add(permission);
+    else
+      throw new AccumuloSecurityException(principal, SecurityErrorCode.USER_DOESNT_EXIST);
+  }
+
+  @Override
+  public void grantTablePermission(String principal, String tableName, TablePermission permission) throws AccumuloException, AccumuloSecurityException {
+    if (acu.users.get(principal) == null)
+      throw new AccumuloSecurityException(principal, SecurityErrorCode.USER_DOESNT_EXIST);
+    MockTable table = acu.tables.get(tableName);
+    if (table == null)
+      throw new AccumuloSecurityException(tableName, SecurityErrorCode.TABLE_DOESNT_EXIST);
+    EnumSet<TablePermission> perms = table.userPermissions.get(principal);
+    if (perms == null)
+      table.userPermissions.put(principal, EnumSet.of(permission));
+    else
+      perms.add(permission);
+  }
+
+  @Override
+  public void grantNamespacePermission(String principal, String namespace, NamespacePermission permission) throws AccumuloException, AccumuloSecurityException {
+    if (acu.users.get(principal) == null)
+      throw new AccumuloSecurityException(principal, SecurityErrorCode.USER_DOESNT_EXIST);
+    MockNamespace mockNamespace = acu.namespaces.get(namespace);
+    if (mockNamespace == null)
+      throw new AccumuloSecurityException(namespace, SecurityErrorCode.NAMESPACE_DOESNT_EXIST);
+    EnumSet<NamespacePermission> perms = mockNamespace.userPermissions.get(principal);
+    if (perms == null)
+      mockNamespace.userPermissions.put(principal, EnumSet.of(permission));
+    else
+      perms.add(permission);
+  }
+
+  @Override
+  public void revokeSystemPermission(String principal, SystemPermission permission) throws AccumuloException, AccumuloSecurityException {
+    MockUser user = acu.users.get(principal);
+    if (user != null)
+      user.permissions.remove(permission);
+    else
+      throw new AccumuloSecurityException(principal, SecurityErrorCode.USER_DOESNT_EXIST);
+  }
+
+  @Override
+  public void revokeTablePermission(String principal, String tableName, TablePermission permission) throws AccumuloException, AccumuloSecurityException {
+    if (acu.users.get(principal) == null)
+      throw new AccumuloSecurityException(principal, SecurityErrorCode.USER_DOESNT_EXIST);
+    MockTable table = acu.tables.get(tableName);
+    if (table == null)
+      throw new AccumuloSecurityException(tableName, SecurityErrorCode.TABLE_DOESNT_EXIST);
+    EnumSet<TablePermission> perms = table.userPermissions.get(principal);
+    if (perms != null)
+      perms.remove(permission);
+
+  }
+
+  @Override
+  public void revokeNamespacePermission(String principal, String namespace, NamespacePermission permission) throws AccumuloException, AccumuloSecurityException {
+    if (acu.users.get(principal) == null)
+      throw new AccumuloSecurityException(principal, SecurityErrorCode.USER_DOESNT_EXIST);
+    MockNamespace mockNamespace = acu.namespaces.get(namespace);
+    if (mockNamespace == null)
+      throw new AccumuloSecurityException(namespace, SecurityErrorCode.NAMESPACE_DOESNT_EXIST);
+    EnumSet<NamespacePermission> perms = mockNamespace.userPermissions.get(principal);
+    if (perms != null)
+      perms.remove(permission);
+
+  }
+
+  @Deprecated
+  @Override
+  public Set<String> listUsers() throws AccumuloException, AccumuloSecurityException {
+    return listLocalUsers();
+  }
+
+  @Override
+  public Set<String> listLocalUsers() throws AccumuloException, AccumuloSecurityException {
+    return acu.users.keySet();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/f4454a06/core/src/main/java/org/apache/accumulo/core/client/mock/MockShell.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mock/MockShell.java b/core/src/main/java/org/apache/accumulo/core/client/mock/MockShell.java
index 2bc9436..0d3aca2 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mock/MockShell.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mock/MockShell.java
@@ -20,7 +20,11 @@ import java.io.ByteArrayInputStream;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.OutputStream;
+import java.io.Writer;
 
+import org.apache.commons.io.output.WriterOutputStream;
+
+import org.apache.commons.cli.CommandLine;
 import jline.console.ConsoleReader;
 
 import org.apache.accumulo.core.Constants;
@@ -35,11 +39,28 @@ public class MockShell extends Shell {
   
   protected InputStream in;
   protected OutputStream out;
-  
+
+  /**
+   * Will only be set if you use either the Writer constructor or the setWriter(Writer) method
+   * @deprecated since 1.6.0; use out
+   */
+  @Deprecated
+  protected Writer writer = null;
+
   public MockShell(InputStream in, OutputStream out) throws IOException {
     super();
     this.in = in;
     this.out = out;
+    // we presume they don't use the writer field unless they use the other constructor.
+  }
+
+  /**
+   * @deprecated since 1.6.0; use OutputStream version
+   */
+  @Deprecated
+  public MockShell(InputStream in, Writer out) throws IOException {
+    this(in, new WriterOutputStream(out, Constants.UTF8.name()));
+    this.writer = out;
   }
   
   public boolean config(String... args) {
@@ -67,6 +88,15 @@ public class MockShell extends Shell {
     // We always want a MockInstance for this test
     instance = new MockInstance();
   }
+
+  /**
+   * @deprecated since 1.6.0; use ShellOptionsJC version
+   */
+  @Deprecated
+  protected void setInstance(CommandLine cl) {
+    // same result as in previous version
+    setInstance((ShellOptionsJC)null);
+  }
   
   public int start() throws IOException {
     if (configError)
@@ -122,6 +152,15 @@ public class MockShell extends Shell {
   public void setConsoleWriter(OutputStream out) {
     this.out = out;
   }
+
+  /**
+   * @deprecated since 1.6.0; use the OutputStream version
+   */
+  @Deprecated
+  public void setConsoleWriter(Writer out) {
+    setConsoleWriter(new WriterOutputStream(out, Constants.UTF8.name()));
+    this.writer = out;
+  }
   
   /**
    * Convenience method to create the byte-array to hand to the console

http://git-wip-us.apache.org/repos/asf/accumulo/blob/f4454a06/core/src/main/java/org/apache/accumulo/core/client/mock/MockTableOperations.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mock/MockTableOperations.java b/core/src/main/java/org/apache/accumulo/core/client/mock/MockTableOperations.java
index d3b1571..ee8880c 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mock/MockTableOperations.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mock/MockTableOperations.java
@@ -16,432 +16,13 @@
  */
 package org.apache.accumulo.core.client.mock;
 
-import java.io.DataInputStream;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Set;
-import java.util.SortedSet;
-import java.util.TreeSet;
-
-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.TableExistsException;
-import org.apache.accumulo.core.client.TableNotFoundException;
-import org.apache.accumulo.core.client.admin.DiskUsage;
-import org.apache.accumulo.core.client.admin.FindMax;
-import org.apache.accumulo.core.client.admin.TableOperationsHelper;
-import org.apache.accumulo.core.client.admin.TimeType;
-import org.apache.accumulo.core.client.impl.Tables;
-import org.apache.accumulo.core.conf.AccumuloConfiguration;
-import org.apache.accumulo.core.data.Key;
-import org.apache.accumulo.core.data.Mutation;
-import org.apache.accumulo.core.data.Range;
-import org.apache.accumulo.core.data.Value;
-import org.apache.accumulo.core.file.FileOperations;
-import org.apache.accumulo.core.file.FileSKVIterator;
-import org.apache.accumulo.core.metadata.MetadataTable;
-import org.apache.accumulo.core.metadata.RootTable;
-import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.core.security.ColumnVisibility;
-import org.apache.accumulo.start.classloader.vfs.AccumuloVFSClassLoader;
-import org.apache.commons.lang.NotImplementedException;
-import org.apache.hadoop.fs.FSDataOutputStream;
-import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.Text;
-
-public class MockTableOperations extends TableOperationsHelper {
-  private static final byte[] ZERO = {0};
-  private final MockAccumulo acu;
-  private final String username;
+/**
+ * @deprecated since 1.6.0; not intended for public api and you should not use it.
+ */
+@Deprecated
+public class MockTableOperations extends MockTableOperationsImpl {
 
   MockTableOperations(MockAccumulo acu, String username) {
-    this.acu = acu;
-    this.username = username;
-  }
-
-  @Override
-  public SortedSet<String> list() {
-    return new TreeSet<String>(acu.tables.keySet());
-  }
-
-  @Override
-  public boolean exists(String tableName) {
-    return acu.tables.containsKey(tableName);
-  }
-
-  private boolean namespaceExists(String namespace) {
-    return acu.namespaces.containsKey(namespace);
-  }
-
-  @Override
-  public void create(String tableName) throws AccumuloException, AccumuloSecurityException, TableExistsException {
-    create(tableName, true, TimeType.MILLIS);
-  }
-
-  @Override
-  public void create(String tableName, boolean versioningIter) throws AccumuloException, AccumuloSecurityException, TableExistsException {
-    create(tableName, versioningIter, TimeType.MILLIS);
-  }
-
-  @Override
-  public void create(String tableName, boolean versioningIter, TimeType timeType) throws AccumuloException, AccumuloSecurityException, TableExistsException {
-    String namespace = Tables.qualify(tableName).getFirst();
-    if (!tableName.matches(Tables.VALID_NAME_REGEX)) {
-      throw new IllegalArgumentException();
-    }
-    if (exists(tableName))
-      throw new TableExistsException(tableName, tableName, "");
-
-    if (!namespaceExists(namespace)) {
-      throw new IllegalArgumentException("Namespace (" + namespace + ") does not exist, create it first");
-    }
-    acu.createTable(username, tableName, versioningIter, timeType);
-  }
-
-  @Override
-  public void addSplits(String tableName, SortedSet<Text> partitionKeys) throws TableNotFoundException, AccumuloException, AccumuloSecurityException {
-    if (!exists(tableName))
-      throw new TableNotFoundException(tableName, tableName, "");
-    acu.addSplits(tableName, partitionKeys);
-  }
-
-  @Deprecated
-  @Override
-  public Collection<Text> getSplits(String tableName) throws TableNotFoundException {
-    return listSplits(tableName);
-  }
-
-  @Deprecated
-  @Override
-  public Collection<Text> getSplits(String tableName, int maxSplits) throws TableNotFoundException {
-    return listSplits(tableName);
-  }
-
-  @Override
-  public Collection<Text> listSplits(String tableName) throws TableNotFoundException {
-    if (!exists(tableName))
-      throw new TableNotFoundException(tableName, tableName, "");
-    return acu.getSplits(tableName);
-  }
-
-  @Override
-  public Collection<Text> listSplits(String tableName, int maxSplits) throws TableNotFoundException {
-    return listSplits(tableName);
-  }
-
-  @Override
-  public void delete(String tableName) throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
-    if (!exists(tableName))
-      throw new TableNotFoundException(tableName, tableName, "");
-    acu.tables.remove(tableName);
-  }
-
-  @Override
-  public void rename(String oldTableName, String newTableName) throws AccumuloSecurityException, TableNotFoundException, AccumuloException,
-      TableExistsException {
-    if (!exists(oldTableName))
-      throw new TableNotFoundException(oldTableName, oldTableName, "");
-    if (exists(newTableName))
-      throw new TableExistsException(newTableName, newTableName, "");
-    MockTable t = acu.tables.remove(oldTableName);
-    String namespace = Tables.qualify(newTableName).getFirst();
-    MockNamespace n = acu.namespaces.get(namespace);
-    if (n == null) {
-      n = new MockNamespace();
-    }
-    t.setNamespaceName(namespace);
-    t.setNamespace(n);
-    acu.namespaces.put(namespace, n);
-    acu.tables.put(newTableName, t);
-  }
-
-  @Deprecated
-  @Override
-  public void flush(String tableName) throws AccumuloException, AccumuloSecurityException {}
-
-  @Override
-  public void setProperty(String tableName, String property, String value) throws AccumuloException, AccumuloSecurityException {
-    acu.tables.get(tableName).settings.put(property, value);
-  }
-
-  @Override
-  public void removeProperty(String tableName, String property) throws AccumuloException, AccumuloSecurityException {
-    acu.tables.get(tableName).settings.remove(property);
-  }
-
-  @Override
-  public Iterable<Entry<String,String>> getProperties(String tableName) throws TableNotFoundException {
-    String namespace = Tables.qualify(tableName).getFirst();
-    if (!exists(tableName)) {
-      if (!namespaceExists(namespace))
-        throw new TableNotFoundException(tableName, new NamespaceNotFoundException(null, namespace, null));
-      throw new TableNotFoundException(null, tableName, null);
-    }
-
-    Set<Entry<String,String>> props = new HashSet<Entry<String,String>>(acu.namespaces.get(namespace).settings.entrySet());
-
-    Set<Entry<String,String>> tableProps = acu.tables.get(tableName).settings.entrySet();
-    for (Entry<String,String> e : tableProps) {
-      if (props.contains(e)) {
-        props.remove(e);
-      }
-      props.add(e);
-    }
-    return props;
-  }
-
-  @Override
-  public void setLocalityGroups(String tableName, Map<String,Set<Text>> groups) throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
-    if (!exists(tableName))
-      throw new TableNotFoundException(tableName, tableName, "");
-    acu.tables.get(tableName).setLocalityGroups(groups);
-  }
-
-  @Override
-  public Map<String,Set<Text>> getLocalityGroups(String tableName) throws AccumuloException, TableNotFoundException {
-    if (!exists(tableName))
-      throw new TableNotFoundException(tableName, tableName, "");
-    return acu.tables.get(tableName).getLocalityGroups();
-  }
-
-  @Override
-  public Set<Range> splitRangeByTablets(String tableName, Range range, int maxSplits) throws AccumuloException, AccumuloSecurityException,
-      TableNotFoundException {
-    if (!exists(tableName))
-      throw new TableNotFoundException(tableName, tableName, "");
-    return Collections.singleton(range);
-  }
-
-  @Override
-  public void importDirectory(String tableName, String dir, String failureDir, boolean setTime) throws IOException, AccumuloException,
-      AccumuloSecurityException, TableNotFoundException {
-    long time = System.currentTimeMillis();
-    MockTable table = acu.tables.get(tableName);
-    if (table == null) {
-      throw new TableNotFoundException(null, tableName, "The table was not found");
-    }
-    Path importPath = new Path(dir);
-    Path failurePath = new Path(failureDir);
-
-    FileSystem fs = acu.getFileSystem();
-    /*
-     * check preconditions
-     */
-    // directories are directories
-    if (fs.isFile(importPath)) {
-      throw new IOException("Import path must be a directory.");
-    }
-    if (fs.isFile(failurePath)) {
-      throw new IOException("Failure path must be a directory.");
-    }
-    // failures are writable
-    Path createPath = failurePath.suffix("/.createFile");
-    FSDataOutputStream createStream = null;
-    try {
-      createStream = fs.create(createPath);
-    } catch (IOException e) {
-      throw new IOException("Error path is not writable.");
-    } finally {
-      if (createStream != null) {
-        createStream.close();
-      }
-    }
-    fs.delete(createPath, false);
-    // failures are empty
-    FileStatus[] failureChildStats = fs.listStatus(failurePath);
-    if (failureChildStats.length > 0) {
-      throw new IOException("Error path must be empty.");
-    }
-    /*
-     * Begin the import - iterate the files in the path
-     */
-    for (FileStatus importStatus : fs.listStatus(importPath)) {
-      try {
-        FileSKVIterator importIterator = FileOperations.getInstance().openReader(importStatus.getPath().toString(), true, fs, fs.getConf(),
-            AccumuloConfiguration.getDefaultConfiguration());
-        while (importIterator.hasTop()) {
-          Key key = importIterator.getTopKey();
-          Value value = importIterator.getTopValue();
-          if (setTime) {
-            key.setTimestamp(time);
-          }
-          Mutation mutation = new Mutation(key.getRow());
-          if (!key.isDeleted()) {
-            mutation.put(key.getColumnFamily(), key.getColumnQualifier(), new ColumnVisibility(key.getColumnVisibilityData().toArray()), key.getTimestamp(),
-                value);
-          } else {
-            mutation.putDelete(key.getColumnFamily(), key.getColumnQualifier(), new ColumnVisibility(key.getColumnVisibilityData().toArray()),
-                key.getTimestamp());
-          }
-          table.addMutation(mutation);
-          importIterator.next();
-        }
-      } catch (Exception e) {
-        FSDataOutputStream failureWriter = null;
-        DataInputStream failureReader = null;
-        try {
-          failureWriter = fs.create(failurePath.suffix("/" + importStatus.getPath().getName()));
-          failureReader = fs.open(importStatus.getPath());
-          int read = 0;
-          byte[] buffer = new byte[1024];
-          while (-1 != (read = failureReader.read(buffer))) {
-            failureWriter.write(buffer, 0, read);
-          }
-        } finally {
-          if (failureReader != null)
-            failureReader.close();
-          if (failureWriter != null)
-            failureWriter.close();
-        }
-      }
-      fs.delete(importStatus.getPath(), true);
-    }
-  }
-
-  @Override
-  public void offline(String tableName) throws AccumuloSecurityException, AccumuloException, TableNotFoundException {
-    offline(tableName, false);
-  }
-
-  @Override
-  public void offline(String tableName, boolean wait) throws AccumuloSecurityException, AccumuloException, TableNotFoundException {
-    if (!exists(tableName))
-      throw new TableNotFoundException(tableName, tableName, "");
-  }
-
-  @Override
-  public void online(String tableName) throws AccumuloSecurityException, AccumuloException, TableNotFoundException {
-    online(tableName, false);
-  }
-
-  @Override
-  public void online(String tableName, boolean wait) throws AccumuloSecurityException, AccumuloException, TableNotFoundException {
-    if (!exists(tableName))
-      throw new TableNotFoundException(tableName, tableName, "");
-  }
-
-  @Override
-  public void clearLocatorCache(String tableName) throws TableNotFoundException {
-    if (!exists(tableName))
-      throw new TableNotFoundException(tableName, tableName, "");
-  }
-
-  @Override
-  public Map<String,String> tableIdMap() {
-    Map<String,String> result = new HashMap<String,String>();
-    for (String table : acu.tables.keySet()) {
-      if (RootTable.NAME.equals(table))
-        result.put(table, RootTable.ID);
-      else if (MetadataTable.NAME.equals(table))
-        result.put(table, MetadataTable.ID);
-      else
-        result.put(table, table);
-    }
-    return result;
-  }
-
-  @Override
-  public List<DiskUsage> getDiskUsage(Set<String> tables) throws AccumuloException, AccumuloSecurityException {
-
-    List<DiskUsage> diskUsages = new ArrayList<DiskUsage>();
-    diskUsages.add(new DiskUsage(new TreeSet<String>(tables), 0l));
-
-    return diskUsages;
-  }
-
-  @Override
-  public void merge(String tableName, Text start, Text end) throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
-    if (!exists(tableName))
-      throw new TableNotFoundException(tableName, tableName, "");
-    acu.merge(tableName, start, end);
-  }
-
-  @Override
-  public void deleteRows(String tableName, Text start, Text end) throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
-    if (!exists(tableName))
-      throw new TableNotFoundException(tableName, tableName, "");
-    MockTable t = acu.tables.get(tableName);
-    Text startText = new Text(start);
-    Text endText = new Text(end);
-    startText.append(ZERO, 0, 1);
-    endText.append(ZERO, 0, 1);
-    Set<Key> keep = new TreeSet<Key>(t.table.subMap(new Key(startText), new Key(endText)).keySet());
-    t.table.keySet().removeAll(keep);
-  }
-
-  @Override
-  public void compact(String tableName, Text start, Text end, boolean flush, boolean wait) throws AccumuloSecurityException, TableNotFoundException,
-      AccumuloException {
-    if (!exists(tableName))
-      throw new TableNotFoundException(tableName, tableName, "");
-  }
-
-  @Override
-  public void compact(String tableName, Text start, Text end, List<IteratorSetting> iterators, boolean flush, boolean wait) throws AccumuloSecurityException,
-      TableNotFoundException, AccumuloException {
-    if (!exists(tableName))
-      throw new TableNotFoundException(tableName, tableName, "");
-  }
-
-  @Override
-  public void cancelCompaction(String tableName) throws AccumuloSecurityException, TableNotFoundException, AccumuloException {
-    if (!exists(tableName))
-      throw new TableNotFoundException(tableName, tableName, "");
-  }
-
-  @Override
-  public void clone(String srcTableName, String newTableName, boolean flush, Map<String,String> propertiesToSet, Set<String> propertiesToExclude)
-      throws AccumuloException, AccumuloSecurityException, TableNotFoundException, TableExistsException {
-    throw new NotImplementedException();
-  }
-
-  @Override
-  public void flush(String tableName, Text start, Text end, boolean wait) throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
-    if (!exists(tableName))
-      throw new TableNotFoundException(tableName, tableName, "");
-  }
-
-  @Override
-  public Text getMaxRow(String tableName, Authorizations auths, Text startRow, boolean startInclusive, Text endRow, boolean endInclusive)
-      throws TableNotFoundException, AccumuloException, AccumuloSecurityException {
-    MockTable table = acu.tables.get(tableName);
-    if (table == null)
-      throw new TableNotFoundException(tableName, tableName, "no such table");
-
-    return FindMax.findMax(new MockScanner(table, auths), startRow, startInclusive, endRow, endInclusive);
-  }
-
-  @Override
-  public void importTable(String tableName, String exportDir) throws TableExistsException, AccumuloException, AccumuloSecurityException {
-    throw new NotImplementedException();
-  }
-
-  @Override
-  public void exportTable(String tableName, String exportDir) throws TableNotFoundException, AccumuloException, AccumuloSecurityException {
-    throw new NotImplementedException();
-  }
-
-  @Override
-  public boolean testClassLoad(String tableName, String className, String asTypeName) throws AccumuloException, AccumuloSecurityException,
-      TableNotFoundException {
-
-    try {
-      AccumuloVFSClassLoader.loadClass(className, Class.forName(asTypeName));
-    } catch (ClassNotFoundException e) {
-      e.printStackTrace();
-      return false;
-    }
-    return true;
+    super(acu,username);
   }
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/f4454a06/core/src/main/java/org/apache/accumulo/core/client/mock/MockTableOperationsImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mock/MockTableOperationsImpl.java b/core/src/main/java/org/apache/accumulo/core/client/mock/MockTableOperationsImpl.java
new file mode 100644
index 0000000..fea9568
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/client/mock/MockTableOperationsImpl.java
@@ -0,0 +1,447 @@
+/*
+ * 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.core.client.mock;
+
+import java.io.DataInputStream;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Set;
+import java.util.SortedSet;
+import java.util.TreeSet;
+
+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.TableExistsException;
+import org.apache.accumulo.core.client.TableNotFoundException;
+import org.apache.accumulo.core.client.admin.DiskUsage;
+import org.apache.accumulo.core.client.admin.FindMax;
+import org.apache.accumulo.core.client.impl.TableOperationsHelper;
+import org.apache.accumulo.core.client.admin.TimeType;
+import org.apache.accumulo.core.client.impl.Tables;
+import org.apache.accumulo.core.conf.AccumuloConfiguration;
+import org.apache.accumulo.core.data.Key;
+import org.apache.accumulo.core.data.Mutation;
+import org.apache.accumulo.core.data.Range;
+import org.apache.accumulo.core.data.Value;
+import org.apache.accumulo.core.file.FileOperations;
+import org.apache.accumulo.core.file.FileSKVIterator;
+import org.apache.accumulo.core.metadata.MetadataTable;
+import org.apache.accumulo.core.metadata.RootTable;
+import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.core.security.ColumnVisibility;
+import org.apache.accumulo.start.classloader.vfs.AccumuloVFSClassLoader;
+import org.apache.commons.lang.NotImplementedException;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.Text;
+
+class MockTableOperationsImpl extends TableOperationsHelper {
+  private static final byte[] ZERO = {0};
+  private final MockAccumulo acu;
+  private final String username;
+
+  MockTableOperationsImpl(MockAccumulo acu, String username) {
+    this.acu = acu;
+    this.username = username;
+  }
+
+  @Override
+  public SortedSet<String> list() {
+    return new TreeSet<String>(acu.tables.keySet());
+  }
+
+  @Override
+  public boolean exists(String tableName) {
+    return acu.tables.containsKey(tableName);
+  }
+
+  private boolean namespaceExists(String namespace) {
+    return acu.namespaces.containsKey(namespace);
+  }
+
+  @Override
+  public void create(String tableName) throws AccumuloException, AccumuloSecurityException, TableExistsException {
+    create(tableName, true, TimeType.MILLIS);
+  }
+
+  @Override
+  public void create(String tableName, boolean versioningIter) throws AccumuloException, AccumuloSecurityException, TableExistsException {
+    create(tableName, versioningIter, TimeType.MILLIS);
+  }
+
+  @Override
+  public void create(String tableName, boolean versioningIter, TimeType timeType) throws AccumuloException, AccumuloSecurityException, TableExistsException {
+    String namespace = Tables.qualify(tableName).getFirst();
+    if (!tableName.matches(Tables.VALID_NAME_REGEX)) {
+      throw new IllegalArgumentException();
+    }
+    if (exists(tableName))
+      throw new TableExistsException(tableName, tableName, "");
+
+    if (!namespaceExists(namespace)) {
+      throw new IllegalArgumentException("Namespace (" + namespace + ") does not exist, create it first");
+    }
+    acu.createTable(username, tableName, versioningIter, timeType);
+  }
+
+  @Override
+  public void addSplits(String tableName, SortedSet<Text> partitionKeys) throws TableNotFoundException, AccumuloException, AccumuloSecurityException {
+    if (!exists(tableName))
+      throw new TableNotFoundException(tableName, tableName, "");
+    acu.addSplits(tableName, partitionKeys);
+  }
+
+  @Deprecated
+  @Override
+  public Collection<Text> getSplits(String tableName) throws TableNotFoundException {
+    return listSplits(tableName);
+  }
+
+  @Deprecated
+  @Override
+  public Collection<Text> getSplits(String tableName, int maxSplits) throws TableNotFoundException {
+    return listSplits(tableName);
+  }
+
+  @Override
+  public Collection<Text> listSplits(String tableName) throws TableNotFoundException {
+    if (!exists(tableName))
+      throw new TableNotFoundException(tableName, tableName, "");
+    return acu.getSplits(tableName);
+  }
+
+  @Override
+  public Collection<Text> listSplits(String tableName, int maxSplits) throws TableNotFoundException {
+    return listSplits(tableName);
+  }
+
+  @Override
+  public void delete(String tableName) throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
+    if (!exists(tableName))
+      throw new TableNotFoundException(tableName, tableName, "");
+    acu.tables.remove(tableName);
+  }
+
+  @Override
+  public void rename(String oldTableName, String newTableName) throws AccumuloSecurityException, TableNotFoundException, AccumuloException,
+      TableExistsException {
+    if (!exists(oldTableName))
+      throw new TableNotFoundException(oldTableName, oldTableName, "");
+    if (exists(newTableName))
+      throw new TableExistsException(newTableName, newTableName, "");
+    MockTable t = acu.tables.remove(oldTableName);
+    String namespace = Tables.qualify(newTableName).getFirst();
+    MockNamespace n = acu.namespaces.get(namespace);
+    if (n == null) {
+      n = new MockNamespace();
+    }
+    t.setNamespaceName(namespace);
+    t.setNamespace(n);
+    acu.namespaces.put(namespace, n);
+    acu.tables.put(newTableName, t);
+  }
+
+  @Deprecated
+  @Override
+  public void flush(String tableName) throws AccumuloException, AccumuloSecurityException {}
+
+  @Override
+  public void setProperty(String tableName, String property, String value) throws AccumuloException, AccumuloSecurityException {
+    acu.tables.get(tableName).settings.put(property, value);
+  }
+
+  @Override
+  public void removeProperty(String tableName, String property) throws AccumuloException, AccumuloSecurityException {
+    acu.tables.get(tableName).settings.remove(property);
+  }
+
+  @Override
+  public Iterable<Entry<String,String>> getProperties(String tableName) throws TableNotFoundException {
+    String namespace = Tables.qualify(tableName).getFirst();
+    if (!exists(tableName)) {
+      if (!namespaceExists(namespace))
+        throw new TableNotFoundException(tableName, new NamespaceNotFoundException(null, namespace, null));
+      throw new TableNotFoundException(null, tableName, null);
+    }
+
+    Set<Entry<String,String>> props = new HashSet<Entry<String,String>>(acu.namespaces.get(namespace).settings.entrySet());
+
+    Set<Entry<String,String>> tableProps = acu.tables.get(tableName).settings.entrySet();
+    for (Entry<String,String> e : tableProps) {
+      if (props.contains(e)) {
+        props.remove(e);
+      }
+      props.add(e);
+    }
+    return props;
+  }
+
+  @Override
+  public void setLocalityGroups(String tableName, Map<String,Set<Text>> groups) throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
+    if (!exists(tableName))
+      throw new TableNotFoundException(tableName, tableName, "");
+    acu.tables.get(tableName).setLocalityGroups(groups);
+  }
+
+  @Override
+  public Map<String,Set<Text>> getLocalityGroups(String tableName) throws AccumuloException, TableNotFoundException {
+    if (!exists(tableName))
+      throw new TableNotFoundException(tableName, tableName, "");
+    return acu.tables.get(tableName).getLocalityGroups();
+  }
+
+  @Override
+  public Set<Range> splitRangeByTablets(String tableName, Range range, int maxSplits) throws AccumuloException, AccumuloSecurityException,
+      TableNotFoundException {
+    if (!exists(tableName))
+      throw new TableNotFoundException(tableName, tableName, "");
+    return Collections.singleton(range);
+  }
+
+  @Override
+  public void importDirectory(String tableName, String dir, String failureDir, boolean setTime) throws IOException, AccumuloException,
+      AccumuloSecurityException, TableNotFoundException {
+    long time = System.currentTimeMillis();
+    MockTable table = acu.tables.get(tableName);
+    if (table == null) {
+      throw new TableNotFoundException(null, tableName, "The table was not found");
+    }
+    Path importPath = new Path(dir);
+    Path failurePath = new Path(failureDir);
+
+    FileSystem fs = acu.getFileSystem();
+    /*
+     * check preconditions
+     */
+    // directories are directories
+    if (fs.isFile(importPath)) {
+      throw new IOException("Import path must be a directory.");
+    }
+    if (fs.isFile(failurePath)) {
+      throw new IOException("Failure path must be a directory.");
+    }
+    // failures are writable
+    Path createPath = failurePath.suffix("/.createFile");
+    FSDataOutputStream createStream = null;
+    try {
+      createStream = fs.create(createPath);
+    } catch (IOException e) {
+      throw new IOException("Error path is not writable.");
+    } finally {
+      if (createStream != null) {
+        createStream.close();
+      }
+    }
+    fs.delete(createPath, false);
+    // failures are empty
+    FileStatus[] failureChildStats = fs.listStatus(failurePath);
+    if (failureChildStats.length > 0) {
+      throw new IOException("Error path must be empty.");
+    }
+    /*
+     * Begin the import - iterate the files in the path
+     */
+    for (FileStatus importStatus : fs.listStatus(importPath)) {
+      try {
+        FileSKVIterator importIterator = FileOperations.getInstance().openReader(importStatus.getPath().toString(), true, fs, fs.getConf(),
+            AccumuloConfiguration.getDefaultConfiguration());
+        while (importIterator.hasTop()) {
+          Key key = importIterator.getTopKey();
+          Value value = importIterator.getTopValue();
+          if (setTime) {
+            key.setTimestamp(time);
+          }
+          Mutation mutation = new Mutation(key.getRow());
+          if (!key.isDeleted()) {
+            mutation.put(key.getColumnFamily(), key.getColumnQualifier(), new ColumnVisibility(key.getColumnVisibilityData().toArray()), key.getTimestamp(),
+                value);
+          } else {
+            mutation.putDelete(key.getColumnFamily(), key.getColumnQualifier(), new ColumnVisibility(key.getColumnVisibilityData().toArray()),
+                key.getTimestamp());
+          }
+          table.addMutation(mutation);
+          importIterator.next();
+        }
+      } catch (Exception e) {
+        FSDataOutputStream failureWriter = null;
+        DataInputStream failureReader = null;
+        try {
+          failureWriter = fs.create(failurePath.suffix("/" + importStatus.getPath().getName()));
+          failureReader = fs.open(importStatus.getPath());
+          int read = 0;
+          byte[] buffer = new byte[1024];
+          while (-1 != (read = failureReader.read(buffer))) {
+            failureWriter.write(buffer, 0, read);
+          }
+        } finally {
+          if (failureReader != null)
+            failureReader.close();
+          if (failureWriter != null)
+            failureWriter.close();
+        }
+      }
+      fs.delete(importStatus.getPath(), true);
+    }
+  }
+
+  @Override
+  public void offline(String tableName) throws AccumuloSecurityException, AccumuloException, TableNotFoundException {
+    offline(tableName, false);
+  }
+
+  @Override
+  public void offline(String tableName, boolean wait) throws AccumuloSecurityException, AccumuloException, TableNotFoundException {
+    if (!exists(tableName))
+      throw new TableNotFoundException(tableName, tableName, "");
+  }
+
+  @Override
+  public void online(String tableName) throws AccumuloSecurityException, AccumuloException, TableNotFoundException {
+    online(tableName, false);
+  }
+
+  @Override
+  public void online(String tableName, boolean wait) throws AccumuloSecurityException, AccumuloException, TableNotFoundException {
+    if (!exists(tableName))
+      throw new TableNotFoundException(tableName, tableName, "");
+  }
+
+  @Override
+  public void clearLocatorCache(String tableName) throws TableNotFoundException {
+    if (!exists(tableName))
+      throw new TableNotFoundException(tableName, tableName, "");
+  }
+
+  @Override
+  public Map<String,String> tableIdMap() {
+    Map<String,String> result = new HashMap<String,String>();
+    for (String table : acu.tables.keySet()) {
+      if (RootTable.NAME.equals(table))
+        result.put(table, RootTable.ID);
+      else if (MetadataTable.NAME.equals(table))
+        result.put(table, MetadataTable.ID);
+      else
+        result.put(table, table);
+    }
+    return result;
+  }
+
+  @Override
+  public List<DiskUsage> getDiskUsage(Set<String> tables) throws AccumuloException, AccumuloSecurityException {
+
+    List<DiskUsage> diskUsages = new ArrayList<DiskUsage>();
+    diskUsages.add(new DiskUsage(new TreeSet<String>(tables), 0l));
+
+    return diskUsages;
+  }
+
+  @Override
+  public void merge(String tableName, Text start, Text end) throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
+    if (!exists(tableName))
+      throw new TableNotFoundException(tableName, tableName, "");
+    acu.merge(tableName, start, end);
+  }
+
+  @Override
+  public void deleteRows(String tableName, Text start, Text end) throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
+    if (!exists(tableName))
+      throw new TableNotFoundException(tableName, tableName, "");
+    MockTable t = acu.tables.get(tableName);
+    Text startText = new Text(start);
+    Text endText = new Text(end);
+    startText.append(ZERO, 0, 1);
+    endText.append(ZERO, 0, 1);
+    Set<Key> keep = new TreeSet<Key>(t.table.subMap(new Key(startText), new Key(endText)).keySet());
+    t.table.keySet().removeAll(keep);
+  }
+
+  @Override
+  public void compact(String tableName, Text start, Text end, boolean flush, boolean wait) throws AccumuloSecurityException, TableNotFoundException,
+      AccumuloException {
+    if (!exists(tableName))
+      throw new TableNotFoundException(tableName, tableName, "");
+  }
+
+  @Override
+  public void compact(String tableName, Text start, Text end, List<IteratorSetting> iterators, boolean flush, boolean wait) throws AccumuloSecurityException,
+      TableNotFoundException, AccumuloException {
+    if (!exists(tableName))
+      throw new TableNotFoundException(tableName, tableName, "");
+  }
+
+  @Override
+  public void cancelCompaction(String tableName) throws AccumuloSecurityException, TableNotFoundException, AccumuloException {
+    if (!exists(tableName))
+      throw new TableNotFoundException(tableName, tableName, "");
+  }
+
+  @Override
+  public void clone(String srcTableName, String newTableName, boolean flush, Map<String,String> propertiesToSet, Set<String> propertiesToExclude)
+      throws AccumuloException, AccumuloSecurityException, TableNotFoundException, TableExistsException {
+    throw new NotImplementedException();
+  }
+
+  @Override
+  public void flush(String tableName, Text start, Text end, boolean wait) throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
+    if (!exists(tableName))
+      throw new TableNotFoundException(tableName, tableName, "");
+  }
+
+  @Override
+  public Text getMaxRow(String tableName, Authorizations auths, Text startRow, boolean startInclusive, Text endRow, boolean endInclusive)
+      throws TableNotFoundException, AccumuloException, AccumuloSecurityException {
+    MockTable table = acu.tables.get(tableName);
+    if (table == null)
+      throw new TableNotFoundException(tableName, tableName, "no such table");
+
+    return FindMax.findMax(new MockScanner(table, auths), startRow, startInclusive, endRow, endInclusive);
+  }
+
+  @Override
+  public void importTable(String tableName, String exportDir) throws TableExistsException, AccumuloException, AccumuloSecurityException {
+    throw new NotImplementedException();
+  }
+
+  @Override
+  public void exportTable(String tableName, String exportDir) throws TableNotFoundException, AccumuloException, AccumuloSecurityException {
+    throw new NotImplementedException();
+  }
+
+  @Override
+  public boolean testClassLoad(String tableName, String className, String asTypeName) throws AccumuloException, AccumuloSecurityException,
+      TableNotFoundException {
+
+    try {
+      AccumuloVFSClassLoader.loadClass(className, Class.forName(asTypeName));
+    } catch (ClassNotFoundException e) {
+      e.printStackTrace();
+      return false;
+    }
+    return true;
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/f4454a06/core/src/main/java/org/apache/accumulo/core/client/mock/MockTabletLocator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mock/MockTabletLocator.java b/core/src/main/java/org/apache/accumulo/core/client/mock/MockTabletLocator.java
index 6bd01a9..ac6a6e9 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mock/MockTabletLocator.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mock/MockTabletLocator.java
@@ -16,55 +16,38 @@
  */
 package org.apache.accumulo.core.client.mock;
 
-import java.util.Collection;
-import java.util.Collections;
 import java.util.List;
 import java.util.Map;
 
 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.client.impl.TabletLocator;
 import org.apache.accumulo.core.data.KeyExtent;
 import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.data.Range;
 import org.apache.accumulo.core.security.Credentials;
+import org.apache.accumulo.core.security.thrift.TCredentials;
 import org.apache.hadoop.io.Text;
 
-public class MockTabletLocator extends TabletLocator {
+/**
+ * @deprecated since 1.6.0; not intended for public api and you should not use it.
+ */
+@Deprecated
+public class MockTabletLocator extends org.apache.accumulo.core.client.mock.impl.MockTabletLocator {
   public MockTabletLocator() {}
-  
-  @Override
-  public TabletLocation locateTablet(Credentials credentials, Text row, boolean skipRow, boolean retry) throws AccumuloException, AccumuloSecurityException,
+
+  public TabletLocation locateTablet(Text row, boolean skipRow, boolean retry, TCredentials credentials) throws AccumuloException, AccumuloSecurityException,
       TableNotFoundException {
-    throw new UnsupportedOperationException();
+    return locateTablet(Credentials.fromThrift(credentials), row, skipRow, retry);
   }
   
-  @Override
-  public <T extends Mutation> void binMutations(Credentials credentials, List<T> mutations, Map<String,TabletServerMutations<T>> binnedMutations, List<T> failures)
+  public <T extends Mutation> void binMutations(List<T> mutations, Map<String,TabletServerMutations<T>> binnedMutations, List<T> failures, TCredentials credentials)
       throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
-    TabletServerMutations<T> tsm = new TabletServerMutations<T>("5");
-    for (T m : mutations)
-      tsm.addMutation(new KeyExtent(), m);
-    binnedMutations.put("", tsm);
+    binMutations(Credentials.fromThrift(credentials), mutations, binnedMutations, failures);
   }
-  
-  @Override
-  public List<Range> binRanges(Credentials credentials, List<Range> ranges, Map<String,Map<KeyExtent,List<Range>>> binnedRanges) throws AccumuloException,
+
+  public List<Range> binRanges(List<Range> ranges, Map<String,Map<KeyExtent,List<Range>>> binnedRanges, TCredentials credentials) throws AccumuloException,
       AccumuloSecurityException, TableNotFoundException {
-    binnedRanges.put("", Collections.singletonMap(new KeyExtent(new Text(), null, null), ranges));
-    return Collections.emptyList();
+    return binRanges(Credentials.fromThrift(credentials), ranges, binnedRanges);
   }
-  
-  @Override
-  public void invalidateCache(KeyExtent failedExtent) {}
-  
-  @Override
-  public void invalidateCache(Collection<KeyExtent> keySet) {}
-  
-  @Override
-  public void invalidateCache() {}
-  
-  @Override
-  public void invalidateCache(String server) {}
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/f4454a06/core/src/main/java/org/apache/accumulo/core/client/mock/impl/MockTabletLocator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mock/impl/MockTabletLocator.java b/core/src/main/java/org/apache/accumulo/core/client/mock/impl/MockTabletLocator.java
new file mode 100644
index 0000000..35f160f
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/client/mock/impl/MockTabletLocator.java
@@ -0,0 +1,70 @@
+/*
+ * 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.core.client.mock.impl;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+
+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.client.impl.TabletLocator;
+import org.apache.accumulo.core.data.KeyExtent;
+import org.apache.accumulo.core.data.Mutation;
+import org.apache.accumulo.core.data.Range;
+import org.apache.accumulo.core.security.Credentials;
+import org.apache.hadoop.io.Text;
+
+public class MockTabletLocator extends TabletLocator {
+  public MockTabletLocator() {}
+
+  @Override
+  public TabletLocation locateTablet(Credentials credentials, Text row, boolean skipRow, boolean retry) throws AccumuloException, AccumuloSecurityException,
+      TableNotFoundException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public <T extends Mutation> void binMutations(Credentials credentials, List<T> mutations, Map<String,TabletServerMutations<T>> binnedMutations, List<T> failures)
+      throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
+    TabletServerMutations<T> tsm = new TabletServerMutations<T>("5");
+    for (T m : mutations)
+      tsm.addMutation(new KeyExtent(), m);
+    binnedMutations.put("", tsm);
+  }
+
+  @Override
+  public List<Range> binRanges(Credentials credentials, List<Range> ranges, Map<String,Map<KeyExtent,List<Range>>> binnedRanges) throws AccumuloException,
+      AccumuloSecurityException, TableNotFoundException {
+    binnedRanges.put("", Collections.singletonMap(new KeyExtent(new Text(), null, null), ranges));
+    return Collections.emptyList();
+  }
+
+  @Override
+  public void invalidateCache(KeyExtent failedExtent) {}
+
+  @Override
+  public void invalidateCache(Collection<KeyExtent> keySet) {}
+
+  @Override
+  public void invalidateCache() {}
+
+  @Override
+  public void invalidateCache(String server) {}
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/f4454a06/core/src/test/java/org/apache/accumulo/core/client/admin/TableOperationsHelperTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/accumulo/core/client/admin/TableOperationsHelperTest.java b/core/src/test/java/org/apache/accumulo/core/client/admin/TableOperationsHelperTest.java
index 32136a8..1d91574 100644
--- a/core/src/test/java/org/apache/accumulo/core/client/admin/TableOperationsHelperTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/client/admin/TableOperationsHelperTest.java
@@ -19,7 +19,6 @@ package org.apache.accumulo.core.client.admin;
 import java.io.IOException;
 import java.util.Collection;
 import java.util.Collections;
-import java.util.EnumSet;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
@@ -34,14 +33,15 @@ import org.apache.accumulo.core.client.IteratorSetting;
 import org.apache.accumulo.core.client.TableExistsException;
 import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.data.Range;
-import org.apache.accumulo.core.iterators.IteratorUtil.IteratorScope;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.hadoop.io.Text;
-import org.junit.Assert;
-import org.junit.Test;
 
-public class TableOperationsHelperTest {
+/**
+ * This class is left in place specifically to test for regressions against the published version of TableOperationsHelper.
+ */
+public class TableOperationsHelperTest extends org.apache.accumulo.core.client.impl.TableOperationsHelperTest {
   
+  @SuppressWarnings("deprecation")
   static class Tester extends TableOperationsHelper {
     Map<String,Map<String,String>> settings = new HashMap<String,Map<String,String>>();
     
@@ -200,15 +200,6 @@ public class TableOperationsHelperTest {
       return null;
     }
     
-    void check(String tablename, String[] values) {
-      Map<String,String> expected = new TreeMap<String,String>();
-      for (String value : values) {
-        String parts[] = value.split("=", 2);
-        expected.put(parts[0], parts[1]);
-      }
-      Assert.assertEquals(expected, settings.get(tablename));
-    }
-    
     @Override
     public void importTable(String tableName, String exportDir) throws TableExistsException, AccumuloException, AccumuloSecurityException {}
     
@@ -224,72 +215,9 @@ public class TableOperationsHelperTest {
       return false;
     }
   }
-  
-  @Test
-  public void testAttachIterator() throws Exception {
-    Tester t = new Tester();
-    Map<String,String> empty = Collections.emptyMap();
-    t.attachIterator("table", new IteratorSetting(10, "someName", "foo.bar", empty), EnumSet.of(IteratorScope.scan));
-    t.check("table", new String[] {"table.iterator.scan.someName=10,foo.bar",});
-    t.removeIterator("table", "someName", EnumSet.of(IteratorScope.scan));
-    t.check("table", new String[] {});
-    
-    IteratorSetting setting = new IteratorSetting(10, "someName", "foo.bar");
-    setting.addOptions(Collections.singletonMap("key", "value"));
-    t.attachIterator("table", setting, EnumSet.of(IteratorScope.majc));
-    setting = new IteratorSetting(10, "someName", "foo.bar");
-    t.attachIterator("table", setting, EnumSet.of(IteratorScope.scan));
-    t.check("table", new String[] {"table.iterator.majc.someName=10,foo.bar", "table.iterator.majc.someName.opt.key=value",
-        "table.iterator.scan.someName=10,foo.bar",});
-    
-    t.removeIterator("table", "someName", EnumSet.of(IteratorScope.scan));
-    setting = new IteratorSetting(20, "otherName", "some.classname");
-    setting.addOptions(Collections.singletonMap("key", "value"));
-    t.attachIterator("table", setting, EnumSet.of(IteratorScope.majc));
-    setting = new IteratorSetting(20, "otherName", "some.classname");
-    t.attachIterator("table", setting, EnumSet.of(IteratorScope.scan));
-    Map<String,EnumSet<IteratorScope>> two = t.listIterators("table");
-    Assert.assertEquals(2, two.size());
-    Assert.assertTrue(two.containsKey("otherName"));
-    Assert.assertTrue(two.get("otherName").size() == 2);
-    Assert.assertTrue(two.get("otherName").contains(IteratorScope.majc));
-    Assert.assertTrue(two.get("otherName").contains(IteratorScope.scan));
-    Assert.assertTrue(two.containsKey("someName"));
-    Assert.assertTrue(two.get("someName").size() == 1);
-    Assert.assertTrue(two.get("someName").contains(IteratorScope.majc));
-    t.removeIterator("table", "someName", EnumSet.allOf(IteratorScope.class));
-    t.check("table", new String[] {"table.iterator.majc.otherName=20,some.classname", "table.iterator.majc.otherName.opt.key=value",
-        "table.iterator.scan.otherName=20,some.classname",});
-    
-    setting = t.getIteratorSetting("table", "otherName", IteratorScope.scan);
-    Assert.assertEquals(20, setting.getPriority());
-    Assert.assertEquals("some.classname", setting.getIteratorClass());
-    Assert.assertTrue(setting.getOptions().isEmpty());
-    setting = t.getIteratorSetting("table", "otherName", IteratorScope.majc);
-    Assert.assertEquals(20, setting.getPriority());
-    Assert.assertEquals("some.classname", setting.getIteratorClass());
-    Assert.assertFalse(setting.getOptions().isEmpty());
-    Assert.assertEquals(Collections.singletonMap("key", "value"), setting.getOptions());
-    t.attachIterator("table", setting, EnumSet.of(IteratorScope.minc));
-    t.check("table", new String[] {"table.iterator.majc.otherName=20,some.classname", "table.iterator.majc.otherName.opt.key=value",
-        "table.iterator.minc.otherName=20,some.classname", "table.iterator.minc.otherName.opt.key=value", "table.iterator.scan.otherName=20,some.classname",});
-    
-    try {
-      t.attachIterator("table", setting);
-      Assert.fail();
-    } catch (AccumuloException e) {}
-    setting.setName("thirdName");
-    try {
-      t.attachIterator("table", setting);
-      Assert.fail();
-    } catch (AccumuloException e) {}
-    setting.setPriority(10);
-    t.setProperty("table", "table.iterator.minc.thirdName.opt.key", "value");
-    try {
-      t.attachIterator("table", setting);
-      Assert.fail();
-    } catch (AccumuloException e) {}
-    t.removeProperty("table", "table.iterator.minc.thirdName.opt.key");
-    t.attachIterator("table", setting);
+
+  @Override
+  protected org.apache.accumulo.core.client.impl.TableOperationsHelper getHelper() {
+    return new Tester();
   }
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/f4454a06/core/src/test/java/org/apache/accumulo/core/client/impl/TableOperationsHelperTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/accumulo/core/client/impl/TableOperationsHelperTest.java b/core/src/test/java/org/apache/accumulo/core/client/impl/TableOperationsHelperTest.java
new file mode 100644
index 0000000..02838ed
--- /dev/null
+++ b/core/src/test/java/org/apache/accumulo/core/client/impl/TableOperationsHelperTest.java
@@ -0,0 +1,305 @@
+/*
+ * 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.core.client.impl;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.EnumSet;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Set;
+import java.util.SortedSet;
+import java.util.TreeMap;
+
+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.TableExistsException;
+import org.apache.accumulo.core.client.TableNotFoundException;
+import org.apache.accumulo.core.client.admin.DiskUsage;
+import org.apache.accumulo.core.client.admin.TimeType;
+import org.apache.accumulo.core.data.Range;
+import org.apache.accumulo.core.iterators.IteratorUtil.IteratorScope;
+import org.apache.accumulo.core.security.Authorizations;
+import org.apache.hadoop.io.Text;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class TableOperationsHelperTest {
+
+  static class Tester extends TableOperationsHelper {
+    Map<String,Map<String,String>> settings = new HashMap<String,Map<String,String>>();
+
+    @Override
+    public SortedSet<String> list() {
+      return null;
+    }
+
+    @Override
+    public boolean exists(String tableName) {
+      return true;
+    }
+
+    @Override
+    public void create(String tableName) throws AccumuloException, AccumuloSecurityException, TableExistsException {}
+
+    @Override
+    public void create(String tableName, boolean limitVersion) throws AccumuloException, AccumuloSecurityException, TableExistsException {
+      create(tableName, limitVersion, TimeType.MILLIS);
+    }
+
+    @Override
+    public void create(String tableName, boolean versioningIter, TimeType timeType) throws AccumuloException, AccumuloSecurityException, TableExistsException {}
+
+    @Override
+    public void addSplits(String tableName, SortedSet<Text> partitionKeys) throws TableNotFoundException, AccumuloException, AccumuloSecurityException {}
+
+    @Deprecated
+    @Override
+    public Collection<Text> getSplits(String tableName) throws TableNotFoundException {
+      return null;
+    }
+
+    @Deprecated
+    @Override
+    public Collection<Text> getSplits(String tableName, int maxSplits) throws TableNotFoundException {
+      return null;
+    }
+
+    @Override
+    public Collection<Text> listSplits(String tableName) throws TableNotFoundException {
+      return null;
+    }
+
+    @Override
+    public Collection<Text> listSplits(String tableName, int maxSplits) throws TableNotFoundException {
+      return null;
+    }
+
+    @Override
+    public Text getMaxRow(String tableName, Authorizations auths, Text startRow, boolean startInclusive, Text endRow, boolean endInclusive)
+        throws TableNotFoundException, AccumuloException, AccumuloSecurityException {
+      return null;
+    }
+
+    @Override
+    public void merge(String tableName, Text start, Text end) throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
+
+    }
+
+    @Override
+    public void deleteRows(String tableName, Text start, Text end) throws AccumuloException, AccumuloSecurityException, TableNotFoundException {}
+
+    @Override
+    public void compact(String tableName, Text start, Text end, boolean flush, boolean wait) throws AccumuloSecurityException, TableNotFoundException,
+        AccumuloException {}
+
+    @Override
+    public void compact(String tableName, Text start, Text end, List<IteratorSetting> iterators, boolean flush, boolean wait) throws AccumuloSecurityException,
+        TableNotFoundException, AccumuloException {}
+
+    @Override
+    public void delete(String tableName) throws AccumuloException, AccumuloSecurityException, TableNotFoundException {}
+
+    @Override
+    public void clone(String srcTableName, String newTableName, boolean flush, Map<String,String> propertiesToSet, Set<String> propertiesToExclude)
+        throws AccumuloException, AccumuloSecurityException, TableNotFoundException, TableExistsException {}
+
+    @Override
+    public void rename(String oldTableName, String newTableName) throws AccumuloSecurityException, TableNotFoundException, AccumuloException,
+        TableExistsException {}
+
+    @Deprecated
+    @Override
+    public void flush(String tableName) throws AccumuloException, AccumuloSecurityException {}
+
+    @Override
+    public void flush(String tableName, Text start, Text end, boolean wait) throws AccumuloException, AccumuloSecurityException, TableNotFoundException {}
+
+    @Override
+    public void setProperty(String tableName, String property, String value) throws AccumuloException, AccumuloSecurityException {
+      if (!settings.containsKey(tableName))
+        settings.put(tableName, new TreeMap<String,String>());
+      settings.get(tableName).put(property, value);
+    }
+
+    @Override
+    public void removeProperty(String tableName, String property) throws AccumuloException, AccumuloSecurityException {
+      if (!settings.containsKey(tableName))
+        return;
+      settings.get(tableName).remove(property);
+    }
+
+    @Override
+    public Iterable<Entry<String,String>> getProperties(String tableName) throws AccumuloException, TableNotFoundException {
+      Map<String,String> empty = Collections.emptyMap();
+      if (!settings.containsKey(tableName))
+        return empty.entrySet();
+      return settings.get(tableName).entrySet();
+    }
+
+    @Override
+    public void setLocalityGroups(String tableName, Map<String,Set<Text>> groups) throws AccumuloException, AccumuloSecurityException, TableNotFoundException {}
+
+    @Override
+    public Map<String,Set<Text>> getLocalityGroups(String tableName) throws AccumuloException, TableNotFoundException {
+      return null;
+    }
+
+    @Override
+    public Set<Range> splitRangeByTablets(String tableName, Range range, int maxSplits) throws AccumuloException, AccumuloSecurityException,
+        TableNotFoundException {
+      return null;
+    }
+
+    @Override
+    public void importDirectory(String tableName, String dir, String failureDir, boolean setTime) throws TableNotFoundException, IOException,
+        AccumuloException, AccumuloSecurityException {}
+
+    @Override
+    public void offline(String tableName) throws AccumuloSecurityException, AccumuloException, TableNotFoundException {
+
+    }
+
+    @Override
+    public void online(String tableName) throws AccumuloSecurityException, AccumuloException, TableNotFoundException {}
+
+    @Override
+    public void offline(String tableName, boolean wait) throws AccumuloSecurityException, AccumuloException, TableNotFoundException {
+
+    }
+
+    @Override
+    public void online(String tableName, boolean wait) throws AccumuloSecurityException, AccumuloException, TableNotFoundException {}
+
+    @Override
+    public void clearLocatorCache(String tableName) throws TableNotFoundException {}
+
+    @Override
+    public Map<String,String> tableIdMap() {
+      return null;
+    }
+
+    @Override
+    public List<DiskUsage> getDiskUsage(Set<String> tables) throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
+      return null;
+    }
+
+    @Override
+    public void importTable(String tableName, String exportDir) throws TableExistsException, AccumuloException, AccumuloSecurityException {}
+
+    @Override
+    public void exportTable(String tableName, String exportDir) throws TableNotFoundException, AccumuloException, AccumuloSecurityException {}
+
+    @Override
+    public void cancelCompaction(String tableName) throws AccumuloSecurityException, TableNotFoundException, AccumuloException {}
+
+    @Override
+    public boolean testClassLoad(String tableName, String className, String asTypeName) throws AccumuloException, AccumuloSecurityException,
+        TableNotFoundException {
+      return false;
+    }
+  }
+
+  protected TableOperationsHelper getHelper() {
+    return new Tester();
+  }
+
+  void check(TableOperationsHelper t, String tablename, String[] values) throws Exception {
+      Map<String,String> expected = new TreeMap<String,String>();
+      for (String value : values) {
+        String parts[] = value.split("=", 2);
+        expected.put(parts[0], parts[1]);
+      }
+      Map<String,String> actual = new TreeMap<String,String>();
+      for (Entry<String,String> entry : t.getProperties(tablename)) {
+        actual.put(entry.getKey(), entry.getValue());
+      }
+      Assert.assertEquals(expected, actual);
+  }
+
+  @Test
+  public void testAttachIterator() throws Exception {
+    TableOperationsHelper t = getHelper();
+    Map<String,String> empty = Collections.emptyMap();
+    t.attachIterator("table", new IteratorSetting(10, "someName", "foo.bar", empty), EnumSet.of(IteratorScope.scan));
+    check(t, "table", new String[] {"table.iterator.scan.someName=10,foo.bar",});
+    t.removeIterator("table", "someName", EnumSet.of(IteratorScope.scan));
+    check(t, "table", new String[] {});
+
+    IteratorSetting setting = new IteratorSetting(10, "someName", "foo.bar");
+    setting.addOptions(Collections.singletonMap("key", "value"));
+    t.attachIterator("table", setting, EnumSet.of(IteratorScope.majc));
+    setting = new IteratorSetting(10, "someName", "foo.bar");
+    t.attachIterator("table", setting, EnumSet.of(IteratorScope.scan));
+    check(t, "table", new String[] {"table.iterator.majc.someName=10,foo.bar", "table.iterator.majc.someName.opt.key=value",
+        "table.iterator.scan.someName=10,foo.bar",});
+
+    t.removeIterator("table", "someName", EnumSet.of(IteratorScope.scan));
+    setting = new IteratorSetting(20, "otherName", "some.classname");
+    setting.addOptions(Collections.singletonMap("key", "value"));
+    t.attachIterator("table", setting, EnumSet.of(IteratorScope.majc));
+    setting = new IteratorSetting(20, "otherName", "some.classname");
+    t.attachIterator("table", setting, EnumSet.of(IteratorScope.scan));
+    Map<String,EnumSet<IteratorScope>> two = t.listIterators("table");
+    Assert.assertEquals(2, two.size());
+    Assert.assertTrue(two.containsKey("otherName"));
+    Assert.assertTrue(two.get("otherName").size() == 2);
+    Assert.assertTrue(two.get("otherName").contains(IteratorScope.majc));
+    Assert.assertTrue(two.get("otherName").contains(IteratorScope.scan));
+    Assert.assertTrue(two.containsKey("someName"));
+    Assert.assertTrue(two.get("someName").size() == 1);
+    Assert.assertTrue(two.get("someName").contains(IteratorScope.majc));
+    t.removeIterator("table", "someName", EnumSet.allOf(IteratorScope.class));
+    check(t, "table", new String[] {"table.iterator.majc.otherName=20,some.classname", "table.iterator.majc.otherName.opt.key=value",
+        "table.iterator.scan.otherName=20,some.classname",});
+
+    setting = t.getIteratorSetting("table", "otherName", IteratorScope.scan);
+    Assert.assertEquals(20, setting.getPriority());
+    Assert.assertEquals("some.classname", setting.getIteratorClass());
+    Assert.assertTrue(setting.getOptions().isEmpty());
+    setting = t.getIteratorSetting("table", "otherName", IteratorScope.majc);
+    Assert.assertEquals(20, setting.getPriority());
+    Assert.assertEquals("some.classname", setting.getIteratorClass());
+    Assert.assertFalse(setting.getOptions().isEmpty());
+    Assert.assertEquals(Collections.singletonMap("key", "value"), setting.getOptions());
+    t.attachIterator("table", setting, EnumSet.of(IteratorScope.minc));
+    check(t, "table", new String[] {"table.iterator.majc.otherName=20,some.classname", "table.iterator.majc.otherName.opt.key=value",
+        "table.iterator.minc.otherName=20,some.classname", "table.iterator.minc.otherName.opt.key=value", "table.iterator.scan.otherName=20,some.classname",});
+
+    try {
+      t.attachIterator("table", setting);
+      Assert.fail();
+    } catch (AccumuloException e) {}
+    setting.setName("thirdName");
+    try {
+      t.attachIterator("table", setting);
+      Assert.fail();
+    } catch (AccumuloException e) {}
+    setting.setPriority(10);
+    t.setProperty("table", "table.iterator.minc.thirdName.opt.key", "value");
+    try {
+      t.attachIterator("table", setting);
+      Assert.fail();
+    } catch (AccumuloException e) {}
+    t.removeProperty("table", "table.iterator.minc.thirdName.opt.key");
+    t.attachIterator("table", setting);
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/f4454a06/server/base/src/main/java/org/apache/accumulo/server/security/SecurityOperation.java
----------------------------------------------------------------------
diff --git a/server/base/src/main/java/org/apache/accumulo/server/security/SecurityOperation.java b/server/base/src/main/java/org/apache/accumulo/server/security/SecurityOperation.java
index 2f9e4d3..d61dd30 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/security/SecurityOperation.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/security/SecurityOperation.java
@@ -25,7 +25,7 @@ import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.NamespaceNotFoundException;
 import org.apache.accumulo.core.client.TableNotFoundException;
-import org.apache.accumulo.core.client.admin.SecurityOperationsImpl;
+import org.apache.accumulo.core.client.impl.SecurityOperationsImpl;
 import org.apache.accumulo.core.client.impl.Namespaces;
 import org.apache.accumulo.core.client.impl.thrift.SecurityErrorCode;
 import org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/f4454a06/server/master/src/main/java/org/apache/accumulo/master/FateServiceHandler.java
----------------------------------------------------------------------
diff --git a/server/master/src/main/java/org/apache/accumulo/master/FateServiceHandler.java b/server/master/src/main/java/org/apache/accumulo/master/FateServiceHandler.java
index d63a63e..cd522f5 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/FateServiceHandler.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/FateServiceHandler.java
@@ -26,7 +26,7 @@ import java.util.Set;
 
 import org.apache.accumulo.core.client.IteratorSetting;
 import org.apache.accumulo.core.client.NamespaceNotFoundException;
-import org.apache.accumulo.core.client.admin.TableOperationsImpl;
+import org.apache.accumulo.core.client.impl.TableOperationsImpl;
 import org.apache.accumulo.core.client.admin.TimeType;
 import org.apache.accumulo.core.client.impl.Namespaces;
 import org.apache.accumulo.core.client.impl.Tables;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/f4454a06/server/master/src/main/java/org/apache/accumulo/master/tableOps/ImportTable.java
----------------------------------------------------------------------
diff --git a/server/master/src/main/java/org/apache/accumulo/master/tableOps/ImportTable.java b/server/master/src/main/java/org/apache/accumulo/master/tableOps/ImportTable.java
index cd59b78..031ba9c 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/tableOps/ImportTable.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/tableOps/ImportTable.java
@@ -34,7 +34,7 @@ import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.BatchWriter;
 import org.apache.accumulo.core.client.BatchWriterConfig;
 import org.apache.accumulo.core.client.Instance;
-import org.apache.accumulo.core.client.admin.TableOperationsImpl;
+import org.apache.accumulo.core.client.impl.TableOperationsImpl;
 import org.apache.accumulo.core.client.impl.Namespaces;
 import org.apache.accumulo.core.client.impl.Tables;
 import org.apache.accumulo.core.client.impl.thrift.TableOperation;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/f4454a06/test/compat/japi-compliance/japi-accumulo-1.5.xml
----------------------------------------------------------------------
diff --git a/test/compat/japi-compliance/japi-accumulo-1.5.xml b/test/compat/japi-compliance/japi-accumulo-1.5.xml
index 9e6f47f..f49dbb5 100644
--- a/test/compat/japi-compliance/japi-accumulo-1.5.xml
+++ b/test/compat/japi-compliance/japi-accumulo-1.5.xml
@@ -25,7 +25,6 @@
 
 <skip_packages>
   org.apache.accumulo.core.client.impl
-  org.apache.accumulo.core.client.mock
   org.apache.accumulo.core.data.thrift
 </skip_packages>
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/f4454a06/test/compat/japi-compliance/japi-accumulo-1.6.xml
----------------------------------------------------------------------
diff --git a/test/compat/japi-compliance/japi-accumulo-1.6.xml b/test/compat/japi-compliance/japi-accumulo-1.6.xml
index 36553b8..0403a96 100644
--- a/test/compat/japi-compliance/japi-accumulo-1.6.xml
+++ b/test/compat/japi-compliance/japi-accumulo-1.6.xml
@@ -25,7 +25,8 @@
 
 <skip_packages>
   org.apache.accumulo.core.client.impl
-  org.apache.accumulo.core.client.mock
+  org.apache.accumulo.core.client.lexicoders.impl
+  org.apache.accumulo.core.client.mapreduce.lib.impl
   org.apache.accumulo.core.data.thrift
   org.apache.accumulo.minicluster.impl
 </skip_packages>


[03/17] ACCUMULO-2726 Adds back missing methods for binary backwards compat with non-Deprecated methods from 1.5.0. Marks things that we're for sure removing with @Deprecated; mostly things that shouldn't have been in public to begin with.

Posted by bu...@apache.org.
http://git-wip-us.apache.org/repos/asf/accumulo/blob/f4454a06/core/src/main/java/org/apache/accumulo/core/client/impl/NamespaceOperationsHelper.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/NamespaceOperationsHelper.java b/core/src/main/java/org/apache/accumulo/core/client/impl/NamespaceOperationsHelper.java
new file mode 100644
index 0000000..2dac1fa
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/NamespaceOperationsHelper.java
@@ -0,0 +1,214 @@
+/*
+ * 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.core.client.impl;
+
+import java.util.EnumSet;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.TreeMap;
+import java.util.TreeSet;
+
+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.admin.NamespaceOperations;
+import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.core.iterators.IteratorUtil.IteratorScope;
+
+public abstract class NamespaceOperationsHelper implements NamespaceOperations {
+
+  @Override
+  public String systemNamespace() {
+    return Namespaces.ACCUMULO_NAMESPACE;
+  }
+
+  @Override
+  public String defaultNamespace() {
+    return Namespaces.DEFAULT_NAMESPACE;
+  }
+
+  @Override
+  public void attachIterator(String namespace, IteratorSetting setting) throws AccumuloSecurityException, AccumuloException, NamespaceNotFoundException {
+    attachIterator(namespace, setting, EnumSet.allOf(IteratorScope.class));
+  }
+
+  @Override
+  public void attachIterator(String namespace, IteratorSetting setting, EnumSet<IteratorScope> scopes) throws AccumuloSecurityException, AccumuloException,
+      NamespaceNotFoundException {
+    checkIteratorConflicts(namespace, setting, scopes);
+    for (IteratorScope scope : scopes) {
+      String root = String.format("%s%s.%s", Property.TABLE_ITERATOR_PREFIX, scope.name().toLowerCase(), setting.getName());
+      for (Entry<String,String> prop : setting.getOptions().entrySet()) {
+        this.setProperty(namespace, root + ".opt." + prop.getKey(), prop.getValue());
+      }
+      this.setProperty(namespace, root, setting.getPriority() + "," + setting.getIteratorClass());
+    }
+  }
+
+  @Override
+  public void removeIterator(String namespace, String name, EnumSet<IteratorScope> scopes) throws AccumuloSecurityException, AccumuloException,
+      NamespaceNotFoundException {
+    if (!exists(namespace))
+      throw new NamespaceNotFoundException(null, namespace, null);
+    Map<String,String> copy = new TreeMap<String,String>();
+    for (Entry<String,String> property : this.getProperties(namespace)) {
+      copy.put(property.getKey(), property.getValue());
+    }
+    for (IteratorScope scope : scopes) {
+      String root = String.format("%s%s.%s", Property.TABLE_ITERATOR_PREFIX, scope.name().toLowerCase(), name);
+      for (Entry<String,String> property : copy.entrySet()) {
+        if (property.getKey().equals(root) || property.getKey().startsWith(root + ".opt."))
+          this.removeProperty(namespace, property.getKey());
+      }
+    }
+  }
+
+  @Override
+  public IteratorSetting getIteratorSetting(String namespace, String name, IteratorScope scope) throws AccumuloSecurityException, AccumuloException,
+      NamespaceNotFoundException {
+    if (!exists(namespace))
+      throw new NamespaceNotFoundException(null, namespace, null);
+    int priority = -1;
+    String classname = null;
+    Map<String,String> settings = new HashMap<String,String>();
+
+    String root = String.format("%s%s.%s", Property.TABLE_ITERATOR_PREFIX, scope.name().toLowerCase(), name);
+    String opt = root + ".opt.";
+    for (Entry<String,String> property : this.getProperties(namespace)) {
+      if (property.getKey().equals(root)) {
+        String parts[] = property.getValue().split(",");
+        if (parts.length != 2) {
+          throw new AccumuloException("Bad value for iterator setting: " + property.getValue());
+        }
+        priority = Integer.parseInt(parts[0]);
+        classname = parts[1];
+      } else if (property.getKey().startsWith(opt)) {
+        settings.put(property.getKey().substring(opt.length()), property.getValue());
+      }
+    }
+    if (priority <= 0 || classname == null) {
+      return null;
+    }
+    return new IteratorSetting(priority, name, classname, settings);
+  }
+
+  @Override
+  public Map<String,EnumSet<IteratorScope>> listIterators(String namespace) throws AccumuloSecurityException, AccumuloException, NamespaceNotFoundException {
+    if (!exists(namespace))
+      throw new NamespaceNotFoundException(null, namespace, null);
+    Map<String,EnumSet<IteratorScope>> result = new TreeMap<String,EnumSet<IteratorScope>>();
+    for (Entry<String,String> property : this.getProperties(namespace)) {
+      String name = property.getKey();
+      String[] parts = name.split("\\.");
+      if (parts.length == 4) {
+        if (parts[0].equals("table") && parts[1].equals("iterator")) {
+          IteratorScope scope = IteratorScope.valueOf(parts[2]);
+          if (!result.containsKey(parts[3]))
+            result.put(parts[3], EnumSet.noneOf(IteratorScope.class));
+          result.get(parts[3]).add(scope);
+        }
+      }
+    }
+    return result;
+  }
+
+  @Override
+  public void checkIteratorConflicts(String namespace, IteratorSetting setting, EnumSet<IteratorScope> scopes) throws AccumuloException,
+      NamespaceNotFoundException, AccumuloSecurityException {
+    if (!exists(namespace))
+      throw new NamespaceNotFoundException(null, namespace, null);
+    for (IteratorScope scope : scopes) {
+      String scopeStr = String.format("%s%s", Property.TABLE_ITERATOR_PREFIX, scope.name().toLowerCase());
+      String nameStr = String.format("%s.%s", scopeStr, setting.getName());
+      String optStr = String.format("%s.opt.", nameStr);
+      Map<String,String> optionConflicts = new TreeMap<String,String>();
+      for (Entry<String,String> property : this.getProperties(namespace)) {
+        if (property.getKey().startsWith(scopeStr)) {
+          if (property.getKey().equals(nameStr))
+            throw new AccumuloException(new IllegalArgumentException("iterator name conflict for " + setting.getName() + ": " + property.getKey() + "="
+                + property.getValue()));
+          if (property.getKey().startsWith(optStr))
+            optionConflicts.put(property.getKey(), property.getValue());
+          if (property.getKey().contains(".opt."))
+            continue;
+          String parts[] = property.getValue().split(",");
+          if (parts.length != 2)
+            throw new AccumuloException("Bad value for existing iterator setting: " + property.getKey() + "=" + property.getValue());
+          try {
+            if (Integer.parseInt(parts[0]) == setting.getPriority())
+              throw new AccumuloException(new IllegalArgumentException("iterator priority conflict: " + property.getKey() + "=" + property.getValue()));
+          } catch (NumberFormatException e) {
+            throw new AccumuloException("Bad value for existing iterator setting: " + property.getKey() + "=" + property.getValue());
+          }
+        }
+      }
+      if (optionConflicts.size() > 0)
+        throw new AccumuloException(new IllegalArgumentException("iterator options conflict for " + setting.getName() + ": " + optionConflicts));
+    }
+  }
+
+  @Override
+  public int addConstraint(String namespace, String constraintClassName) throws AccumuloException, AccumuloSecurityException, NamespaceNotFoundException {
+    TreeSet<Integer> constraintNumbers = new TreeSet<Integer>();
+    TreeMap<String,Integer> constraintClasses = new TreeMap<String,Integer>();
+    int i;
+    for (Entry<String,String> property : this.getProperties(namespace)) {
+      if (property.getKey().startsWith(Property.TABLE_CONSTRAINT_PREFIX.toString())) {
+        try {
+          i = Integer.parseInt(property.getKey().substring(Property.TABLE_CONSTRAINT_PREFIX.toString().length()));
+        } catch (NumberFormatException e) {
+          throw new AccumuloException("Bad key for existing constraint: " + property.toString());
+        }
+        constraintNumbers.add(i);
+        constraintClasses.put(property.getValue(), i);
+      }
+    }
+    i = 1;
+    while (constraintNumbers.contains(i))
+      i++;
+    if (constraintClasses.containsKey(constraintClassName))
+      throw new AccumuloException("Constraint " + constraintClassName + " already exists for namespace " + namespace + " with number "
+          + constraintClasses.get(constraintClassName));
+    this.setProperty(namespace, Property.TABLE_CONSTRAINT_PREFIX.toString() + i, constraintClassName);
+    return i;
+  }
+
+  @Override
+  public void removeConstraint(String namespace, int number) throws AccumuloException, AccumuloSecurityException, NamespaceNotFoundException {
+    this.removeProperty(namespace, Property.TABLE_CONSTRAINT_PREFIX.toString() + number);
+  }
+
+  @Override
+  public Map<String,Integer> listConstraints(String namespace) throws AccumuloException, NamespaceNotFoundException, AccumuloSecurityException {
+    Map<String,Integer> constraints = new TreeMap<String,Integer>();
+    for (Entry<String,String> property : this.getProperties(namespace)) {
+      if (property.getKey().startsWith(Property.TABLE_CONSTRAINT_PREFIX.toString())) {
+        if (constraints.containsKey(property.getValue()))
+          throw new AccumuloException("Same constraint configured twice: " + property.getKey() + "=" + Property.TABLE_CONSTRAINT_PREFIX
+              + constraints.get(property.getValue()) + "=" + property.getKey());
+        try {
+          constraints.put(property.getValue(), Integer.parseInt(property.getKey().substring(Property.TABLE_CONSTRAINT_PREFIX.toString().length())));
+        } catch (NumberFormatException e) {
+          throw new AccumuloException("Bad key for existing constraint: " + property.toString());
+        }
+      }
+    }
+    return constraints;
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/f4454a06/core/src/main/java/org/apache/accumulo/core/client/impl/NamespaceOperationsImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/NamespaceOperationsImpl.java b/core/src/main/java/org/apache/accumulo/core/client/impl/NamespaceOperationsImpl.java
new file mode 100644
index 0000000..2af187f
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/NamespaceOperationsImpl.java
@@ -0,0 +1,244 @@
+/*
+ * 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.core.client.impl;
+
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.EnumSet;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.SortedSet;
+import java.util.TreeSet;
+
+import org.apache.accumulo.core.client.AccumuloException;
+import org.apache.accumulo.core.client.AccumuloSecurityException;
+import org.apache.accumulo.core.client.Instance;
+import org.apache.accumulo.core.client.IteratorSetting;
+import org.apache.accumulo.core.client.NamespaceExistsException;
+import org.apache.accumulo.core.client.NamespaceNotEmptyException;
+import org.apache.accumulo.core.client.NamespaceNotFoundException;
+import org.apache.accumulo.core.client.admin.TableOperations;
+import org.apache.accumulo.core.client.TableExistsException;
+import org.apache.accumulo.core.client.TableNotFoundException;
+import org.apache.accumulo.core.client.impl.thrift.ClientService;
+import org.apache.accumulo.core.client.impl.thrift.SecurityErrorCode;
+import org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException;
+import org.apache.accumulo.core.client.impl.thrift.ThriftTableOperationException;
+import org.apache.accumulo.core.constraints.Constraint;
+import org.apache.accumulo.core.iterators.IteratorUtil.IteratorScope;
+import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
+import org.apache.accumulo.core.master.thrift.FateOperation;
+import org.apache.accumulo.core.master.thrift.MasterClientService;
+import org.apache.accumulo.core.security.Credentials;
+import org.apache.accumulo.core.util.ArgumentChecker;
+import org.apache.accumulo.core.util.OpTimer;
+import org.apache.accumulo.trace.instrument.Tracer;
+import org.apache.log4j.Level;
+import org.apache.log4j.Logger;
+
+public class NamespaceOperationsImpl extends NamespaceOperationsHelper {
+  private Instance instance;
+  private Credentials credentials;
+  private TableOperationsImpl tableOps;
+
+  private static final Logger log = Logger.getLogger(TableOperations.class);
+
+  public NamespaceOperationsImpl(Instance instance, Credentials credentials, TableOperationsImpl tableOps) {
+    ArgumentChecker.notNull(instance, credentials);
+    this.instance = instance;
+    this.credentials = credentials;
+    this.tableOps = tableOps;
+  }
+
+  @Override
+  public SortedSet<String> list() {
+    OpTimer opTimer = new OpTimer(log, Level.TRACE).start("Fetching list of namespaces...");
+    TreeSet<String> namespaces = new TreeSet<String>(Namespaces.getNameToIdMap(instance).keySet());
+    opTimer.stop("Fetched " + namespaces.size() + " namespaces in %DURATION%");
+    return namespaces;
+  }
+
+  @Override
+  public boolean exists(String namespace) {
+    ArgumentChecker.notNull(namespace);
+
+    OpTimer opTimer = new OpTimer(log, Level.TRACE).start("Checking if namespace " + namespace + " exists...");
+    boolean exists = Namespaces.getNameToIdMap(instance).containsKey(namespace);
+    opTimer.stop("Checked existance of " + exists + " in %DURATION%");
+    return exists;
+  }
+
+  @Override
+  public void create(String namespace) throws AccumuloException, AccumuloSecurityException, NamespaceExistsException {
+    ArgumentChecker.notNull(namespace);
+
+    try {
+      doNamespaceFateOperation(FateOperation.NAMESPACE_CREATE, Arrays.asList(ByteBuffer.wrap(namespace.getBytes())), Collections.<String,String> emptyMap());
+    } catch (NamespaceNotFoundException e) {
+      // should not happen
+      throw new AssertionError(e);
+    }
+  }
+
+  @Override
+  public void delete(String namespace) throws AccumuloException, AccumuloSecurityException, NamespaceNotFoundException, NamespaceNotEmptyException {
+    ArgumentChecker.notNull(namespace);
+    String namespaceId = Namespaces.getNamespaceId(instance, namespace);
+
+    if (namespaceId.equals(Namespaces.ACCUMULO_NAMESPACE_ID) || namespaceId.equals(Namespaces.DEFAULT_NAMESPACE_ID)) {
+      log.debug(credentials.getPrincipal() + " attempted to delete the " + namespaceId + " namespace");
+      throw new AccumuloSecurityException(credentials.getPrincipal(), SecurityErrorCode.UNSUPPORTED_OPERATION);
+    }
+
+    if (Namespaces.getTableIds(instance, namespaceId).size() > 0) {
+      throw new NamespaceNotEmptyException(namespaceId, namespace, null);
+    }
+
+    List<ByteBuffer> args = Arrays.asList(ByteBuffer.wrap(namespace.getBytes()));
+    Map<String,String> opts = new HashMap<String,String>();
+
+    try {
+      doNamespaceFateOperation(FateOperation.NAMESPACE_DELETE, args, opts);
+    } catch (NamespaceExistsException e) {
+      // should not happen
+      throw new AssertionError(e);
+    }
+
+  }
+
+  @Override
+  public void rename(String oldNamespaceName, String newNamespaceName) throws AccumuloSecurityException, NamespaceNotFoundException, AccumuloException,
+      NamespaceExistsException {
+
+    List<ByteBuffer> args = Arrays.asList(ByteBuffer.wrap(oldNamespaceName.getBytes()), ByteBuffer.wrap(newNamespaceName.getBytes()));
+    Map<String,String> opts = new HashMap<String,String>();
+    doNamespaceFateOperation(FateOperation.NAMESPACE_RENAME, args, opts);
+  }
+
+  @Override
+  public void setProperty(final String namespace, final String property, final String value) throws AccumuloException, AccumuloSecurityException,
+      NamespaceNotFoundException {
+    ArgumentChecker.notNull(namespace, property, value);
+
+    MasterClient.executeNamespace(instance, new ClientExec<MasterClientService.Client>() {
+      @Override
+      public void execute(MasterClientService.Client client) throws Exception {
+        client.setNamespaceProperty(Tracer.traceInfo(), credentials.toThrift(instance), namespace, property, value);
+      }
+    });
+  }
+
+  @Override
+  public void removeProperty(final String namespace, final String property) throws AccumuloException, AccumuloSecurityException, NamespaceNotFoundException {
+    ArgumentChecker.notNull(namespace, property);
+
+    MasterClient.executeNamespace(instance, new ClientExec<MasterClientService.Client>() {
+      @Override
+      public void execute(MasterClientService.Client client) throws Exception {
+        client.removeNamespaceProperty(Tracer.traceInfo(), credentials.toThrift(instance), namespace, property);
+      }
+    });
+  }
+
+  @Override
+  public Iterable<Entry<String,String>> getProperties(final String namespace) throws AccumuloException, NamespaceNotFoundException {
+    ArgumentChecker.notNull(namespace);
+    try {
+      return ServerClient.executeRaw(instance, new ClientExecReturn<Map<String,String>,ClientService.Client>() {
+        @Override
+        public Map<String,String> execute(ClientService.Client client) throws Exception {
+          return client.getNamespaceConfiguration(Tracer.traceInfo(), credentials.toThrift(instance), namespace);
+        }
+      }).entrySet();
+    } catch (ThriftTableOperationException e) {
+      switch (e.getType()) {
+        case NAMESPACE_NOTFOUND:
+          throw new NamespaceNotFoundException(e);
+        case OTHER:
+        default:
+          throw new AccumuloException(e.description, e);
+      }
+    } catch (AccumuloException e) {
+      throw e;
+    } catch (Exception e) {
+      throw new AccumuloException(e);
+    }
+
+  }
+
+  @Override
+  public Map<String,String> namespaceIdMap() {
+    return Namespaces.getNameToIdMap(instance);
+  }
+
+  @Override
+  public boolean testClassLoad(final String namespace, final String className, final String asTypeName) throws NamespaceNotFoundException, AccumuloException,
+      AccumuloSecurityException {
+    ArgumentChecker.notNull(namespace, className, asTypeName);
+
+    try {
+      return ServerClient.executeRaw(instance, new ClientExecReturn<Boolean,ClientService.Client>() {
+        @Override
+        public Boolean execute(ClientService.Client client) throws Exception {
+          return client.checkNamespaceClass(Tracer.traceInfo(), credentials.toThrift(instance), namespace, className, asTypeName);
+        }
+      });
+    } catch (ThriftTableOperationException e) {
+      switch (e.getType()) {
+        case NAMESPACE_NOTFOUND:
+          throw new NamespaceNotFoundException(e);
+        default:
+          throw new AccumuloException(e.description, e);
+      }
+    } catch (ThriftSecurityException e) {
+      throw new AccumuloSecurityException(e.user, e.code, e);
+    } catch (AccumuloException e) {
+      throw e;
+    } catch (Exception e) {
+      throw new AccumuloException(e);
+    }
+  }
+
+  @Override
+  public void attachIterator(String namespace, IteratorSetting setting, EnumSet<IteratorScope> scopes) throws AccumuloSecurityException, AccumuloException,
+      NamespaceNotFoundException {
+    testClassLoad(namespace, setting.getIteratorClass(), SortedKeyValueIterator.class.getName());
+    super.attachIterator(namespace, setting, scopes);
+  }
+
+  @Override
+  public int addConstraint(String namespace, String constraintClassName) throws AccumuloException, AccumuloSecurityException, NamespaceNotFoundException {
+    testClassLoad(namespace, constraintClassName, Constraint.class.getName());
+    return super.addConstraint(namespace, constraintClassName);
+  }
+
+  private String doNamespaceFateOperation(FateOperation op, List<ByteBuffer> args, Map<String,String> opts) throws AccumuloSecurityException,
+      AccumuloException, NamespaceExistsException, NamespaceNotFoundException {
+    try {
+      return tableOps.doFateOperation(op, args, opts);
+    } catch (TableExistsException e) {
+      // should not happen
+      throw new AssertionError(e);
+    } catch (TableNotFoundException e) {
+      // should not happen
+      throw new AssertionError(e);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/f4454a06/core/src/main/java/org/apache/accumulo/core/client/impl/SecurityOperationsImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/SecurityOperationsImpl.java b/core/src/main/java/org/apache/accumulo/core/client/impl/SecurityOperationsImpl.java
new file mode 100644
index 0000000..6d8c59b
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/SecurityOperationsImpl.java
@@ -0,0 +1,331 @@
+/*
+ * 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.core.client.impl;
+
+import java.nio.ByteBuffer;
+import java.util.Set;
+
+import org.apache.accumulo.core.client.AccumuloException;
+import org.apache.accumulo.core.client.AccumuloSecurityException;
+import org.apache.accumulo.core.client.Instance;
+import org.apache.accumulo.core.client.admin.SecurityOperations;
+import org.apache.accumulo.core.client.impl.thrift.ClientService;
+import org.apache.accumulo.core.client.impl.thrift.SecurityErrorCode;
+import org.apache.accumulo.core.client.impl.thrift.TableOperationExceptionType;
+import org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException;
+import org.apache.accumulo.core.client.impl.thrift.ThriftTableOperationException;
+import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
+import org.apache.accumulo.core.client.security.tokens.PasswordToken;
+import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.core.security.Credentials;
+import org.apache.accumulo.core.security.NamespacePermission;
+import org.apache.accumulo.core.security.SystemPermission;
+import org.apache.accumulo.core.security.TablePermission;
+import org.apache.accumulo.core.util.ArgumentChecker;
+import org.apache.accumulo.core.util.ByteBufferUtil;
+import org.apache.accumulo.trace.instrument.Tracer;
+
+public class SecurityOperationsImpl implements SecurityOperations {
+
+  private Instance instance;
+  private Credentials credentials;
+
+  private void execute(ClientExec<ClientService.Client> exec) throws AccumuloException, AccumuloSecurityException {
+    try {
+      ServerClient.executeRaw(instance, exec);
+    } catch (ThriftTableOperationException ttoe) {
+      // recast missing table
+      if (ttoe.getType() == TableOperationExceptionType.NOTFOUND)
+        throw new AccumuloSecurityException(null, SecurityErrorCode.TABLE_DOESNT_EXIST);
+      else if (ttoe.getType() == TableOperationExceptionType.NAMESPACE_NOTFOUND)
+        throw new AccumuloSecurityException(null, SecurityErrorCode.NAMESPACE_DOESNT_EXIST);
+      else
+        throw new AccumuloException(ttoe);
+    } catch (ThriftSecurityException e) {
+      throw new AccumuloSecurityException(e.user, e.code, e);
+    } catch (AccumuloException e) {
+      throw e;
+    } catch (Exception e) {
+      throw new AccumuloException(e);
+    }
+  }
+
+  private <T> T execute(ClientExecReturn<T,ClientService.Client> exec) throws AccumuloException, AccumuloSecurityException {
+    try {
+      return ServerClient.executeRaw(instance, exec);
+    } catch (ThriftTableOperationException ttoe) {
+      // recast missing table
+      if (ttoe.getType() == TableOperationExceptionType.NOTFOUND)
+        throw new AccumuloSecurityException(null, SecurityErrorCode.TABLE_DOESNT_EXIST);
+      else if (ttoe.getType() == TableOperationExceptionType.NAMESPACE_NOTFOUND)
+        throw new AccumuloSecurityException(null, SecurityErrorCode.NAMESPACE_DOESNT_EXIST);
+      else
+        throw new AccumuloException(ttoe);
+    } catch (ThriftSecurityException e) {
+      throw new AccumuloSecurityException(e.user, e.code, e);
+    } catch (AccumuloException e) {
+      throw e;
+    } catch (Exception e) {
+      throw new AccumuloException(e);
+    }
+  }
+
+  public SecurityOperationsImpl(Instance instance, Credentials credentials) {
+    ArgumentChecker.notNull(instance, credentials);
+    this.instance = instance;
+    this.credentials = credentials;
+  }
+
+  @Deprecated
+  @Override
+  public void createUser(String user, byte[] password, final Authorizations authorizations) throws AccumuloException, AccumuloSecurityException {
+    createLocalUser(user, new PasswordToken(password));
+    changeUserAuthorizations(user, authorizations);
+  }
+
+  @Override
+  public void createLocalUser(final String principal, final PasswordToken password) throws AccumuloException, AccumuloSecurityException {
+    ArgumentChecker.notNull(principal, password);
+    execute(new ClientExec<ClientService.Client>() {
+      @Override
+      public void execute(ClientService.Client client) throws Exception {
+        client.createLocalUser(Tracer.traceInfo(), credentials.toThrift(instance), principal, ByteBuffer.wrap(password.getPassword()));
+      }
+    });
+  }
+
+  @Deprecated
+  @Override
+  public void dropUser(final String user) throws AccumuloException, AccumuloSecurityException {
+    dropLocalUser(user);
+  }
+
+  @Override
+  public void dropLocalUser(final String principal) throws AccumuloException, AccumuloSecurityException {
+    ArgumentChecker.notNull(principal);
+    execute(new ClientExec<ClientService.Client>() {
+      @Override
+      public void execute(ClientService.Client client) throws Exception {
+        client.dropLocalUser(Tracer.traceInfo(), credentials.toThrift(instance), principal);
+      }
+    });
+  }
+
+  @Deprecated
+  @Override
+  public boolean authenticateUser(String user, byte[] password) throws AccumuloException, AccumuloSecurityException {
+    return authenticateUser(user, new PasswordToken(password));
+  }
+
+  @Override
+  public boolean authenticateUser(final String principal, final AuthenticationToken token) throws AccumuloException, AccumuloSecurityException {
+    ArgumentChecker.notNull(principal, token);
+    final Credentials toAuth = new Credentials(principal, token);
+    return execute(new ClientExecReturn<Boolean,ClientService.Client>() {
+      @Override
+      public Boolean execute(ClientService.Client client) throws Exception {
+        return client.authenticateUser(Tracer.traceInfo(), credentials.toThrift(instance), toAuth.toThrift(instance));
+      }
+    });
+  }
+
+  @Override
+  @Deprecated
+  public void changeUserPassword(String user, byte[] password) throws AccumuloException, AccumuloSecurityException {
+    changeLocalUserPassword(user, new PasswordToken(password));
+  }
+
+  @Override
+  public void changeLocalUserPassword(final String principal, final PasswordToken token) throws AccumuloException, AccumuloSecurityException {
+    ArgumentChecker.notNull(principal, token);
+    final Credentials toChange = new Credentials(principal, token);
+    execute(new ClientExec<ClientService.Client>() {
+      @Override
+      public void execute(ClientService.Client client) throws Exception {
+        client.changeLocalUserPassword(Tracer.traceInfo(), credentials.toThrift(instance), principal, ByteBuffer.wrap(token.getPassword()));
+      }
+    });
+    if (this.credentials.getPrincipal().equals(principal)) {
+      this.credentials = toChange;
+    }
+  }
+
+  @Override
+  public void changeUserAuthorizations(final String principal, final Authorizations authorizations) throws AccumuloException, AccumuloSecurityException {
+    ArgumentChecker.notNull(principal, authorizations);
+    execute(new ClientExec<ClientService.Client>() {
+      @Override
+      public void execute(ClientService.Client client) throws Exception {
+        client.changeAuthorizations(Tracer.traceInfo(), credentials.toThrift(instance), principal,
+            ByteBufferUtil.toByteBuffers(authorizations.getAuthorizations()));
+      }
+    });
+  }
+
+  @Override
+  public Authorizations getUserAuthorizations(final String principal) throws AccumuloException, AccumuloSecurityException {
+    ArgumentChecker.notNull(principal);
+    return execute(new ClientExecReturn<Authorizations,ClientService.Client>() {
+      @Override
+      public Authorizations execute(ClientService.Client client) throws Exception {
+        return new Authorizations(client.getUserAuthorizations(Tracer.traceInfo(), credentials.toThrift(instance), principal));
+      }
+    });
+  }
+
+  @Override
+  public boolean hasSystemPermission(final String principal, final SystemPermission perm) throws AccumuloException, AccumuloSecurityException {
+    ArgumentChecker.notNull(principal, perm);
+    return execute(new ClientExecReturn<Boolean,ClientService.Client>() {
+      @Override
+      public Boolean execute(ClientService.Client client) throws Exception {
+        return client.hasSystemPermission(Tracer.traceInfo(), credentials.toThrift(instance), principal, perm.getId());
+      }
+    });
+  }
+
+  @Override
+  public boolean hasTablePermission(final String principal, final String table, final TablePermission perm) throws AccumuloException, AccumuloSecurityException {
+    ArgumentChecker.notNull(principal, table, perm);
+    try {
+      return execute(new ClientExecReturn<Boolean,ClientService.Client>() {
+        @Override
+        public Boolean execute(ClientService.Client client) throws Exception {
+          return client.hasTablePermission(Tracer.traceInfo(), credentials.toThrift(instance), principal, table, perm.getId());
+        }
+      });
+    } catch (AccumuloSecurityException e) {
+      if (e.getSecurityErrorCode() == org.apache.accumulo.core.client.security.SecurityErrorCode.NAMESPACE_DOESNT_EXIST)
+        throw new AccumuloSecurityException(null, SecurityErrorCode.TABLE_DOESNT_EXIST, e);
+      else
+        throw e;
+    }
+  }
+
+  @Override
+  public boolean hasNamespacePermission(final String principal, final String namespace, final NamespacePermission permission) throws AccumuloException,
+      AccumuloSecurityException {
+    ArgumentChecker.notNull(principal, namespace, permission);
+    return execute(new ClientExecReturn<Boolean,ClientService.Client>() {
+      @Override
+      public Boolean execute(ClientService.Client client) throws Exception {
+        return client.hasNamespacePermission(Tracer.traceInfo(), credentials.toThrift(instance), principal, namespace, permission.getId());
+      }
+    });
+  }
+
+  @Override
+  public void grantSystemPermission(final String principal, final SystemPermission permission) throws AccumuloException, AccumuloSecurityException {
+    ArgumentChecker.notNull(principal, permission);
+    execute(new ClientExec<ClientService.Client>() {
+      @Override
+      public void execute(ClientService.Client client) throws Exception {
+        client.grantSystemPermission(Tracer.traceInfo(), credentials.toThrift(instance), principal, permission.getId());
+      }
+    });
+  }
+
+  @Override
+  public void grantTablePermission(final String principal, final String table, final TablePermission permission) throws AccumuloException,
+      AccumuloSecurityException {
+    ArgumentChecker.notNull(principal, table, permission);
+    try {
+      execute(new ClientExec<ClientService.Client>() {
+        @Override
+        public void execute(ClientService.Client client) throws Exception {
+          client.grantTablePermission(Tracer.traceInfo(), credentials.toThrift(instance), principal, table, permission.getId());
+        }
+      });
+    } catch (AccumuloSecurityException e) {
+      if (e.getSecurityErrorCode() == org.apache.accumulo.core.client.security.SecurityErrorCode.NAMESPACE_DOESNT_EXIST)
+        throw new AccumuloSecurityException(null, SecurityErrorCode.TABLE_DOESNT_EXIST, e);
+      else
+        throw e;
+    }
+  }
+
+  @Override
+  public void grantNamespacePermission(final String principal, final String namespace, final NamespacePermission permission) throws AccumuloException,
+      AccumuloSecurityException {
+    ArgumentChecker.notNull(principal, namespace, permission);
+    execute(new ClientExec<ClientService.Client>() {
+      @Override
+      public void execute(ClientService.Client client) throws Exception {
+        client.grantNamespacePermission(Tracer.traceInfo(), credentials.toThrift(instance), principal, namespace, permission.getId());
+      }
+    });
+  }
+
+  @Override
+  public void revokeSystemPermission(final String principal, final SystemPermission permission) throws AccumuloException, AccumuloSecurityException {
+    ArgumentChecker.notNull(principal, permission);
+    execute(new ClientExec<ClientService.Client>() {
+      @Override
+      public void execute(ClientService.Client client) throws Exception {
+        client.revokeSystemPermission(Tracer.traceInfo(), credentials.toThrift(instance), principal, permission.getId());
+      }
+    });
+  }
+
+  @Override
+  public void revokeTablePermission(final String principal, final String table, final TablePermission permission) throws AccumuloException,
+      AccumuloSecurityException {
+    ArgumentChecker.notNull(principal, table, permission);
+    try {
+      execute(new ClientExec<ClientService.Client>() {
+        @Override
+        public void execute(ClientService.Client client) throws Exception {
+          client.revokeTablePermission(Tracer.traceInfo(), credentials.toThrift(instance), principal, table, permission.getId());
+        }
+      });
+    } catch (AccumuloSecurityException e) {
+      if (e.getSecurityErrorCode() == org.apache.accumulo.core.client.security.SecurityErrorCode.NAMESPACE_DOESNT_EXIST)
+        throw new AccumuloSecurityException(null, SecurityErrorCode.TABLE_DOESNT_EXIST, e);
+      else
+        throw e;
+    }
+  }
+
+  @Override
+  public void revokeNamespacePermission(final String principal, final String namespace, final NamespacePermission permission) throws AccumuloException,
+      AccumuloSecurityException {
+    ArgumentChecker.notNull(principal, namespace, permission);
+    execute(new ClientExec<ClientService.Client>() {
+      @Override
+      public void execute(ClientService.Client client) throws Exception {
+        client.revokeNamespacePermission(Tracer.traceInfo(), credentials.toThrift(instance), principal, namespace, permission.getId());
+      }
+    });
+  }
+
+  @Deprecated
+  @Override
+  public Set<String> listUsers() throws AccumuloException, AccumuloSecurityException {
+    return listLocalUsers();
+  }
+
+  @Override
+  public Set<String> listLocalUsers() throws AccumuloException, AccumuloSecurityException {
+    return execute(new ClientExecReturn<Set<String>,ClientService.Client>() {
+      @Override
+      public Set<String> execute(ClientService.Client client) throws Exception {
+        return client.listLocalUsers(Tracer.traceInfo(), credentials.toThrift(instance));
+      }
+    });
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/f4454a06/core/src/main/java/org/apache/accumulo/core/client/impl/TableOperationsHelper.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/TableOperationsHelper.java b/core/src/main/java/org/apache/accumulo/core/client/impl/TableOperationsHelper.java
new file mode 100644
index 0000000..58f1a42
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/TableOperationsHelper.java
@@ -0,0 +1,199 @@
+/*
+ * 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.core.client.impl;
+
+import java.util.EnumSet;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.TreeMap;
+import java.util.TreeSet;
+
+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.TableNotFoundException;
+import org.apache.accumulo.core.client.admin.TableOperations;
+import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.core.iterators.IteratorUtil.IteratorScope;
+import org.apache.accumulo.core.util.ArgumentChecker;
+
+public abstract class TableOperationsHelper implements TableOperations {
+
+  @Override
+  public void attachIterator(String tableName, IteratorSetting setting) throws AccumuloSecurityException, AccumuloException, TableNotFoundException {
+    attachIterator(tableName, setting, EnumSet.allOf(IteratorScope.class));
+  }
+
+  @Override
+  public void attachIterator(String tableName, IteratorSetting setting, EnumSet<IteratorScope> scopes) throws AccumuloSecurityException, AccumuloException,
+      TableNotFoundException {
+    ArgumentChecker.notNull(tableName, setting, scopes);
+    checkIteratorConflicts(tableName, setting, scopes);
+    for (IteratorScope scope : scopes) {
+      String root = String.format("%s%s.%s", Property.TABLE_ITERATOR_PREFIX, scope.name().toLowerCase(), setting.getName());
+      for (Entry<String,String> prop : setting.getOptions().entrySet()) {
+        this.setProperty(tableName, root + ".opt." + prop.getKey(), prop.getValue());
+      }
+      this.setProperty(tableName, root, setting.getPriority() + "," + setting.getIteratorClass());
+    }
+  }
+
+  @Override
+  public void removeIterator(String tableName, String name, EnumSet<IteratorScope> scopes) throws AccumuloSecurityException, AccumuloException,
+      TableNotFoundException {
+    Map<String,String> copy = new TreeMap<String,String>();
+    for (Entry<String,String> property : this.getProperties(tableName)) {
+      copy.put(property.getKey(), property.getValue());
+    }
+    for (IteratorScope scope : scopes) {
+      String root = String.format("%s%s.%s", Property.TABLE_ITERATOR_PREFIX, scope.name().toLowerCase(), name);
+      for (Entry<String,String> property : copy.entrySet()) {
+        if (property.getKey().equals(root) || property.getKey().startsWith(root + ".opt."))
+          this.removeProperty(tableName, property.getKey());
+      }
+    }
+  }
+
+  @Override
+  public IteratorSetting getIteratorSetting(String tableName, String name, IteratorScope scope) throws AccumuloSecurityException, AccumuloException,
+      TableNotFoundException {
+    ArgumentChecker.notNull(tableName, name, scope);
+    int priority = -1;
+    String classname = null;
+    Map<String,String> settings = new HashMap<String,String>();
+
+    String root = String.format("%s%s.%s", Property.TABLE_ITERATOR_PREFIX, scope.name().toLowerCase(), name);
+    String opt = root + ".opt.";
+    for (Entry<String,String> property : this.getProperties(tableName)) {
+      if (property.getKey().equals(root)) {
+        String parts[] = property.getValue().split(",");
+        if (parts.length != 2) {
+          throw new AccumuloException("Bad value for iterator setting: " + property.getValue());
+        }
+        priority = Integer.parseInt(parts[0]);
+        classname = parts[1];
+      } else if (property.getKey().startsWith(opt)) {
+        settings.put(property.getKey().substring(opt.length()), property.getValue());
+      }
+    }
+    if (priority <= 0 || classname == null) {
+      return null;
+    }
+    return new IteratorSetting(priority, name, classname, settings);
+  }
+
+  @Override
+  public Map<String,EnumSet<IteratorScope>> listIterators(String tableName) throws AccumuloSecurityException, AccumuloException, TableNotFoundException {
+    Map<String,EnumSet<IteratorScope>> result = new TreeMap<String,EnumSet<IteratorScope>>();
+    for (Entry<String,String> property : this.getProperties(tableName)) {
+      String name = property.getKey();
+      String[] parts = name.split("\\.");
+      if (parts.length == 4) {
+        if (parts[0].equals("table") && parts[1].equals("iterator")) {
+          IteratorScope scope = IteratorScope.valueOf(parts[2]);
+          if (!result.containsKey(parts[3]))
+            result.put(parts[3], EnumSet.noneOf(IteratorScope.class));
+          result.get(parts[3]).add(scope);
+        }
+      }
+    }
+    return result;
+  }
+
+  @Override
+  public void checkIteratorConflicts(String tableName, IteratorSetting setting, EnumSet<IteratorScope> scopes) throws AccumuloException, TableNotFoundException {
+    ArgumentChecker.notNull(tableName, setting, scopes);
+    for (IteratorScope scope : scopes) {
+      String scopeStr = String.format("%s%s", Property.TABLE_ITERATOR_PREFIX, scope.name().toLowerCase());
+      String nameStr = String.format("%s.%s", scopeStr, setting.getName());
+      String optStr = String.format("%s.opt.", nameStr);
+      Map<String,String> optionConflicts = new TreeMap<String,String>();
+      for (Entry<String,String> property : this.getProperties(tableName)) {
+        if (property.getKey().startsWith(scopeStr)) {
+          if (property.getKey().equals(nameStr))
+            throw new AccumuloException(new IllegalArgumentException("iterator name conflict for " + setting.getName() + ": " + property.getKey() + "="
+                + property.getValue()));
+          if (property.getKey().startsWith(optStr))
+            optionConflicts.put(property.getKey(), property.getValue());
+          if (property.getKey().contains(".opt."))
+            continue;
+          String parts[] = property.getValue().split(",");
+          if (parts.length != 2)
+            throw new AccumuloException("Bad value for existing iterator setting: " + property.getKey() + "=" + property.getValue());
+          try {
+            if (Integer.parseInt(parts[0]) == setting.getPriority())
+              throw new AccumuloException(new IllegalArgumentException("iterator priority conflict: " + property.getKey() + "=" + property.getValue()));
+          } catch (NumberFormatException e) {
+            throw new AccumuloException("Bad value for existing iterator setting: " + property.getKey() + "=" + property.getValue());
+          }
+        }
+      }
+      if (optionConflicts.size() > 0)
+        throw new AccumuloException(new IllegalArgumentException("iterator options conflict for " + setting.getName() + ": " + optionConflicts));
+    }
+  }
+
+  @Override
+  public int addConstraint(String tableName, String constraintClassName) throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
+    TreeSet<Integer> constraintNumbers = new TreeSet<Integer>();
+    TreeMap<String,Integer> constraintClasses = new TreeMap<String,Integer>();
+    int i;
+    for (Entry<String,String> property : this.getProperties(tableName)) {
+      if (property.getKey().startsWith(Property.TABLE_CONSTRAINT_PREFIX.toString())) {
+        try {
+          i = Integer.parseInt(property.getKey().substring(Property.TABLE_CONSTRAINT_PREFIX.toString().length()));
+        } catch (NumberFormatException e) {
+          throw new AccumuloException("Bad key for existing constraint: " + property.toString());
+        }
+        constraintNumbers.add(i);
+        constraintClasses.put(property.getValue(), i);
+      }
+    }
+    i = 1;
+    while (constraintNumbers.contains(i))
+      i++;
+    if (constraintClasses.containsKey(constraintClassName))
+      throw new AccumuloException("Constraint " + constraintClassName + " already exists for table " + tableName + " with number "
+          + constraintClasses.get(constraintClassName));
+    this.setProperty(tableName, Property.TABLE_CONSTRAINT_PREFIX.toString() + i, constraintClassName);
+    return i;
+  }
+
+  @Override
+  public void removeConstraint(String tableName, int number) throws AccumuloException, AccumuloSecurityException {
+    this.removeProperty(tableName, Property.TABLE_CONSTRAINT_PREFIX.toString() + number);
+  }
+
+  @Override
+  public Map<String,Integer> listConstraints(String tableName) throws AccumuloException, TableNotFoundException {
+    Map<String,Integer> constraints = new TreeMap<String,Integer>();
+    for (Entry<String,String> property : this.getProperties(tableName)) {
+      if (property.getKey().startsWith(Property.TABLE_CONSTRAINT_PREFIX.toString())) {
+        if (constraints.containsKey(property.getValue()))
+          throw new AccumuloException("Same constraint configured twice: " + property.getKey() + "=" + Property.TABLE_CONSTRAINT_PREFIX
+              + constraints.get(property.getValue()) + "=" + property.getKey());
+        try {
+          constraints.put(property.getValue(), Integer.parseInt(property.getKey().substring(Property.TABLE_CONSTRAINT_PREFIX.toString().length())));
+        } catch (NumberFormatException e) {
+          throw new AccumuloException("Bad key for existing constraint: " + property.toString());
+        }
+      }
+    }
+    return constraints;
+  }
+}


[05/17] git commit: ACCUMULO-2726 Adds back missing methods for binary backwards compat with non-Deprecated methods from 1.5.0. Marks things that we're for sure removing with @Deprecated; mostly things that shouldn't have been in public to begin with.

Posted by bu...@apache.org.
ACCUMULO-2726 Adds back missing methods for binary backwards compat with non-Deprecated methods from 1.5.0. Marks things that we're for sure removing with @Deprecated; mostly things that shouldn't have been in public to begin with.


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

Branch: refs/heads/master
Commit: f4454a06567c1fd4ff8b9a374d633a42c23a8a80
Parents: e450d74
Author: Sean Busbey <bu...@cloudera.com>
Authored: Wed Apr 23 17:31:36 2014 -0500
Committer: Sean Busbey <bu...@cloudera.com>
Committed: Fri Apr 25 11:26:46 2014 -0500

----------------------------------------------------------------------
 .../core/client/admin/ActiveCompaction.java     |   79 +-
 .../accumulo/core/client/admin/ActiveScan.java  |  112 +-
 .../client/admin/InstanceOperationsImpl.java    |  190 +--
 .../client/admin/NamespaceOperationsHelper.java |  214 ---
 .../client/admin/NamespaceOperationsImpl.java   |  248 ---
 .../client/admin/SecurityOperationsImpl.java    |  315 +---
 .../client/admin/TableOperationsHelper.java     |  184 +-
 .../core/client/admin/TableOperationsImpl.java  | 1572 +----------------
 .../core/client/impl/ActiveCompactionImpl.java  |  109 ++
 .../core/client/impl/ActiveScanImpl.java        |  146 ++
 .../core/client/impl/ConnectorImpl.java         |    8 +-
 .../client/impl/InstanceOperationsImpl.java     |  209 +++
 .../client/impl/NamespaceOperationsHelper.java  |  214 +++
 .../client/impl/NamespaceOperationsImpl.java    |  244 +++
 .../client/impl/SecurityOperationsImpl.java     |  331 ++++
 .../core/client/impl/TableOperationsHelper.java |  199 +++
 .../core/client/impl/TableOperationsImpl.java   | 1589 ++++++++++++++++++
 .../mapreduce/lib/impl/InputConfigurator.java   |    2 +-
 .../core/client/mock/MockConnector.java         |    6 +-
 .../client/mock/MockInstanceOperations.java     |   70 +-
 .../client/mock/MockInstanceOperationsImpl.java |   87 +
 .../client/mock/MockNamespaceOperations.java    |    4 +-
 .../client/mock/MockSecurityOperations.java     |  205 +--
 .../client/mock/MockSecurityOperationsImpl.java |  225 +++
 .../accumulo/core/client/mock/MockShell.java    |   41 +-
 .../core/client/mock/MockTableOperations.java   |  431 +----
 .../client/mock/MockTableOperationsImpl.java    |  447 +++++
 .../core/client/mock/MockTabletLocator.java     |   45 +-
 .../client/mock/impl/MockTabletLocator.java     |   70 +
 .../client/admin/TableOperationsHelperTest.java |   90 +-
 .../client/impl/TableOperationsHelperTest.java  |  305 ++++
 .../server/security/SecurityOperation.java      |    2 +-
 .../accumulo/master/FateServiceHandler.java     |    2 +-
 .../accumulo/master/tableOps/ImportTable.java   |    2 +-
 .../japi-compliance/japi-accumulo-1.5.xml       |    1 -
 .../japi-compliance/japi-accumulo-1.6.xml       |    3 +-
 36 files changed, 4333 insertions(+), 3668 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/f4454a06/core/src/main/java/org/apache/accumulo/core/client/admin/ActiveCompaction.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/admin/ActiveCompaction.java b/core/src/main/java/org/apache/accumulo/core/client/admin/ActiveCompaction.java
index 9c39ea6..41b9c67 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/admin/ActiveCompaction.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/admin/ActiveCompaction.java
@@ -16,31 +16,18 @@
  */
 package org.apache.accumulo.core.client.admin;
 
-import java.util.ArrayList;
 import java.util.List;
-import java.util.Map;
 
-import org.apache.accumulo.core.client.Instance;
 import org.apache.accumulo.core.client.IteratorSetting;
 import org.apache.accumulo.core.client.TableNotFoundException;
-import org.apache.accumulo.core.client.impl.Tables;
 import org.apache.accumulo.core.data.KeyExtent;
-import org.apache.accumulo.core.data.thrift.IterInfo;
 
 
 /**
  * 
  * @since 1.5.0
  */
-public class ActiveCompaction {
-  
-  private org.apache.accumulo.core.tabletserver.thrift.ActiveCompaction tac;
-  private Instance instance;
-
-  ActiveCompaction(Instance instance, org.apache.accumulo.core.tabletserver.thrift.ActiveCompaction tac) {
-    this.tac = tac;
-    this.instance = instance;
-  }
+public abstract class ActiveCompaction {
 
   public static enum CompactionType {
     /**
@@ -88,97 +75,55 @@ public class ActiveCompaction {
    * 
    * @return name of the table the compaction is running against
    */
-  
-  public String getTable() throws TableNotFoundException {
-    return Tables.getTableName(instance, getExtent().getTableId().toString());
-  }
+  public abstract String getTable() throws TableNotFoundException;
   
   /**
    * @return tablet thats is compacting
    */
-
-  public KeyExtent getExtent() {
-    return new KeyExtent(tac.getExtent());
-  }
+  public abstract KeyExtent getExtent();
   
   /**
    * @return how long the compaction has been running in milliseconds
    */
-
-  public long getAge() {
-    return tac.getAge();
-  }
+  public abstract long getAge();
   
   /**
    * @return the files the compaction is reading from
    */
-
-  public List<String> getInputFiles() {
-    return tac.getInputFiles();
-  }
+  public abstract List<String> getInputFiles();
   
   /**
    * @return file compactions is writing too
    */
-
-  public String getOutputFile() {
-    return tac.getOutputFile();
-  }
+  public abstract String getOutputFile();
   
   /**
    * @return the type of compaction
    */
-  public CompactionType getType() {
-    return CompactionType.valueOf(tac.getType().name());
-  }
+  public abstract CompactionType getType();
   
   /**
    * @return the reason the compaction was started
    */
-
-  public CompactionReason getReason() {
-    return CompactionReason.valueOf(tac.getReason().name());
-  }
+  public abstract CompactionReason getReason();
   
   /**
    * @return the locality group that is compacting
    */
-
-  public String getLocalityGroup() {
-    return tac.getLocalityGroup();
-  }
+  public abstract String getLocalityGroup();
   
   /**
    * @return the number of key/values read by the compaction
    */
+  public abstract long getEntriesRead();
 
-  public long getEntriesRead() {
-    return tac.getEntriesRead();
-  }
-  
   /**
    * @return the number of key/values written by the compaction
    */
-
-  public long getEntriesWritten() {
-    return tac.getEntriesWritten();
-  }
+  public abstract long getEntriesWritten();
   
   /**
    * @return the per compaction iterators configured
    */
-
-  public List<IteratorSetting> getIterators() {
-    ArrayList<IteratorSetting> ret = new ArrayList<IteratorSetting>();
-    
-    for (IterInfo ii : tac.getSsiList()) {
-      IteratorSetting settings = new IteratorSetting(ii.getPriority(), ii.getIterName(), ii.getClassName());
-      Map<String,String> options = tac.getSsio().get(ii.getIterName());
-      settings.addOptions(options);
-      
-      ret.add(settings);
-    }
-    
-    return ret;
-  }
+  public abstract List<IteratorSetting> getIterators();
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/f4454a06/core/src/main/java/org/apache/accumulo/core/client/admin/ActiveScan.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/admin/ActiveScan.java b/core/src/main/java/org/apache/accumulo/core/client/admin/ActiveScan.java
index 30e47af..fc9808f 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/admin/ActiveScan.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/admin/ActiveScan.java
@@ -16,164 +16,82 @@
  */
 package org.apache.accumulo.core.client.admin;
 
-import java.util.ArrayList;
 import java.util.List;
 import java.util.Map;
 
-import org.apache.accumulo.core.client.Instance;
-import org.apache.accumulo.core.client.TableNotFoundException;
-import org.apache.accumulo.core.client.impl.Tables;
 import org.apache.accumulo.core.data.Column;
 import org.apache.accumulo.core.data.KeyExtent;
-import org.apache.accumulo.core.data.thrift.IterInfo;
-import org.apache.accumulo.core.data.thrift.TColumn;
 import org.apache.accumulo.core.security.Authorizations;
 
 /**
  * A class that contains information about an ActiveScan
  * 
  */
+public abstract class ActiveScan {
 
-public class ActiveScan {
-  
-  private long scanid;
-  private String client;
-  private String table;
-  private long age;
-  private long idle;
-  private ScanType type;
-  private ScanState state;
-  private KeyExtent extent;
-  private List<Column> columns;
-  private List<String> ssiList;
-  private Map<String,Map<String,String>> ssio;
-  private String user;
-  private Authorizations authorizations;
-  
-  ActiveScan(Instance instance, org.apache.accumulo.core.tabletserver.thrift.ActiveScan activeScan) throws TableNotFoundException {
-    this.client = activeScan.client;
-    this.user = activeScan.user;
-    this.age = activeScan.age;
-    this.idle = activeScan.idleTime;
-    this.table = Tables.getTableName(instance, activeScan.tableId);
-    this.type = ScanType.valueOf(activeScan.getType().name());
-    this.state = ScanState.valueOf(activeScan.state.name());
-    this.extent = new KeyExtent(activeScan.extent);
-    this.authorizations = new Authorizations(activeScan.authorizations);
-    
-    this.columns = new ArrayList<Column>(activeScan.columns.size());
-    
-    for (TColumn tcolumn : activeScan.columns)
-      this.columns.add(new Column(tcolumn));
-    
-    this.ssiList = new ArrayList<String>();
-    for (IterInfo ii : activeScan.ssiList) {
-      this.ssiList.add(ii.iterName + "=" + ii.priority + "," + ii.className);
-    }
-    this.ssio = activeScan.ssio;
-  }
-  
   /**
    * @return an id that uniquely identifies that scan on the server
    */
-  public long getScanid() {
-    return scanid;
-  }
+  public abstract long getScanid();
   
   /**
    * @return the address of the client that initiated the scan
    */
-  
-  public String getClient() {
-    return client;
-  }
+  public abstract String getClient();
   
   /**
    * @return the user that initiated the scan
    */
-  
-  public String getUser() {
-    return user;
-  }
+  public abstract String getUser();
   
   /**
    * @return the table the scan is running against
    */
-  
-  public String getTable() {
-    return table;
-  }
+  public abstract String getTable();
   
   /**
    * @return the age of the scan in milliseconds
    */
-  
-  public long getAge() {
-    return age;
-  }
+  public abstract long getAge();
   
   /**
    * @return milliseconds since last time client read data from the scan
    */
+  public abstract long getLastContactTime();
   
-  public long getLastContactTime() {
-    return idle;
-  }
+  public abstract ScanType getType();
   
-  public ScanType getType() {
-    return type;
-  }
-  
-  public ScanState getState() {
-    return state;
-  }
+  public abstract ScanState getState();
   
   /**
    * @return tablet the scan is running against, if a batch scan may be one of many or null
    */
-  
-  public KeyExtent getExtent() {
-    return extent;
-  }
+  public abstract KeyExtent getExtent();
   
   /**
    * @return columns requested by the scan
    */
-  
-  public List<Column> getColumns() {
-    return columns;
-  }
+  public abstract List<Column> getColumns();
   
   /**
    * @return server side iterators used by the scan
    */
-  
-  public List<String> getSsiList() {
-    return ssiList;
-  }
+  public abstract List<String> getSsiList();
   
   /**
    * @return server side iterator options
    */
-  
-  public Map<String,Map<String,String>> getSsio() {
-    return ssio;
-  }
+  public abstract Map<String,Map<String,String>> getSsio();
   
   /**
    * @return the authorizations being used for this scan
    * @since 1.5.0
    */
-  
-  public Authorizations getAuthorizations() {
-    return authorizations;
-  }
+  public abstract Authorizations getAuthorizations();
   
   /**
    * @return the time this scan has been idle in the tablet server
    * @since 1.5.0
    */
-  public long getIdleTime() {
-    return idle;
-  }
+  public abstract long getIdleTime();
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/f4454a06/core/src/main/java/org/apache/accumulo/core/client/admin/InstanceOperationsImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/admin/InstanceOperationsImpl.java b/core/src/main/java/org/apache/accumulo/core/client/admin/InstanceOperationsImpl.java
index f80eee5..11e389f 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/admin/InstanceOperationsImpl.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/admin/InstanceOperationsImpl.java
@@ -16,44 +16,16 @@
  */
 package org.apache.accumulo.core.client.admin;
 
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-import java.util.Map;
-
-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.Instance;
-import org.apache.accumulo.core.client.TableNotFoundException;
-import org.apache.accumulo.core.client.impl.ClientExec;
-import org.apache.accumulo.core.client.impl.ClientExecReturn;
-import org.apache.accumulo.core.client.impl.MasterClient;
-import org.apache.accumulo.core.client.impl.ServerClient;
-import org.apache.accumulo.core.client.impl.ServerConfigurationUtil;
-import org.apache.accumulo.core.client.impl.thrift.ClientService;
-import org.apache.accumulo.core.client.impl.thrift.ConfigurationType;
-import org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException;
-import org.apache.accumulo.core.master.thrift.MasterClientService;
 import org.apache.accumulo.core.security.Credentials;
-import org.apache.accumulo.core.tabletserver.thrift.TabletClientService;
-import org.apache.accumulo.core.tabletserver.thrift.TabletClientService.Client;
-import org.apache.accumulo.core.util.AddressUtil;
-import org.apache.accumulo.core.util.ArgumentChecker;
-import org.apache.accumulo.core.util.ThriftUtil;
-import org.apache.accumulo.core.zookeeper.ZooUtil;
-import org.apache.accumulo.fate.zookeeper.ZooCache;
-import org.apache.accumulo.trace.instrument.Tracer;
-import org.apache.thrift.TException;
-import org.apache.thrift.transport.TTransport;
-import org.apache.thrift.transport.TTransportException;
+import org.apache.accumulo.core.security.thrift.TCredentials;
 
 /**
  * Provides a class for administering the accumulo instance
+ * @deprecated since 1.6.0; not intended for public api and you should not use it.
  */
-public class InstanceOperationsImpl implements InstanceOperations {
-  private Instance instance;
-  private Credentials credentials;
+@Deprecated
+public class InstanceOperationsImpl extends org.apache.accumulo.core.client.impl.InstanceOperationsImpl {
   
   /**
    * @param instance
@@ -61,151 +33,17 @@ public class InstanceOperationsImpl implements InstanceOperations {
    * @param credentials
    *          the Credential, containing principal and Authentication Token
    */
-  public InstanceOperationsImpl(Instance instance, Credentials credentials) {
-    ArgumentChecker.notNull(instance, credentials);
-    this.instance = instance;
-    this.credentials = credentials;
-  }
-  
-  @Override
-  public void setProperty(final String property, final String value) throws AccumuloException, AccumuloSecurityException {
-    ArgumentChecker.notNull(property, value);
-    MasterClient.execute(instance, new ClientExec<MasterClientService.Client>() {
-      @Override
-      public void execute(MasterClientService.Client client) throws Exception {
-        client.setSystemProperty(Tracer.traceInfo(), credentials.toThrift(instance), property, value);
-      }
-    });
-  }
-  
-  @Override
-  public void removeProperty(final String property) throws AccumuloException, AccumuloSecurityException {
-    ArgumentChecker.notNull(property);
-    MasterClient.execute(instance, new ClientExec<MasterClientService.Client>() {
-      @Override
-      public void execute(MasterClientService.Client client) throws Exception {
-        client.removeSystemProperty(Tracer.traceInfo(), credentials.toThrift(instance), property);
-      }
-    });
-  }
-  
-  @Override
-  public Map<String,String> getSystemConfiguration() throws AccumuloException, AccumuloSecurityException {
-    return ServerClient.execute(instance, new ClientExecReturn<Map<String,String>,ClientService.Client>() {
-      @Override
-      public Map<String,String> execute(ClientService.Client client) throws Exception {
-        return client.getConfiguration(Tracer.traceInfo(), credentials.toThrift(instance), ConfigurationType.CURRENT);
-      }
-    });
-  }
-  
-  @Override
-  public Map<String,String> getSiteConfiguration() throws AccumuloException, AccumuloSecurityException {
-    return ServerClient.execute(instance, new ClientExecReturn<Map<String,String>,ClientService.Client>() {
-      @Override
-      public Map<String,String> execute(ClientService.Client client) throws Exception {
-        return client.getConfiguration(Tracer.traceInfo(), credentials.toThrift(instance), ConfigurationType.SITE);
-      }
-    });
+  private InstanceOperationsImpl(Instance instance, Credentials credentials) {
+    super(instance, credentials);
   }
   
-  @Override
-  public List<String> getTabletServers() {
-    ZooCache cache = ZooCache.getInstance(instance.getZooKeepers(), instance.getZooKeepersSessionTimeOut());
-    String path = ZooUtil.getRoot(instance) + Constants.ZTSERVERS;
-    List<String> results = new ArrayList<String>();
-    for (String candidate : cache.getChildren(path)) {
-      List<String> children = cache.getChildren(path + "/" + candidate);
-      if (children != null && children.size() > 0) {
-        List<String> copy = new ArrayList<String>(children);
-        Collections.sort(copy);
-        byte[] data = cache.get(path + "/" + candidate + "/" + copy.get(0));
-        if (data != null && !"master".equals(new String(data, Constants.UTF8))) {
-          results.add(candidate);
-        }
-      }
-    }
-    return results;
-  }
-  
-  @Override
-  public List<ActiveScan> getActiveScans(String tserver) throws AccumuloException, AccumuloSecurityException {
-    Client client = null;
-    try {
-      client = ThriftUtil.getTServerClient(tserver, ServerConfigurationUtil.getConfiguration(instance));
-      
-      List<ActiveScan> as = new ArrayList<ActiveScan>();
-      for (org.apache.accumulo.core.tabletserver.thrift.ActiveScan activeScan : client.getActiveScans(Tracer.traceInfo(), credentials.toThrift(instance))) {
-        try {
-          as.add(new ActiveScan(instance, activeScan));
-        } catch (TableNotFoundException e) {
-          throw new AccumuloException(e);
-        }
-      }
-      return as;
-    } catch (TTransportException e) {
-      throw new AccumuloException(e);
-    } catch (ThriftSecurityException e) {
-      throw new AccumuloSecurityException(e.user, e.code, e);
-    } catch (TException e) {
-      throw new AccumuloException(e);
-    } finally {
-      if (client != null)
-        ThriftUtil.returnClient(client);
-    }
-  }
-  
-  @Override
-  public boolean testClassLoad(final String className, final String asTypeName) throws AccumuloException, AccumuloSecurityException {
-    return ServerClient.execute(instance, new ClientExecReturn<Boolean,ClientService.Client>() {
-      @Override
-      public Boolean execute(ClientService.Client client) throws Exception {
-        return client.checkClass(Tracer.traceInfo(), credentials.toThrift(instance), className, asTypeName);
-      }
-    });
-  }
-  
-  @Override
-  public List<ActiveCompaction> getActiveCompactions(String tserver) throws AccumuloException, AccumuloSecurityException {
-    Client client = null;
-    try {
-      client = ThriftUtil.getTServerClient(tserver, ServerConfigurationUtil.getConfiguration(instance));
-      
-      List<ActiveCompaction> as = new ArrayList<ActiveCompaction>();
-      for (org.apache.accumulo.core.tabletserver.thrift.ActiveCompaction activeCompaction : client.getActiveCompactions(Tracer.traceInfo(),
-          credentials.toThrift(instance))) {
-        as.add(new ActiveCompaction(instance, activeCompaction));
-      }
-      return as;
-    } catch (TTransportException e) {
-      throw new AccumuloException(e);
-    } catch (ThriftSecurityException e) {
-      throw new AccumuloSecurityException(e.user, e.code, e);
-    } catch (TException e) {
-      throw new AccumuloException(e);
-    } finally {
-      if (client != null)
-        ThriftUtil.returnClient(client);
-    }
-  }
-  
-  @Override
-  public void ping(String tserver) throws AccumuloException {
-    TTransport transport = null;
-    try {
-      transport = ThriftUtil.createTransport(AddressUtil.parseAddress(tserver, false), ServerConfigurationUtil.getConfiguration(instance));
-      TabletClientService.Client client = ThriftUtil.createClient(new TabletClientService.Client.Factory(), transport);
-      client.getTabletServerStatus(Tracer.traceInfo(), credentials.toThrift(instance));
-    } catch (TTransportException e) {
-      throw new AccumuloException(e);
-    } catch (ThriftSecurityException e) {
-      throw new AccumuloException(e);
-    } catch (TException e) {
-      throw new AccumuloException(e);
-    } finally {
-      if (transport != null) {
-        transport.close();
-      }
-    }
+  /**
+   * @param instance
+   *          the connection information for this instance
+   * @param credentials
+   *          the Credential, containing principal and Authentication Token
+   */
+  public InstanceOperationsImpl(Instance instance, TCredentials credentials) {
+    this(instance, Credentials.fromThrift(credentials));
   }
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/f4454a06/core/src/main/java/org/apache/accumulo/core/client/admin/NamespaceOperationsHelper.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/admin/NamespaceOperationsHelper.java b/core/src/main/java/org/apache/accumulo/core/client/admin/NamespaceOperationsHelper.java
deleted file mode 100644
index b9a7791..0000000
--- a/core/src/main/java/org/apache/accumulo/core/client/admin/NamespaceOperationsHelper.java
+++ /dev/null
@@ -1,214 +0,0 @@
-/*
- * 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.core.client.admin;
-
-import java.util.EnumSet;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.TreeMap;
-import java.util.TreeSet;
-
-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.impl.Namespaces;
-import org.apache.accumulo.core.conf.Property;
-import org.apache.accumulo.core.iterators.IteratorUtil.IteratorScope;
-
-public abstract class NamespaceOperationsHelper implements NamespaceOperations {
-
-  @Override
-  public String systemNamespace() {
-    return Namespaces.ACCUMULO_NAMESPACE;
-  }
-
-  @Override
-  public String defaultNamespace() {
-    return Namespaces.DEFAULT_NAMESPACE;
-  }
-
-  @Override
-  public void attachIterator(String namespace, IteratorSetting setting) throws AccumuloSecurityException, AccumuloException, NamespaceNotFoundException {
-    attachIterator(namespace, setting, EnumSet.allOf(IteratorScope.class));
-  }
-
-  @Override
-  public void attachIterator(String namespace, IteratorSetting setting, EnumSet<IteratorScope> scopes) throws AccumuloSecurityException, AccumuloException,
-      NamespaceNotFoundException {
-    checkIteratorConflicts(namespace, setting, scopes);
-    for (IteratorScope scope : scopes) {
-      String root = String.format("%s%s.%s", Property.TABLE_ITERATOR_PREFIX, scope.name().toLowerCase(), setting.getName());
-      for (Entry<String,String> prop : setting.getOptions().entrySet()) {
-        this.setProperty(namespace, root + ".opt." + prop.getKey(), prop.getValue());
-      }
-      this.setProperty(namespace, root, setting.getPriority() + "," + setting.getIteratorClass());
-    }
-  }
-
-  @Override
-  public void removeIterator(String namespace, String name, EnumSet<IteratorScope> scopes) throws AccumuloSecurityException, AccumuloException,
-      NamespaceNotFoundException {
-    if (!exists(namespace))
-      throw new NamespaceNotFoundException(null, namespace, null);
-    Map<String,String> copy = new TreeMap<String,String>();
-    for (Entry<String,String> property : this.getProperties(namespace)) {
-      copy.put(property.getKey(), property.getValue());
-    }
-    for (IteratorScope scope : scopes) {
-      String root = String.format("%s%s.%s", Property.TABLE_ITERATOR_PREFIX, scope.name().toLowerCase(), name);
-      for (Entry<String,String> property : copy.entrySet()) {
-        if (property.getKey().equals(root) || property.getKey().startsWith(root + ".opt."))
-          this.removeProperty(namespace, property.getKey());
-      }
-    }
-  }
-
-  @Override
-  public IteratorSetting getIteratorSetting(String namespace, String name, IteratorScope scope) throws AccumuloSecurityException, AccumuloException,
-      NamespaceNotFoundException {
-    if (!exists(namespace))
-      throw new NamespaceNotFoundException(null, namespace, null);
-    int priority = -1;
-    String classname = null;
-    Map<String,String> settings = new HashMap<String,String>();
-
-    String root = String.format("%s%s.%s", Property.TABLE_ITERATOR_PREFIX, scope.name().toLowerCase(), name);
-    String opt = root + ".opt.";
-    for (Entry<String,String> property : this.getProperties(namespace)) {
-      if (property.getKey().equals(root)) {
-        String parts[] = property.getValue().split(",");
-        if (parts.length != 2) {
-          throw new AccumuloException("Bad value for iterator setting: " + property.getValue());
-        }
-        priority = Integer.parseInt(parts[0]);
-        classname = parts[1];
-      } else if (property.getKey().startsWith(opt)) {
-        settings.put(property.getKey().substring(opt.length()), property.getValue());
-      }
-    }
-    if (priority <= 0 || classname == null) {
-      return null;
-    }
-    return new IteratorSetting(priority, name, classname, settings);
-  }
-
-  @Override
-  public Map<String,EnumSet<IteratorScope>> listIterators(String namespace) throws AccumuloSecurityException, AccumuloException, NamespaceNotFoundException {
-    if (!exists(namespace))
-      throw new NamespaceNotFoundException(null, namespace, null);
-    Map<String,EnumSet<IteratorScope>> result = new TreeMap<String,EnumSet<IteratorScope>>();
-    for (Entry<String,String> property : this.getProperties(namespace)) {
-      String name = property.getKey();
-      String[] parts = name.split("\\.");
-      if (parts.length == 4) {
-        if (parts[0].equals("table") && parts[1].equals("iterator")) {
-          IteratorScope scope = IteratorScope.valueOf(parts[2]);
-          if (!result.containsKey(parts[3]))
-            result.put(parts[3], EnumSet.noneOf(IteratorScope.class));
-          result.get(parts[3]).add(scope);
-        }
-      }
-    }
-    return result;
-  }
-
-  @Override
-  public void checkIteratorConflicts(String namespace, IteratorSetting setting, EnumSet<IteratorScope> scopes) throws AccumuloException,
-      NamespaceNotFoundException, AccumuloSecurityException {
-    if (!exists(namespace))
-      throw new NamespaceNotFoundException(null, namespace, null);
-    for (IteratorScope scope : scopes) {
-      String scopeStr = String.format("%s%s", Property.TABLE_ITERATOR_PREFIX, scope.name().toLowerCase());
-      String nameStr = String.format("%s.%s", scopeStr, setting.getName());
-      String optStr = String.format("%s.opt.", nameStr);
-      Map<String,String> optionConflicts = new TreeMap<String,String>();
-      for (Entry<String,String> property : this.getProperties(namespace)) {
-        if (property.getKey().startsWith(scopeStr)) {
-          if (property.getKey().equals(nameStr))
-            throw new AccumuloException(new IllegalArgumentException("iterator name conflict for " + setting.getName() + ": " + property.getKey() + "="
-                + property.getValue()));
-          if (property.getKey().startsWith(optStr))
-            optionConflicts.put(property.getKey(), property.getValue());
-          if (property.getKey().contains(".opt."))
-            continue;
-          String parts[] = property.getValue().split(",");
-          if (parts.length != 2)
-            throw new AccumuloException("Bad value for existing iterator setting: " + property.getKey() + "=" + property.getValue());
-          try {
-            if (Integer.parseInt(parts[0]) == setting.getPriority())
-              throw new AccumuloException(new IllegalArgumentException("iterator priority conflict: " + property.getKey() + "=" + property.getValue()));
-          } catch (NumberFormatException e) {
-            throw new AccumuloException("Bad value for existing iterator setting: " + property.getKey() + "=" + property.getValue());
-          }
-        }
-      }
-      if (optionConflicts.size() > 0)
-        throw new AccumuloException(new IllegalArgumentException("iterator options conflict for " + setting.getName() + ": " + optionConflicts));
-    }
-  }
-
-  @Override
-  public int addConstraint(String namespace, String constraintClassName) throws AccumuloException, AccumuloSecurityException, NamespaceNotFoundException {
-    TreeSet<Integer> constraintNumbers = new TreeSet<Integer>();
-    TreeMap<String,Integer> constraintClasses = new TreeMap<String,Integer>();
-    int i;
-    for (Entry<String,String> property : this.getProperties(namespace)) {
-      if (property.getKey().startsWith(Property.TABLE_CONSTRAINT_PREFIX.toString())) {
-        try {
-          i = Integer.parseInt(property.getKey().substring(Property.TABLE_CONSTRAINT_PREFIX.toString().length()));
-        } catch (NumberFormatException e) {
-          throw new AccumuloException("Bad key for existing constraint: " + property.toString());
-        }
-        constraintNumbers.add(i);
-        constraintClasses.put(property.getValue(), i);
-      }
-    }
-    i = 1;
-    while (constraintNumbers.contains(i))
-      i++;
-    if (constraintClasses.containsKey(constraintClassName))
-      throw new AccumuloException("Constraint " + constraintClassName + " already exists for namespace " + namespace + " with number "
-          + constraintClasses.get(constraintClassName));
-    this.setProperty(namespace, Property.TABLE_CONSTRAINT_PREFIX.toString() + i, constraintClassName);
-    return i;
-  }
-
-  @Override
-  public void removeConstraint(String namespace, int number) throws AccumuloException, AccumuloSecurityException, NamespaceNotFoundException {
-    this.removeProperty(namespace, Property.TABLE_CONSTRAINT_PREFIX.toString() + number);
-  }
-
-  @Override
-  public Map<String,Integer> listConstraints(String namespace) throws AccumuloException, NamespaceNotFoundException, AccumuloSecurityException {
-    Map<String,Integer> constraints = new TreeMap<String,Integer>();
-    for (Entry<String,String> property : this.getProperties(namespace)) {
-      if (property.getKey().startsWith(Property.TABLE_CONSTRAINT_PREFIX.toString())) {
-        if (constraints.containsKey(property.getValue()))
-          throw new AccumuloException("Same constraint configured twice: " + property.getKey() + "=" + Property.TABLE_CONSTRAINT_PREFIX
-              + constraints.get(property.getValue()) + "=" + property.getKey());
-        try {
-          constraints.put(property.getValue(), Integer.parseInt(property.getKey().substring(Property.TABLE_CONSTRAINT_PREFIX.toString().length())));
-        } catch (NumberFormatException e) {
-          throw new AccumuloException("Bad key for existing constraint: " + property.toString());
-        }
-      }
-    }
-    return constraints;
-  }
-}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/f4454a06/core/src/main/java/org/apache/accumulo/core/client/admin/NamespaceOperationsImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/admin/NamespaceOperationsImpl.java b/core/src/main/java/org/apache/accumulo/core/client/admin/NamespaceOperationsImpl.java
deleted file mode 100644
index 569a3b6..0000000
--- a/core/src/main/java/org/apache/accumulo/core/client/admin/NamespaceOperationsImpl.java
+++ /dev/null
@@ -1,248 +0,0 @@
-/*
- * 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.core.client.admin;
-
-import java.nio.ByteBuffer;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.EnumSet;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.SortedSet;
-import java.util.TreeSet;
-
-import org.apache.accumulo.core.client.AccumuloException;
-import org.apache.accumulo.core.client.AccumuloSecurityException;
-import org.apache.accumulo.core.client.Instance;
-import org.apache.accumulo.core.client.IteratorSetting;
-import org.apache.accumulo.core.client.NamespaceExistsException;
-import org.apache.accumulo.core.client.NamespaceNotEmptyException;
-import org.apache.accumulo.core.client.NamespaceNotFoundException;
-import org.apache.accumulo.core.client.TableExistsException;
-import org.apache.accumulo.core.client.TableNotFoundException;
-import org.apache.accumulo.core.client.impl.ClientExec;
-import org.apache.accumulo.core.client.impl.ClientExecReturn;
-import org.apache.accumulo.core.client.impl.MasterClient;
-import org.apache.accumulo.core.client.impl.Namespaces;
-import org.apache.accumulo.core.client.impl.ServerClient;
-import org.apache.accumulo.core.client.impl.thrift.ClientService;
-import org.apache.accumulo.core.client.impl.thrift.SecurityErrorCode;
-import org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException;
-import org.apache.accumulo.core.client.impl.thrift.ThriftTableOperationException;
-import org.apache.accumulo.core.constraints.Constraint;
-import org.apache.accumulo.core.iterators.IteratorUtil.IteratorScope;
-import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
-import org.apache.accumulo.core.master.thrift.FateOperation;
-import org.apache.accumulo.core.master.thrift.MasterClientService;
-import org.apache.accumulo.core.security.Credentials;
-import org.apache.accumulo.core.util.ArgumentChecker;
-import org.apache.accumulo.core.util.OpTimer;
-import org.apache.accumulo.trace.instrument.Tracer;
-import org.apache.log4j.Level;
-import org.apache.log4j.Logger;
-
-public class NamespaceOperationsImpl extends NamespaceOperationsHelper {
-  private Instance instance;
-  private Credentials credentials;
-  private TableOperationsImpl tableOps;
-
-  private static final Logger log = Logger.getLogger(TableOperations.class);
-
-  public NamespaceOperationsImpl(Instance instance, Credentials credentials, TableOperationsImpl tableOps) {
-    ArgumentChecker.notNull(instance, credentials);
-    this.instance = instance;
-    this.credentials = credentials;
-    this.tableOps = tableOps;
-  }
-
-  @Override
-  public SortedSet<String> list() {
-    OpTimer opTimer = new OpTimer(log, Level.TRACE).start("Fetching list of namespaces...");
-    TreeSet<String> namespaces = new TreeSet<String>(Namespaces.getNameToIdMap(instance).keySet());
-    opTimer.stop("Fetched " + namespaces.size() + " namespaces in %DURATION%");
-    return namespaces;
-  }
-
-  @Override
-  public boolean exists(String namespace) {
-    ArgumentChecker.notNull(namespace);
-
-    OpTimer opTimer = new OpTimer(log, Level.TRACE).start("Checking if namespace " + namespace + " exists...");
-    boolean exists = Namespaces.getNameToIdMap(instance).containsKey(namespace);
-    opTimer.stop("Checked existance of " + exists + " in %DURATION%");
-    return exists;
-  }
-
-  @Override
-  public void create(String namespace) throws AccumuloException, AccumuloSecurityException, NamespaceExistsException {
-    ArgumentChecker.notNull(namespace);
-
-    try {
-      doNamespaceFateOperation(FateOperation.NAMESPACE_CREATE, Arrays.asList(ByteBuffer.wrap(namespace.getBytes())), Collections.<String,String> emptyMap());
-    } catch (NamespaceNotFoundException e) {
-      // should not happen
-      throw new AssertionError(e);
-    }
-  }
-
-  @Override
-  public void delete(String namespace) throws AccumuloException, AccumuloSecurityException, NamespaceNotFoundException, NamespaceNotEmptyException {
-    ArgumentChecker.notNull(namespace);
-    String namespaceId = Namespaces.getNamespaceId(instance, namespace);
-
-    if (namespaceId.equals(Namespaces.ACCUMULO_NAMESPACE_ID) || namespaceId.equals(Namespaces.DEFAULT_NAMESPACE_ID)) {
-      log.debug(credentials.getPrincipal() + " attempted to delete the " + namespaceId + " namespace");
-      throw new AccumuloSecurityException(credentials.getPrincipal(), SecurityErrorCode.UNSUPPORTED_OPERATION);
-    }
-
-    if (Namespaces.getTableIds(instance, namespaceId).size() > 0) {
-      throw new NamespaceNotEmptyException(namespaceId, namespace, null);
-    }
-
-    List<ByteBuffer> args = Arrays.asList(ByteBuffer.wrap(namespace.getBytes()));
-    Map<String,String> opts = new HashMap<String,String>();
-
-    try {
-      doNamespaceFateOperation(FateOperation.NAMESPACE_DELETE, args, opts);
-    } catch (NamespaceExistsException e) {
-      // should not happen
-      throw new AssertionError(e);
-    }
-
-  }
-
-  @Override
-  public void rename(String oldNamespaceName, String newNamespaceName) throws AccumuloSecurityException, NamespaceNotFoundException, AccumuloException,
-      NamespaceExistsException {
-
-    List<ByteBuffer> args = Arrays.asList(ByteBuffer.wrap(oldNamespaceName.getBytes()), ByteBuffer.wrap(newNamespaceName.getBytes()));
-    Map<String,String> opts = new HashMap<String,String>();
-    doNamespaceFateOperation(FateOperation.NAMESPACE_RENAME, args, opts);
-  }
-
-  @Override
-  public void setProperty(final String namespace, final String property, final String value) throws AccumuloException, AccumuloSecurityException,
-      NamespaceNotFoundException {
-    ArgumentChecker.notNull(namespace, property, value);
-
-    MasterClient.executeNamespace(instance, new ClientExec<MasterClientService.Client>() {
-      @Override
-      public void execute(MasterClientService.Client client) throws Exception {
-        client.setNamespaceProperty(Tracer.traceInfo(), credentials.toThrift(instance), namespace, property, value);
-      }
-    });
-  }
-
-  @Override
-  public void removeProperty(final String namespace, final String property) throws AccumuloException, AccumuloSecurityException, NamespaceNotFoundException {
-    ArgumentChecker.notNull(namespace, property);
-
-    MasterClient.executeNamespace(instance, new ClientExec<MasterClientService.Client>() {
-      @Override
-      public void execute(MasterClientService.Client client) throws Exception {
-        client.removeNamespaceProperty(Tracer.traceInfo(), credentials.toThrift(instance), namespace, property);
-      }
-    });
-  }
-
-  @Override
-  public Iterable<Entry<String,String>> getProperties(final String namespace) throws AccumuloException, NamespaceNotFoundException {
-    ArgumentChecker.notNull(namespace);
-    try {
-      return ServerClient.executeRaw(instance, new ClientExecReturn<Map<String,String>,ClientService.Client>() {
-        @Override
-        public Map<String,String> execute(ClientService.Client client) throws Exception {
-          return client.getNamespaceConfiguration(Tracer.traceInfo(), credentials.toThrift(instance), namespace);
-        }
-      }).entrySet();
-    } catch (ThriftTableOperationException e) {
-      switch (e.getType()) {
-        case NAMESPACE_NOTFOUND:
-          throw new NamespaceNotFoundException(e);
-        case OTHER:
-        default:
-          throw new AccumuloException(e.description, e);
-      }
-    } catch (AccumuloException e) {
-      throw e;
-    } catch (Exception e) {
-      throw new AccumuloException(e);
-    }
-
-  }
-
-  @Override
-  public Map<String,String> namespaceIdMap() {
-    return Namespaces.getNameToIdMap(instance);
-  }
-
-  @Override
-  public boolean testClassLoad(final String namespace, final String className, final String asTypeName) throws NamespaceNotFoundException, AccumuloException,
-      AccumuloSecurityException {
-    ArgumentChecker.notNull(namespace, className, asTypeName);
-
-    try {
-      return ServerClient.executeRaw(instance, new ClientExecReturn<Boolean,ClientService.Client>() {
-        @Override
-        public Boolean execute(ClientService.Client client) throws Exception {
-          return client.checkNamespaceClass(Tracer.traceInfo(), credentials.toThrift(instance), namespace, className, asTypeName);
-        }
-      });
-    } catch (ThriftTableOperationException e) {
-      switch (e.getType()) {
-        case NAMESPACE_NOTFOUND:
-          throw new NamespaceNotFoundException(e);
-        default:
-          throw new AccumuloException(e.description, e);
-      }
-    } catch (ThriftSecurityException e) {
-      throw new AccumuloSecurityException(e.user, e.code, e);
-    } catch (AccumuloException e) {
-      throw e;
-    } catch (Exception e) {
-      throw new AccumuloException(e);
-    }
-  }
-
-  @Override
-  public void attachIterator(String namespace, IteratorSetting setting, EnumSet<IteratorScope> scopes) throws AccumuloSecurityException, AccumuloException,
-      NamespaceNotFoundException {
-    testClassLoad(namespace, setting.getIteratorClass(), SortedKeyValueIterator.class.getName());
-    super.attachIterator(namespace, setting, scopes);
-  }
-
-  @Override
-  public int addConstraint(String namespace, String constraintClassName) throws AccumuloException, AccumuloSecurityException, NamespaceNotFoundException {
-    testClassLoad(namespace, constraintClassName, Constraint.class.getName());
-    return super.addConstraint(namespace, constraintClassName);
-  }
-
-  private String doNamespaceFateOperation(FateOperation op, List<ByteBuffer> args, Map<String,String> opts) throws AccumuloSecurityException,
-      AccumuloException, NamespaceExistsException, NamespaceNotFoundException {
-    try {
-      return tableOps.doFateOperation(op, args, opts);
-    } catch (TableExistsException e) {
-      // should not happen
-      throw new AssertionError(e);
-    } catch (TableNotFoundException e) {
-      // should not happen
-      throw new AssertionError(e);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/f4454a06/core/src/main/java/org/apache/accumulo/core/client/admin/SecurityOperationsImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/admin/SecurityOperationsImpl.java b/core/src/main/java/org/apache/accumulo/core/client/admin/SecurityOperationsImpl.java
index 9d662f4..875cc72 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/admin/SecurityOperationsImpl.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/admin/SecurityOperationsImpl.java
@@ -16,318 +16,21 @@
  */
 package org.apache.accumulo.core.client.admin;
 
-import java.nio.ByteBuffer;
-import java.util.Set;
-
-import org.apache.accumulo.core.client.AccumuloException;
-import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.Instance;
-import org.apache.accumulo.core.client.impl.ClientExec;
-import org.apache.accumulo.core.client.impl.ClientExecReturn;
-import org.apache.accumulo.core.client.impl.ServerClient;
-import org.apache.accumulo.core.client.impl.thrift.ClientService;
-import org.apache.accumulo.core.client.impl.thrift.SecurityErrorCode;
-import org.apache.accumulo.core.client.impl.thrift.TableOperationExceptionType;
-import org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException;
-import org.apache.accumulo.core.client.impl.thrift.ThriftTableOperationException;
-import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
-import org.apache.accumulo.core.client.security.tokens.PasswordToken;
-import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.security.Credentials;
-import org.apache.accumulo.core.security.NamespacePermission;
-import org.apache.accumulo.core.security.SystemPermission;
-import org.apache.accumulo.core.security.TablePermission;
-import org.apache.accumulo.core.util.ArgumentChecker;
-import org.apache.accumulo.core.util.ByteBufferUtil;
-import org.apache.accumulo.trace.instrument.Tracer;
-
-public class SecurityOperationsImpl implements SecurityOperations {
-
-  private Instance instance;
-  private Credentials credentials;
+import org.apache.accumulo.core.security.thrift.TCredentials;
 
-  private void execute(ClientExec<ClientService.Client> exec) throws AccumuloException, AccumuloSecurityException {
-    try {
-      ServerClient.executeRaw(instance, exec);
-    } catch (ThriftTableOperationException ttoe) {
-      // recast missing table
-      if (ttoe.getType() == TableOperationExceptionType.NOTFOUND)
-        throw new AccumuloSecurityException(null, SecurityErrorCode.TABLE_DOESNT_EXIST);
-      else if (ttoe.getType() == TableOperationExceptionType.NAMESPACE_NOTFOUND)
-        throw new AccumuloSecurityException(null, SecurityErrorCode.NAMESPACE_DOESNT_EXIST);
-      else
-        throw new AccumuloException(ttoe);
-    } catch (ThriftSecurityException e) {
-      throw new AccumuloSecurityException(e.user, e.code, e);
-    } catch (AccumuloException e) {
-      throw e;
-    } catch (Exception e) {
-      throw new AccumuloException(e);
-    }
-  }
-
-  private <T> T execute(ClientExecReturn<T,ClientService.Client> exec) throws AccumuloException, AccumuloSecurityException {
-    try {
-      return ServerClient.executeRaw(instance, exec);
-    } catch (ThriftTableOperationException ttoe) {
-      // recast missing table
-      if (ttoe.getType() == TableOperationExceptionType.NOTFOUND)
-        throw new AccumuloSecurityException(null, SecurityErrorCode.TABLE_DOESNT_EXIST);
-      else if (ttoe.getType() == TableOperationExceptionType.NAMESPACE_NOTFOUND)
-        throw new AccumuloSecurityException(null, SecurityErrorCode.NAMESPACE_DOESNT_EXIST);
-      else
-        throw new AccumuloException(ttoe);
-    } catch (ThriftSecurityException e) {
-      throw new AccumuloSecurityException(e.user, e.code, e);
-    } catch (AccumuloException e) {
-      throw e;
-    } catch (Exception e) {
-      throw new AccumuloException(e);
-    }
-  }
+/**
+ * @deprecated since 1.6.0; not intended for public api and you should not use it.
+ */
+@Deprecated
+public class SecurityOperationsImpl extends org.apache.accumulo.core.client.impl.SecurityOperationsImpl {
 
   public SecurityOperationsImpl(Instance instance, Credentials credentials) {
-    ArgumentChecker.notNull(instance, credentials);
-    this.instance = instance;
-    this.credentials = credentials;
-  }
-
-  @Deprecated
-  @Override
-  public void createUser(String user, byte[] password, final Authorizations authorizations) throws AccumuloException, AccumuloSecurityException {
-    createLocalUser(user, new PasswordToken(password));
-    changeUserAuthorizations(user, authorizations);
-  }
-
-  @Override
-  public void createLocalUser(final String principal, final PasswordToken password) throws AccumuloException, AccumuloSecurityException {
-    ArgumentChecker.notNull(principal, password);
-    execute(new ClientExec<ClientService.Client>() {
-      @Override
-      public void execute(ClientService.Client client) throws Exception {
-        client.createLocalUser(Tracer.traceInfo(), credentials.toThrift(instance), principal, ByteBuffer.wrap(password.getPassword()));
-      }
-    });
-  }
-
-  @Deprecated
-  @Override
-  public void dropUser(final String user) throws AccumuloException, AccumuloSecurityException {
-    dropLocalUser(user);
-  }
-
-  @Override
-  public void dropLocalUser(final String principal) throws AccumuloException, AccumuloSecurityException {
-    ArgumentChecker.notNull(principal);
-    execute(new ClientExec<ClientService.Client>() {
-      @Override
-      public void execute(ClientService.Client client) throws Exception {
-        client.dropLocalUser(Tracer.traceInfo(), credentials.toThrift(instance), principal);
-      }
-    });
-  }
-
-  @Deprecated
-  @Override
-  public boolean authenticateUser(String user, byte[] password) throws AccumuloException, AccumuloSecurityException {
-    return authenticateUser(user, new PasswordToken(password));
-  }
-
-  @Override
-  public boolean authenticateUser(final String principal, final AuthenticationToken token) throws AccumuloException, AccumuloSecurityException {
-    ArgumentChecker.notNull(principal, token);
-    final Credentials toAuth = new Credentials(principal, token);
-    return execute(new ClientExecReturn<Boolean,ClientService.Client>() {
-      @Override
-      public Boolean execute(ClientService.Client client) throws Exception {
-        return client.authenticateUser(Tracer.traceInfo(), credentials.toThrift(instance), toAuth.toThrift(instance));
-      }
-    });
-  }
-
-  @Override
-  @Deprecated
-  public void changeUserPassword(String user, byte[] password) throws AccumuloException, AccumuloSecurityException {
-    changeLocalUserPassword(user, new PasswordToken(password));
-  }
-
-  @Override
-  public void changeLocalUserPassword(final String principal, final PasswordToken token) throws AccumuloException, AccumuloSecurityException {
-    ArgumentChecker.notNull(principal, token);
-    final Credentials toChange = new Credentials(principal, token);
-    execute(new ClientExec<ClientService.Client>() {
-      @Override
-      public void execute(ClientService.Client client) throws Exception {
-        client.changeLocalUserPassword(Tracer.traceInfo(), credentials.toThrift(instance), principal, ByteBuffer.wrap(token.getPassword()));
-      }
-    });
-    if (this.credentials.getPrincipal().equals(principal)) {
-      this.credentials = toChange;
-    }
-  }
-
-  @Override
-  public void changeUserAuthorizations(final String principal, final Authorizations authorizations) throws AccumuloException, AccumuloSecurityException {
-    ArgumentChecker.notNull(principal, authorizations);
-    execute(new ClientExec<ClientService.Client>() {
-      @Override
-      public void execute(ClientService.Client client) throws Exception {
-        client.changeAuthorizations(Tracer.traceInfo(), credentials.toThrift(instance), principal,
-            ByteBufferUtil.toByteBuffers(authorizations.getAuthorizations()));
-      }
-    });
-  }
-
-  @Override
-  public Authorizations getUserAuthorizations(final String principal) throws AccumuloException, AccumuloSecurityException {
-    ArgumentChecker.notNull(principal);
-    return execute(new ClientExecReturn<Authorizations,ClientService.Client>() {
-      @Override
-      public Authorizations execute(ClientService.Client client) throws Exception {
-        return new Authorizations(client.getUserAuthorizations(Tracer.traceInfo(), credentials.toThrift(instance), principal));
-      }
-    });
-  }
-
-  @Override
-  public boolean hasSystemPermission(final String principal, final SystemPermission perm) throws AccumuloException, AccumuloSecurityException {
-    ArgumentChecker.notNull(principal, perm);
-    return execute(new ClientExecReturn<Boolean,ClientService.Client>() {
-      @Override
-      public Boolean execute(ClientService.Client client) throws Exception {
-        return client.hasSystemPermission(Tracer.traceInfo(), credentials.toThrift(instance), principal, perm.getId());
-      }
-    });
-  }
-
-  @Override
-  public boolean hasTablePermission(final String principal, final String table, final TablePermission perm) throws AccumuloException, AccumuloSecurityException {
-    ArgumentChecker.notNull(principal, table, perm);
-    try {
-      return execute(new ClientExecReturn<Boolean,ClientService.Client>() {
-        @Override
-        public Boolean execute(ClientService.Client client) throws Exception {
-          return client.hasTablePermission(Tracer.traceInfo(), credentials.toThrift(instance), principal, table, perm.getId());
-        }
-      });
-    } catch (AccumuloSecurityException e) {
-      if (e.getSecurityErrorCode() == org.apache.accumulo.core.client.security.SecurityErrorCode.NAMESPACE_DOESNT_EXIST)
-        throw new AccumuloSecurityException(null, SecurityErrorCode.TABLE_DOESNT_EXIST, e);
-      else
-        throw e;
-    }
-  }
-
-  @Override
-  public boolean hasNamespacePermission(final String principal, final String namespace, final NamespacePermission permission) throws AccumuloException,
-      AccumuloSecurityException {
-    ArgumentChecker.notNull(principal, namespace, permission);
-    return execute(new ClientExecReturn<Boolean,ClientService.Client>() {
-      @Override
-      public Boolean execute(ClientService.Client client) throws Exception {
-        return client.hasNamespacePermission(Tracer.traceInfo(), credentials.toThrift(instance), principal, namespace, permission.getId());
-      }
-    });
-  }
-
-  @Override
-  public void grantSystemPermission(final String principal, final SystemPermission permission) throws AccumuloException, AccumuloSecurityException {
-    ArgumentChecker.notNull(principal, permission);
-    execute(new ClientExec<ClientService.Client>() {
-      @Override
-      public void execute(ClientService.Client client) throws Exception {
-        client.grantSystemPermission(Tracer.traceInfo(), credentials.toThrift(instance), principal, permission.getId());
-      }
-    });
-  }
-
-  @Override
-  public void grantTablePermission(final String principal, final String table, final TablePermission permission) throws AccumuloException,
-      AccumuloSecurityException {
-    ArgumentChecker.notNull(principal, table, permission);
-    try {
-      execute(new ClientExec<ClientService.Client>() {
-        @Override
-        public void execute(ClientService.Client client) throws Exception {
-          client.grantTablePermission(Tracer.traceInfo(), credentials.toThrift(instance), principal, table, permission.getId());
-        }
-      });
-    } catch (AccumuloSecurityException e) {
-      if (e.getSecurityErrorCode() == org.apache.accumulo.core.client.security.SecurityErrorCode.NAMESPACE_DOESNT_EXIST)
-        throw new AccumuloSecurityException(null, SecurityErrorCode.TABLE_DOESNT_EXIST, e);
-      else
-        throw e;
-    }
-  }
-
-  @Override
-  public void grantNamespacePermission(final String principal, final String namespace, final NamespacePermission permission) throws AccumuloException,
-      AccumuloSecurityException {
-    ArgumentChecker.notNull(principal, namespace, permission);
-    execute(new ClientExec<ClientService.Client>() {
-      @Override
-      public void execute(ClientService.Client client) throws Exception {
-        client.grantNamespacePermission(Tracer.traceInfo(), credentials.toThrift(instance), principal, namespace, permission.getId());
-      }
-    });
-  }
-
-  @Override
-  public void revokeSystemPermission(final String principal, final SystemPermission permission) throws AccumuloException, AccumuloSecurityException {
-    ArgumentChecker.notNull(principal, permission);
-    execute(new ClientExec<ClientService.Client>() {
-      @Override
-      public void execute(ClientService.Client client) throws Exception {
-        client.revokeSystemPermission(Tracer.traceInfo(), credentials.toThrift(instance), principal, permission.getId());
-      }
-    });
-  }
-
-  @Override
-  public void revokeTablePermission(final String principal, final String table, final TablePermission permission) throws AccumuloException,
-      AccumuloSecurityException {
-    ArgumentChecker.notNull(principal, table, permission);
-    try {
-      execute(new ClientExec<ClientService.Client>() {
-        @Override
-        public void execute(ClientService.Client client) throws Exception {
-          client.revokeTablePermission(Tracer.traceInfo(), credentials.toThrift(instance), principal, table, permission.getId());
-        }
-      });
-    } catch (AccumuloSecurityException e) {
-      if (e.getSecurityErrorCode() == org.apache.accumulo.core.client.security.SecurityErrorCode.NAMESPACE_DOESNT_EXIST)
-        throw new AccumuloSecurityException(null, SecurityErrorCode.TABLE_DOESNT_EXIST, e);
-      else
-        throw e;
-    }
-  }
-
-  @Override
-  public void revokeNamespacePermission(final String principal, final String namespace, final NamespacePermission permission) throws AccumuloException,
-      AccumuloSecurityException {
-    ArgumentChecker.notNull(principal, namespace, permission);
-    execute(new ClientExec<ClientService.Client>() {
-      @Override
-      public void execute(ClientService.Client client) throws Exception {
-        client.revokeNamespacePermission(Tracer.traceInfo(), credentials.toThrift(instance), principal, namespace, permission.getId());
-      }
-    });
+    super(instance, credentials);
   }
 
-  @Deprecated
-  @Override
-  public Set<String> listUsers() throws AccumuloException, AccumuloSecurityException {
-    return listLocalUsers();
+  public SecurityOperationsImpl(Instance instance, TCredentials credentials) {
+    this(instance, Credentials.fromThrift(credentials));
   }
-
-  @Override
-  public Set<String> listLocalUsers() throws AccumuloException, AccumuloSecurityException {
-    return execute(new ClientExecReturn<Set<String>,ClientService.Client>() {
-      @Override
-      public Set<String> execute(ClientService.Client client) throws Exception {
-        return client.listLocalUsers(Tracer.traceInfo(), credentials.toThrift(instance));
-      }
-    });
-  }
-
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/f4454a06/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperationsHelper.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperationsHelper.java b/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperationsHelper.java
index 843f572..1384a54 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperationsHelper.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperationsHelper.java
@@ -16,183 +16,9 @@
  */
 package org.apache.accumulo.core.client.admin;
 
-import java.util.EnumSet;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.TreeMap;
-import java.util.TreeSet;
-
-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.TableNotFoundException;
-import org.apache.accumulo.core.conf.Property;
-import org.apache.accumulo.core.iterators.IteratorUtil.IteratorScope;
-import org.apache.accumulo.core.util.ArgumentChecker;
-
-public abstract class TableOperationsHelper implements TableOperations {
-
-  @Override
-  public void attachIterator(String tableName, IteratorSetting setting) throws AccumuloSecurityException, AccumuloException, TableNotFoundException {
-    attachIterator(tableName, setting, EnumSet.allOf(IteratorScope.class));
-  }
-
-  @Override
-  public void attachIterator(String tableName, IteratorSetting setting, EnumSet<IteratorScope> scopes) throws AccumuloSecurityException, AccumuloException,
-      TableNotFoundException {
-    ArgumentChecker.notNull(tableName, setting, scopes);
-    checkIteratorConflicts(tableName, setting, scopes);
-    for (IteratorScope scope : scopes) {
-      String root = String.format("%s%s.%s", Property.TABLE_ITERATOR_PREFIX, scope.name().toLowerCase(), setting.getName());
-      for (Entry<String,String> prop : setting.getOptions().entrySet()) {
-        this.setProperty(tableName, root + ".opt." + prop.getKey(), prop.getValue());
-      }
-      this.setProperty(tableName, root, setting.getPriority() + "," + setting.getIteratorClass());
-    }
-  }
-
-  @Override
-  public void removeIterator(String tableName, String name, EnumSet<IteratorScope> scopes) throws AccumuloSecurityException, AccumuloException,
-      TableNotFoundException {
-    Map<String,String> copy = new TreeMap<String,String>();
-    for (Entry<String,String> property : this.getProperties(tableName)) {
-      copy.put(property.getKey(), property.getValue());
-    }
-    for (IteratorScope scope : scopes) {
-      String root = String.format("%s%s.%s", Property.TABLE_ITERATOR_PREFIX, scope.name().toLowerCase(), name);
-      for (Entry<String,String> property : copy.entrySet()) {
-        if (property.getKey().equals(root) || property.getKey().startsWith(root + ".opt."))
-          this.removeProperty(tableName, property.getKey());
-      }
-    }
-  }
-
-  @Override
-  public IteratorSetting getIteratorSetting(String tableName, String name, IteratorScope scope) throws AccumuloSecurityException, AccumuloException,
-      TableNotFoundException {
-    ArgumentChecker.notNull(tableName, name, scope);
-    int priority = -1;
-    String classname = null;
-    Map<String,String> settings = new HashMap<String,String>();
-
-    String root = String.format("%s%s.%s", Property.TABLE_ITERATOR_PREFIX, scope.name().toLowerCase(), name);
-    String opt = root + ".opt.";
-    for (Entry<String,String> property : this.getProperties(tableName)) {
-      if (property.getKey().equals(root)) {
-        String parts[] = property.getValue().split(",");
-        if (parts.length != 2) {
-          throw new AccumuloException("Bad value for iterator setting: " + property.getValue());
-        }
-        priority = Integer.parseInt(parts[0]);
-        classname = parts[1];
-      } else if (property.getKey().startsWith(opt)) {
-        settings.put(property.getKey().substring(opt.length()), property.getValue());
-      }
-    }
-    if (priority <= 0 || classname == null) {
-      return null;
-    }
-    return new IteratorSetting(priority, name, classname, settings);
-  }
-
-  @Override
-  public Map<String,EnumSet<IteratorScope>> listIterators(String tableName) throws AccumuloSecurityException, AccumuloException, TableNotFoundException {
-    Map<String,EnumSet<IteratorScope>> result = new TreeMap<String,EnumSet<IteratorScope>>();
-    for (Entry<String,String> property : this.getProperties(tableName)) {
-      String name = property.getKey();
-      String[] parts = name.split("\\.");
-      if (parts.length == 4) {
-        if (parts[0].equals("table") && parts[1].equals("iterator")) {
-          IteratorScope scope = IteratorScope.valueOf(parts[2]);
-          if (!result.containsKey(parts[3]))
-            result.put(parts[3], EnumSet.noneOf(IteratorScope.class));
-          result.get(parts[3]).add(scope);
-        }
-      }
-    }
-    return result;
-  }
-
-  @Override
-  public void checkIteratorConflicts(String tableName, IteratorSetting setting, EnumSet<IteratorScope> scopes) throws AccumuloException, TableNotFoundException {
-    ArgumentChecker.notNull(tableName, setting, scopes);
-    for (IteratorScope scope : scopes) {
-      String scopeStr = String.format("%s%s", Property.TABLE_ITERATOR_PREFIX, scope.name().toLowerCase());
-      String nameStr = String.format("%s.%s", scopeStr, setting.getName());
-      String optStr = String.format("%s.opt.", nameStr);
-      Map<String,String> optionConflicts = new TreeMap<String,String>();
-      for (Entry<String,String> property : this.getProperties(tableName)) {
-        if (property.getKey().startsWith(scopeStr)) {
-          if (property.getKey().equals(nameStr))
-            throw new AccumuloException(new IllegalArgumentException("iterator name conflict for " + setting.getName() + ": " + property.getKey() + "="
-                + property.getValue()));
-          if (property.getKey().startsWith(optStr))
-            optionConflicts.put(property.getKey(), property.getValue());
-          if (property.getKey().contains(".opt."))
-            continue;
-          String parts[] = property.getValue().split(",");
-          if (parts.length != 2)
-            throw new AccumuloException("Bad value for existing iterator setting: " + property.getKey() + "=" + property.getValue());
-          try {
-            if (Integer.parseInt(parts[0]) == setting.getPriority())
-              throw new AccumuloException(new IllegalArgumentException("iterator priority conflict: " + property.getKey() + "=" + property.getValue()));
-          } catch (NumberFormatException e) {
-            throw new AccumuloException("Bad value for existing iterator setting: " + property.getKey() + "=" + property.getValue());
-          }
-        }
-      }
-      if (optionConflicts.size() > 0)
-        throw new AccumuloException(new IllegalArgumentException("iterator options conflict for " + setting.getName() + ": " + optionConflicts));
-    }
-  }
-
-  @Override
-  public int addConstraint(String tableName, String constraintClassName) throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
-    TreeSet<Integer> constraintNumbers = new TreeSet<Integer>();
-    TreeMap<String,Integer> constraintClasses = new TreeMap<String,Integer>();
-    int i;
-    for (Entry<String,String> property : this.getProperties(tableName)) {
-      if (property.getKey().startsWith(Property.TABLE_CONSTRAINT_PREFIX.toString())) {
-        try {
-          i = Integer.parseInt(property.getKey().substring(Property.TABLE_CONSTRAINT_PREFIX.toString().length()));
-        } catch (NumberFormatException e) {
-          throw new AccumuloException("Bad key for existing constraint: " + property.toString());
-        }
-        constraintNumbers.add(i);
-        constraintClasses.put(property.getValue(), i);
-      }
-    }
-    i = 1;
-    while (constraintNumbers.contains(i))
-      i++;
-    if (constraintClasses.containsKey(constraintClassName))
-      throw new AccumuloException("Constraint " + constraintClassName + " already exists for table " + tableName + " with number "
-          + constraintClasses.get(constraintClassName));
-    this.setProperty(tableName, Property.TABLE_CONSTRAINT_PREFIX.toString() + i, constraintClassName);
-    return i;
-  }
-
-  @Override
-  public void removeConstraint(String tableName, int number) throws AccumuloException, AccumuloSecurityException {
-    this.removeProperty(tableName, Property.TABLE_CONSTRAINT_PREFIX.toString() + number);
-  }
-
-  @Override
-  public Map<String,Integer> listConstraints(String tableName) throws AccumuloException, TableNotFoundException {
-    Map<String,Integer> constraints = new TreeMap<String,Integer>();
-    for (Entry<String,String> property : this.getProperties(tableName)) {
-      if (property.getKey().startsWith(Property.TABLE_CONSTRAINT_PREFIX.toString())) {
-        if (constraints.containsKey(property.getValue()))
-          throw new AccumuloException("Same constraint configured twice: " + property.getKey() + "=" + Property.TABLE_CONSTRAINT_PREFIX
-              + constraints.get(property.getValue()) + "=" + property.getKey());
-        try {
-          constraints.put(property.getValue(), Integer.parseInt(property.getKey().substring(Property.TABLE_CONSTRAINT_PREFIX.toString().length())));
-        } catch (NumberFormatException e) {
-          throw new AccumuloException("Bad key for existing constraint: " + property.toString());
-        }
-      }
-    }
-    return constraints;
-  }
+/**
+ * @deprecated since 1.6.0; not intended for public api and you should not use it.
+ */
+@Deprecated
+public abstract class TableOperationsHelper extends org.apache.accumulo.core.client.impl.TableOperationsHelper {
 }


[16/17] git commit: ACCUMULO-2726 cleanup for master's branch.

Posted by bu...@apache.org.
ACCUMULO-2726 cleanup for master's branch.

* remove things we deprecated in 1.6.0 related to this ticket.
* make sure changes in the implementations in master are reflected in the new implementation locations
* make names in Mock consistent


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

Branch: refs/heads/master
Commit: f277fea98588b5e9f97d237a6479b9dfa08a37a1
Parents: 592c8b0
Author: Sean Busbey <bu...@cloudera.com>
Authored: Fri Apr 25 12:45:22 2014 -0500
Committer: Sean Busbey <bu...@cloudera.com>
Committed: Fri Apr 25 16:08:24 2014 -0500

----------------------------------------------------------------------
 .../client/admin/InstanceOperationsImpl.java    |  216 ---
 .../client/admin/SecurityOperationsImpl.java    |  291 ----
 .../client/admin/TableOperationsHelper.java     |  208 ---
 .../core/client/admin/TableOperationsImpl.java  | 1581 ------------------
 .../core/client/impl/ConnectorImpl.java         |    4 -
 .../client/impl/InstanceOperationsImpl.java     |   18 +-
 .../client/impl/SecurityOperationsImpl.java     |   54 +-
 .../core/client/impl/TableOperationsHelper.java |   14 +-
 .../core/client/impl/TableOperationsImpl.java   |   99 +-
 .../core/client/mock/MockConnector.java         |    6 +-
 .../client/mock/MockInstanceOperations.java     |   73 +-
 .../client/mock/MockInstanceOperationsImpl.java |   87 -
 .../client/mock/MockSecurityOperations.java     |  209 ++-
 .../client/mock/MockSecurityOperationsImpl.java |  225 ---
 .../core/client/mock/MockTableOperations.java   |  431 ++++-
 .../client/mock/MockTableOperationsImpl.java    |  447 -----
 .../core/client/mock/MockTabletLocator.java     |   53 -
 .../client/admin/TableOperationsHelperTest.java |  223 ---
 .../apache/accumulo/shell/mock/MockShell.java   |   39 -
 19 files changed, 813 insertions(+), 3465 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/f277fea9/core/src/main/java/org/apache/accumulo/core/client/admin/InstanceOperationsImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/admin/InstanceOperationsImpl.java b/core/src/main/java/org/apache/accumulo/core/client/admin/InstanceOperationsImpl.java
deleted file mode 100644
index d3a6285..0000000
--- a/core/src/main/java/org/apache/accumulo/core/client/admin/InstanceOperationsImpl.java
+++ /dev/null
@@ -1,216 +0,0 @@
-/*
- * 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.core.client.admin;
-
-import static com.google.common.base.Preconditions.checkArgument;
-
-import java.nio.charset.StandardCharsets;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-import java.util.Map;
-
-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.Instance;
-import org.apache.accumulo.core.security.Credentials;
-import org.apache.accumulo.core.tabletserver.thrift.TabletClientService;
-import org.apache.accumulo.core.tabletserver.thrift.TabletClientService.Client;
-import org.apache.accumulo.core.util.AddressUtil;
-import org.apache.accumulo.core.util.ThriftUtil;
-import org.apache.accumulo.core.zookeeper.ZooUtil;
-import org.apache.accumulo.fate.zookeeper.ZooCache;
-import org.apache.accumulo.trace.instrument.Tracer;
-import org.apache.thrift.TException;
-import org.apache.thrift.transport.TTransport;
-import org.apache.thrift.transport.TTransportException;
-import org.apache.accumulo.core.security.thrift.TCredentials;
-
-/**
- * Provides a class for administering the accumulo instance
- * @deprecated since 1.6.0; not intended for public api and you should not use it.
- */
-@Deprecated
-public class InstanceOperationsImpl extends org.apache.accumulo.core.client.impl.InstanceOperationsImpl {
-  
-  /**
-   * @param instance
-   *          the connection information for this instance
-   * @param credentials
-   *          the Credential, containing principal and Authentication Token
-   */
-  public InstanceOperationsImpl(Instance instance, Credentials credentials) {
-    checkArgument(instance != null, "instance is null");
-    checkArgument(credentials != null, "credentials is null");
-    this.instance = instance;
-    this.credentials = credentials;
-  }
-  
-  @Override
-  public void setProperty(final String property, final String value) throws AccumuloException, AccumuloSecurityException {
-    checkArgument(property != null, "property is null");
-    checkArgument(value != null, "value is null");
-    MasterClient.execute(instance, new ClientExec<MasterClientService.Client>() {
-      @Override
-      public void execute(MasterClientService.Client client) throws Exception {
-        client.setSystemProperty(Tracer.traceInfo(), credentials.toThrift(instance), property, value);
-      }
-    });
-  }
-  
-  @Override
-  public void removeProperty(final String property) throws AccumuloException, AccumuloSecurityException {
-    checkArgument(property != null, "property is null");
-    MasterClient.execute(instance, new ClientExec<MasterClientService.Client>() {
-      @Override
-      public void execute(MasterClientService.Client client) throws Exception {
-        client.removeSystemProperty(Tracer.traceInfo(), credentials.toThrift(instance), property);
-      }
-    });
-  }
-  
-  @Override
-  public Map<String,String> getSystemConfiguration() throws AccumuloException, AccumuloSecurityException {
-    return ServerClient.execute(instance, new ClientExecReturn<Map<String,String>,ClientService.Client>() {
-      @Override
-      public Map<String,String> execute(ClientService.Client client) throws Exception {
-        return client.getConfiguration(Tracer.traceInfo(), credentials.toThrift(instance), ConfigurationType.CURRENT);
-      }
-    });
-  }
-  
-  @Override
-  public Map<String,String> getSiteConfiguration() throws AccumuloException, AccumuloSecurityException {
-    return ServerClient.execute(instance, new ClientExecReturn<Map<String,String>,ClientService.Client>() {
-      @Override
-      public Map<String,String> execute(ClientService.Client client) throws Exception {
-        return client.getConfiguration(Tracer.traceInfo(), credentials.toThrift(instance), ConfigurationType.SITE);
-      }
-    });
-  }
-  
-  @Override
-  public List<String> getTabletServers() {
-    ZooCache cache = ZooCache.getInstance(instance.getZooKeepers(), instance.getZooKeepersSessionTimeOut());
-    String path = ZooUtil.getRoot(instance) + Constants.ZTSERVERS;
-    List<String> results = new ArrayList<String>();
-    for (String candidate : cache.getChildren(path)) {
-      List<String> children = cache.getChildren(path + "/" + candidate);
-      if (children != null && children.size() > 0) {
-        List<String> copy = new ArrayList<String>(children);
-        Collections.sort(copy);
-        byte[] data = cache.get(path + "/" + candidate + "/" + copy.get(0));
-        if (data != null && !"master".equals(new String(data, StandardCharsets.UTF_8))) {
-          results.add(candidate);
-        }
-      }
-    }
-    return results;
-  }
-  
-  @Override
-  public List<ActiveScan> getActiveScans(String tserver) throws AccumuloException, AccumuloSecurityException {
-    Client client = null;
-    try {
-      client = ThriftUtil.getTServerClient(tserver, ServerConfigurationUtil.getConfiguration(instance));
-      
-      List<ActiveScan> as = new ArrayList<ActiveScan>();
-      for (org.apache.accumulo.core.tabletserver.thrift.ActiveScan activeScan : client.getActiveScans(Tracer.traceInfo(), credentials.toThrift(instance))) {
-        try {
-          as.add(new ActiveScan(instance, activeScan));
-        } catch (TableNotFoundException e) {
-          throw new AccumuloException(e);
-        }
-      }
-      return as;
-    } catch (TTransportException e) {
-      throw new AccumuloException(e);
-    } catch (ThriftSecurityException e) {
-      throw new AccumuloSecurityException(e.user, e.code, e);
-    } catch (TException e) {
-      throw new AccumuloException(e);
-    } finally {
-      if (client != null)
-        ThriftUtil.returnClient(client);
-    }
-  }
-  
-  @Override
-  public boolean testClassLoad(final String className, final String asTypeName) throws AccumuloException, AccumuloSecurityException {
-    return ServerClient.execute(instance, new ClientExecReturn<Boolean,ClientService.Client>() {
-      @Override
-      public Boolean execute(ClientService.Client client) throws Exception {
-        return client.checkClass(Tracer.traceInfo(), credentials.toThrift(instance), className, asTypeName);
-      }
-    });
-  }
-  
-  @Override
-  public List<ActiveCompaction> getActiveCompactions(String tserver) throws AccumuloException, AccumuloSecurityException {
-    Client client = null;
-    try {
-      client = ThriftUtil.getTServerClient(tserver, ServerConfigurationUtil.getConfiguration(instance));
-      
-      List<ActiveCompaction> as = new ArrayList<ActiveCompaction>();
-      for (org.apache.accumulo.core.tabletserver.thrift.ActiveCompaction activeCompaction : client.getActiveCompactions(Tracer.traceInfo(),
-          credentials.toThrift(instance))) {
-        as.add(new ActiveCompaction(instance, activeCompaction));
-      }
-      return as;
-    } catch (TTransportException e) {
-      throw new AccumuloException(e);
-    } catch (ThriftSecurityException e) {
-      throw new AccumuloSecurityException(e.user, e.code, e);
-    } catch (TException e) {
-      throw new AccumuloException(e);
-    } finally {
-      if (client != null)
-        ThriftUtil.returnClient(client);
-    }
-  }
-  
-  @Override
-  public void ping(String tserver) throws AccumuloException {
-    TTransport transport = null;
-    try {
-      transport = ThriftUtil.createTransport(AddressUtil.parseAddress(tserver, false), ServerConfigurationUtil.getConfiguration(instance));
-      TabletClientService.Client client = ThriftUtil.createClient(new TabletClientService.Client.Factory(), transport);
-      client.getTabletServerStatus(Tracer.traceInfo(), credentials.toThrift(instance));
-    } catch (TTransportException e) {
-      throw new AccumuloException(e);
-    } catch (ThriftSecurityException e) {
-      throw new AccumuloException(e);
-    } catch (TException e) {
-      throw new AccumuloException(e);
-    } finally {
-      if (transport != null) {
-        transport.close();
-      }
-    }
-  }
-  
-  /**
-   * @param instance
-   *          the connection information for this instance
-   * @param credentials
-   *          the Credential, containing principal and Authentication Token
-   */
-  public InstanceOperationsImpl(Instance instance, TCredentials credentials) {
-    this(instance, Credentials.fromThrift(credentials));
-  }
-}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/f277fea9/core/src/main/java/org/apache/accumulo/core/client/admin/SecurityOperationsImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/admin/SecurityOperationsImpl.java b/core/src/main/java/org/apache/accumulo/core/client/admin/SecurityOperationsImpl.java
deleted file mode 100644
index 3d647a0..0000000
--- a/core/src/main/java/org/apache/accumulo/core/client/admin/SecurityOperationsImpl.java
+++ /dev/null
@@ -1,291 +0,0 @@
-/*
- * 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.core.client.admin;
-
-import static com.google.common.base.Preconditions.checkArgument;
-import java.nio.ByteBuffer;
-import java.util.Set;
-
-import org.apache.accumulo.core.client.AccumuloException;
-import org.apache.accumulo.core.client.AccumuloSecurityException;
-import org.apache.accumulo.core.client.Instance;
-import org.apache.accumulo.core.security.Credentials;
-import org.apache.accumulo.core.security.NamespacePermission;
-import org.apache.accumulo.core.security.SystemPermission;
-import org.apache.accumulo.core.security.TablePermission;
-import org.apache.accumulo.core.util.ByteBufferUtil;
-import org.apache.accumulo.trace.instrument.Tracer;
-import org.apache.accumulo.core.security.thrift.TCredentials;
-
-/**
- * @deprecated since 1.6.0; not intended for public api and you should not use it.
- */
-@Deprecated
-public class SecurityOperationsImpl extends org.apache.accumulo.core.client.impl.SecurityOperationsImpl {
-
-  public SecurityOperationsImpl(Instance instance, Credentials credentials) {
-    checkArgument(instance != null, "instance is null");
-    checkArgument(credentials != null, "credentials is null");
-    this.instance = instance;
-    this.credentials = credentials;
-  }
-
-  @Deprecated
-  @Override
-  public void createUser(String user, byte[] password, final Authorizations authorizations) throws AccumuloException, AccumuloSecurityException {
-    createLocalUser(user, new PasswordToken(password));
-    changeUserAuthorizations(user, authorizations);
-  }
-
-  @Override
-  public void createLocalUser(final String principal, final PasswordToken password) throws AccumuloException, AccumuloSecurityException {
-    checkArgument(principal != null, "principal is null");
-    checkArgument(password != null, "password is null");
-    execute(new ClientExec<ClientService.Client>() {
-      @Override
-      public void execute(ClientService.Client client) throws Exception {
-        client.createLocalUser(Tracer.traceInfo(), credentials.toThrift(instance), principal, ByteBuffer.wrap(password.getPassword()));
-      }
-    });
-  }
-
-  @Deprecated
-  @Override
-  public void dropUser(final String user) throws AccumuloException, AccumuloSecurityException {
-    dropLocalUser(user);
-  }
-
-  @Override
-  public void dropLocalUser(final String principal) throws AccumuloException, AccumuloSecurityException {
-    checkArgument(principal != null, "principal is null");
-    execute(new ClientExec<ClientService.Client>() {
-      @Override
-      public void execute(ClientService.Client client) throws Exception {
-        client.dropLocalUser(Tracer.traceInfo(), credentials.toThrift(instance), principal);
-      }
-    });
-  }
-
-  @Deprecated
-  @Override
-  public boolean authenticateUser(String user, byte[] password) throws AccumuloException, AccumuloSecurityException {
-    return authenticateUser(user, new PasswordToken(password));
-  }
-
-  @Override
-  public boolean authenticateUser(final String principal, final AuthenticationToken token) throws AccumuloException, AccumuloSecurityException {
-    checkArgument(principal != null, "principal is null");
-    checkArgument(token != null, "token is null");
-    final Credentials toAuth = new Credentials(principal, token);
-    return execute(new ClientExecReturn<Boolean,ClientService.Client>() {
-      @Override
-      public Boolean execute(ClientService.Client client) throws Exception {
-        return client.authenticateUser(Tracer.traceInfo(), credentials.toThrift(instance), toAuth.toThrift(instance));
-      }
-    });
-  }
-
-  @Override
-  @Deprecated
-  public void changeUserPassword(String user, byte[] password) throws AccumuloException, AccumuloSecurityException {
-    changeLocalUserPassword(user, new PasswordToken(password));
-  }
-
-  @Override
-  public void changeLocalUserPassword(final String principal, final PasswordToken token) throws AccumuloException, AccumuloSecurityException {
-    checkArgument(principal != null, "principal is null");
-    checkArgument(token != null, "token is null");
-    final Credentials toChange = new Credentials(principal, token);
-    execute(new ClientExec<ClientService.Client>() {
-      @Override
-      public void execute(ClientService.Client client) throws Exception {
-        client.changeLocalUserPassword(Tracer.traceInfo(), credentials.toThrift(instance), principal, ByteBuffer.wrap(token.getPassword()));
-      }
-    });
-    if (this.credentials.getPrincipal().equals(principal)) {
-      this.credentials = toChange;
-    }
-  }
-
-  @Override
-  public void changeUserAuthorizations(final String principal, final Authorizations authorizations) throws AccumuloException, AccumuloSecurityException {
-    checkArgument(principal != null, "principal is null");
-    checkArgument(authorizations != null, "authorizations is null");
-    execute(new ClientExec<ClientService.Client>() {
-      @Override
-      public void execute(ClientService.Client client) throws Exception {
-        client.changeAuthorizations(Tracer.traceInfo(), credentials.toThrift(instance), principal,
-            ByteBufferUtil.toByteBuffers(authorizations.getAuthorizations()));
-      }
-    });
-  }
-
-  @Override
-  public Authorizations getUserAuthorizations(final String principal) throws AccumuloException, AccumuloSecurityException {
-    checkArgument(principal != null, "principal is null");
-    return execute(new ClientExecReturn<Authorizations,ClientService.Client>() {
-      @Override
-      public Authorizations execute(ClientService.Client client) throws Exception {
-        return new Authorizations(client.getUserAuthorizations(Tracer.traceInfo(), credentials.toThrift(instance), principal));
-      }
-    });
-  }
-
-  @Override
-  public boolean hasSystemPermission(final String principal, final SystemPermission perm) throws AccumuloException, AccumuloSecurityException {
-    checkArgument(principal != null, "principal is null");
-    checkArgument(perm != null, "perm is null");
-    return execute(new ClientExecReturn<Boolean,ClientService.Client>() {
-      @Override
-      public Boolean execute(ClientService.Client client) throws Exception {
-        return client.hasSystemPermission(Tracer.traceInfo(), credentials.toThrift(instance), principal, perm.getId());
-      }
-    });
-  }
-
-  @Override
-  public boolean hasTablePermission(final String principal, final String table, final TablePermission perm) throws AccumuloException, AccumuloSecurityException {
-    checkArgument(principal != null, "principal is null");
-    checkArgument(table != null, "table is null");
-    checkArgument(perm != null, "perm is null");
-    try {
-      return execute(new ClientExecReturn<Boolean,ClientService.Client>() {
-        @Override
-        public Boolean execute(ClientService.Client client) throws Exception {
-          return client.hasTablePermission(Tracer.traceInfo(), credentials.toThrift(instance), principal, table, perm.getId());
-        }
-      });
-    } catch (AccumuloSecurityException e) {
-      if (e.getSecurityErrorCode() == org.apache.accumulo.core.client.security.SecurityErrorCode.NAMESPACE_DOESNT_EXIST)
-        throw new AccumuloSecurityException(null, SecurityErrorCode.TABLE_DOESNT_EXIST, e);
-      else
-        throw e;
-    }
-  }
-
-  @Override
-  public boolean hasNamespacePermission(final String principal, final String namespace, final NamespacePermission permission) throws AccumuloException,
-      AccumuloSecurityException {
-    checkArgument(principal != null, "principal is null");
-    checkArgument(namespace != null, "namespace is null");
-    checkArgument(permission != null, "permission is null");
-    return execute(new ClientExecReturn<Boolean,ClientService.Client>() {
-      @Override
-      public Boolean execute(ClientService.Client client) throws Exception {
-        return client.hasNamespacePermission(Tracer.traceInfo(), credentials.toThrift(instance), principal, namespace, permission.getId());
-      }
-    });
-  }
-
-  @Override
-  public void grantSystemPermission(final String principal, final SystemPermission permission) throws AccumuloException, AccumuloSecurityException {
-    checkArgument(principal != null, "principal is null");
-    checkArgument(permission != null, "permission is null");
-    execute(new ClientExec<ClientService.Client>() {
-      @Override
-      public void execute(ClientService.Client client) throws Exception {
-        client.grantSystemPermission(Tracer.traceInfo(), credentials.toThrift(instance), principal, permission.getId());
-      }
-    });
-  }
-
-  @Override
-  public void grantTablePermission(final String principal, final String table, final TablePermission permission) throws AccumuloException,
-      AccumuloSecurityException {
-    checkArgument(principal != null, "principal is null");
-    checkArgument(table != null, "table is null");
-    checkArgument(permission != null, "permission is null");
-    try {
-      execute(new ClientExec<ClientService.Client>() {
-        @Override
-        public void execute(ClientService.Client client) throws Exception {
-          client.grantTablePermission(Tracer.traceInfo(), credentials.toThrift(instance), principal, table, permission.getId());
-        }
-      });
-    } catch (AccumuloSecurityException e) {
-      if (e.getSecurityErrorCode() == org.apache.accumulo.core.client.security.SecurityErrorCode.NAMESPACE_DOESNT_EXIST)
-        throw new AccumuloSecurityException(null, SecurityErrorCode.TABLE_DOESNT_EXIST, e);
-      else
-        throw e;
-    }
-  }
-
-  @Override
-  public void grantNamespacePermission(final String principal, final String namespace, final NamespacePermission permission) throws AccumuloException,
-      AccumuloSecurityException {
-    checkArgument(principal != null, "principal is null");
-    checkArgument(namespace != null, "namespace is null");
-    checkArgument(permission != null, "permission is null");
-    execute(new ClientExec<ClientService.Client>() {
-      @Override
-      public void execute(ClientService.Client client) throws Exception {
-        client.grantNamespacePermission(Tracer.traceInfo(), credentials.toThrift(instance), principal, namespace, permission.getId());
-      }
-    });
-  }
-
-  @Override
-  public void revokeSystemPermission(final String principal, final SystemPermission permission) throws AccumuloException, AccumuloSecurityException {
-    checkArgument(principal != null, "principal is null");
-    checkArgument(permission != null, "permission is null");
-    execute(new ClientExec<ClientService.Client>() {
-      @Override
-      public void execute(ClientService.Client client) throws Exception {
-        client.revokeSystemPermission(Tracer.traceInfo(), credentials.toThrift(instance), principal, permission.getId());
-      }
-    });
-  }
-
-  @Override
-  public void revokeTablePermission(final String principal, final String table, final TablePermission permission) throws AccumuloException,
-      AccumuloSecurityException {
-    checkArgument(principal != null, "principal is null");
-    checkArgument(table != null, "table is null");
-    checkArgument(permission != null, "permission is null");
-    try {
-      execute(new ClientExec<ClientService.Client>() {
-        @Override
-        public void execute(ClientService.Client client) throws Exception {
-          client.revokeTablePermission(Tracer.traceInfo(), credentials.toThrift(instance), principal, table, permission.getId());
-        }
-      });
-    } catch (AccumuloSecurityException e) {
-      if (e.getSecurityErrorCode() == org.apache.accumulo.core.client.security.SecurityErrorCode.NAMESPACE_DOESNT_EXIST)
-        throw new AccumuloSecurityException(null, SecurityErrorCode.TABLE_DOESNT_EXIST, e);
-      else
-        throw e;
-    }
-  }
-
-  @Override
-  public void revokeNamespacePermission(final String principal, final String namespace, final NamespacePermission permission) throws AccumuloException,
-      AccumuloSecurityException {
-    checkArgument(principal != null, "principal is null");
-    checkArgument(namespace != null, "namespace is null");
-    checkArgument(permission != null, "permission is null");
-    execute(new ClientExec<ClientService.Client>() {
-      @Override
-      public void execute(ClientService.Client client) throws Exception {
-        client.revokeNamespacePermission(Tracer.traceInfo(), credentials.toThrift(instance), principal, namespace, permission.getId());
-      }
-    });
-  }
-
-  public SecurityOperationsImpl(Instance instance, TCredentials credentials) {
-    this(instance, Credentials.fromThrift(credentials));
-  }
-}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/f277fea9/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperationsHelper.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperationsHelper.java b/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperationsHelper.java
deleted file mode 100644
index 9d0d770..0000000
--- a/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperationsHelper.java
+++ /dev/null
@@ -1,208 +0,0 @@
-/*
- * 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.core.client.admin;
-
-import static com.google.common.base.Preconditions.checkArgument;
-import java.util.EnumSet;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.TreeMap;
-import java.util.TreeSet;
-
-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.TableNotFoundException;
-import org.apache.accumulo.core.conf.Property;
-import org.apache.accumulo.core.iterators.IteratorUtil.IteratorScope;
-
-/**
- * @deprecated since 1.6.0; not intended for public api and you should not use it.
- */
-@Deprecated
-public abstract class TableOperationsHelper extends org.apache.accumulo.core.client.impl.TableOperationsHelper {
-
-  @Override
-  public void attachIterator(String tableName, IteratorSetting setting) throws AccumuloSecurityException, AccumuloException, TableNotFoundException {
-    attachIterator(tableName, setting, EnumSet.allOf(IteratorScope.class));
-  }
-
-  @Override
-  public void attachIterator(String tableName, IteratorSetting setting, EnumSet<IteratorScope> scopes) throws AccumuloSecurityException, AccumuloException,
-      TableNotFoundException {
-    checkArgument(tableName != null, "tableName is null");
-    checkArgument(setting != null, "setting is null");
-    checkArgument(scopes != null, "scopes is null");
-    checkIteratorConflicts(tableName, setting, scopes);
-    for (IteratorScope scope : scopes) {
-      String root = String.format("%s%s.%s", Property.TABLE_ITERATOR_PREFIX, scope.name().toLowerCase(), setting.getName());
-      for (Entry<String,String> prop : setting.getOptions().entrySet()) {
-        this.setProperty(tableName, root + ".opt." + prop.getKey(), prop.getValue());
-      }
-      this.setProperty(tableName, root, setting.getPriority() + "," + setting.getIteratorClass());
-    }
-  }
-
-  @Override
-  public void removeIterator(String tableName, String name, EnumSet<IteratorScope> scopes) throws AccumuloSecurityException, AccumuloException,
-      TableNotFoundException {
-    Map<String,String> copy = new TreeMap<String,String>();
-    for (Entry<String,String> property : this.getProperties(tableName)) {
-      copy.put(property.getKey(), property.getValue());
-    }
-    for (IteratorScope scope : scopes) {
-      String root = String.format("%s%s.%s", Property.TABLE_ITERATOR_PREFIX, scope.name().toLowerCase(), name);
-      for (Entry<String,String> property : copy.entrySet()) {
-        if (property.getKey().equals(root) || property.getKey().startsWith(root + ".opt."))
-          this.removeProperty(tableName, property.getKey());
-      }
-    }
-  }
-
-  @Override
-  public IteratorSetting getIteratorSetting(String tableName, String name, IteratorScope scope) throws AccumuloSecurityException, AccumuloException,
-      TableNotFoundException {
-    checkArgument(tableName != null, "tableName is null");
-    checkArgument(name != null, "name is null");
-    checkArgument(scope != null, "scope is null");
-    int priority = -1;
-    String classname = null;
-    Map<String,String> settings = new HashMap<String,String>();
-
-    String root = String.format("%s%s.%s", Property.TABLE_ITERATOR_PREFIX, scope.name().toLowerCase(), name);
-    String opt = root + ".opt.";
-    for (Entry<String,String> property : this.getProperties(tableName)) {
-      if (property.getKey().equals(root)) {
-        String parts[] = property.getValue().split(",");
-        if (parts.length != 2) {
-          throw new AccumuloException("Bad value for iterator setting: " + property.getValue());
-        }
-        priority = Integer.parseInt(parts[0]);
-        classname = parts[1];
-      } else if (property.getKey().startsWith(opt)) {
-        settings.put(property.getKey().substring(opt.length()), property.getValue());
-      }
-    }
-    if (priority <= 0 || classname == null) {
-      return null;
-    }
-    return new IteratorSetting(priority, name, classname, settings);
-  }
-
-  @Override
-  public Map<String,EnumSet<IteratorScope>> listIterators(String tableName) throws AccumuloSecurityException, AccumuloException, TableNotFoundException {
-    Map<String,EnumSet<IteratorScope>> result = new TreeMap<String,EnumSet<IteratorScope>>();
-    for (Entry<String,String> property : this.getProperties(tableName)) {
-      String name = property.getKey();
-      String[] parts = name.split("\\.");
-      if (parts.length == 4) {
-        if (parts[0].equals("table") && parts[1].equals("iterator")) {
-          IteratorScope scope = IteratorScope.valueOf(parts[2]);
-          if (!result.containsKey(parts[3]))
-            result.put(parts[3], EnumSet.noneOf(IteratorScope.class));
-          result.get(parts[3]).add(scope);
-        }
-      }
-    }
-    return result;
-  }
-
-  @Override
-  public void checkIteratorConflicts(String tableName, IteratorSetting setting, EnumSet<IteratorScope> scopes) throws AccumuloException, TableNotFoundException {
-    checkArgument(tableName != null, "tableName is null");
-    checkArgument(setting != null, "setting is null");
-    checkArgument(scopes != null, "scopes is null");
-    for (IteratorScope scope : scopes) {
-      String scopeStr = String.format("%s%s", Property.TABLE_ITERATOR_PREFIX, scope.name().toLowerCase());
-      String nameStr = String.format("%s.%s", scopeStr, setting.getName());
-      String optStr = String.format("%s.opt.", nameStr);
-      Map<String,String> optionConflicts = new TreeMap<String,String>();
-      for (Entry<String,String> property : this.getProperties(tableName)) {
-        if (property.getKey().startsWith(scopeStr)) {
-          if (property.getKey().equals(nameStr))
-            throw new AccumuloException(new IllegalArgumentException("iterator name conflict for " + setting.getName() + ": " + property.getKey() + "="
-                + property.getValue()));
-          if (property.getKey().startsWith(optStr))
-            optionConflicts.put(property.getKey(), property.getValue());
-          if (property.getKey().contains(".opt."))
-            continue;
-          String parts[] = property.getValue().split(",");
-          if (parts.length != 2)
-            throw new AccumuloException("Bad value for existing iterator setting: " + property.getKey() + "=" + property.getValue());
-          try {
-            if (Integer.parseInt(parts[0]) == setting.getPriority())
-              throw new AccumuloException(new IllegalArgumentException("iterator priority conflict: " + property.getKey() + "=" + property.getValue()));
-          } catch (NumberFormatException e) {
-            throw new AccumuloException("Bad value for existing iterator setting: " + property.getKey() + "=" + property.getValue());
-          }
-        }
-      }
-      if (optionConflicts.size() > 0)
-        throw new AccumuloException(new IllegalArgumentException("iterator options conflict for " + setting.getName() + ": " + optionConflicts));
-    }
-  }
-
-  @Override
-  public int addConstraint(String tableName, String constraintClassName) throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
-    TreeSet<Integer> constraintNumbers = new TreeSet<Integer>();
-    TreeMap<String,Integer> constraintClasses = new TreeMap<String,Integer>();
-    int i;
-    for (Entry<String,String> property : this.getProperties(tableName)) {
-      if (property.getKey().startsWith(Property.TABLE_CONSTRAINT_PREFIX.toString())) {
-        try {
-          i = Integer.parseInt(property.getKey().substring(Property.TABLE_CONSTRAINT_PREFIX.toString().length()));
-        } catch (NumberFormatException e) {
-          throw new AccumuloException("Bad key for existing constraint: " + property.toString());
-        }
-        constraintNumbers.add(i);
-        constraintClasses.put(property.getValue(), i);
-      }
-    }
-    i = 1;
-    while (constraintNumbers.contains(i))
-      i++;
-    if (constraintClasses.containsKey(constraintClassName))
-      throw new AccumuloException("Constraint " + constraintClassName + " already exists for table " + tableName + " with number "
-          + constraintClasses.get(constraintClassName));
-    this.setProperty(tableName, Property.TABLE_CONSTRAINT_PREFIX.toString() + i, constraintClassName);
-    return i;
-  }
-
-  @Override
-  public void removeConstraint(String tableName, int number) throws AccumuloException, AccumuloSecurityException {
-    this.removeProperty(tableName, Property.TABLE_CONSTRAINT_PREFIX.toString() + number);
-  }
-
-  @Override
-  public Map<String,Integer> listConstraints(String tableName) throws AccumuloException, TableNotFoundException {
-    Map<String,Integer> constraints = new TreeMap<String,Integer>();
-    for (Entry<String,String> property : this.getProperties(tableName)) {
-      if (property.getKey().startsWith(Property.TABLE_CONSTRAINT_PREFIX.toString())) {
-        if (constraints.containsKey(property.getValue()))
-          throw new AccumuloException("Same constraint configured twice: " + property.getKey() + "=" + Property.TABLE_CONSTRAINT_PREFIX
-              + constraints.get(property.getValue()) + "=" + property.getKey());
-        try {
-          constraints.put(property.getValue(), Integer.parseInt(property.getKey().substring(Property.TABLE_CONSTRAINT_PREFIX.toString().length())));
-        } catch (NumberFormatException e) {
-          throw new AccumuloException("Bad key for existing constraint: " + property.toString());
-        }
-      }
-    }
-    return constraints;
-  }
-}


[02/17] ACCUMULO-2726 Adds back missing methods for binary backwards compat with non-Deprecated methods from 1.5.0. Marks things that we're for sure removing with @Deprecated; mostly things that shouldn't have been in public to begin with.

Posted by bu...@apache.org.
http://git-wip-us.apache.org/repos/asf/accumulo/blob/f4454a06/core/src/main/java/org/apache/accumulo/core/client/impl/TableOperationsImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/TableOperationsImpl.java b/core/src/main/java/org/apache/accumulo/core/client/impl/TableOperationsImpl.java
new file mode 100644
index 0000000..cd49986
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/TableOperationsImpl.java
@@ -0,0 +1,1589 @@
+/*
+ * 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.core.client.impl;
+
+import java.io.BufferedReader;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.EnumSet;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Random;
+import java.util.Set;
+import java.util.SortedSet;
+import java.util.TreeMap;
+import java.util.TreeSet;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.zip.ZipEntry;
+import java.util.zip.ZipInputStream;
+
+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.Instance;
+import org.apache.accumulo.core.client.IsolatedScanner;
+import org.apache.accumulo.core.client.IteratorSetting;
+import org.apache.accumulo.core.client.NamespaceExistsException;
+import org.apache.accumulo.core.client.NamespaceNotFoundException;
+import org.apache.accumulo.core.client.RowIterator;
+import org.apache.accumulo.core.client.Scanner;
+import org.apache.accumulo.core.client.TableDeletedException;
+import org.apache.accumulo.core.client.TableExistsException;
+import org.apache.accumulo.core.client.TableNotFoundException;
+import org.apache.accumulo.core.client.TableOfflineException;
+import org.apache.accumulo.core.client.admin.DiskUsage;
+import org.apache.accumulo.core.client.admin.FindMax;
+import org.apache.accumulo.core.client.admin.TableOperations;
+import org.apache.accumulo.core.client.admin.TimeType;
+import org.apache.accumulo.core.client.impl.TabletLocator.TabletLocation;
+import org.apache.accumulo.core.client.impl.thrift.ClientService;
+import org.apache.accumulo.core.client.impl.thrift.ClientService.Client;
+import org.apache.accumulo.core.client.impl.thrift.TDiskUsage;
+import org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException;
+import org.apache.accumulo.core.client.impl.thrift.ThriftTableOperationException;
+import org.apache.accumulo.core.conf.AccumuloConfiguration;
+import org.apache.accumulo.core.conf.ConfigurationCopy;
+import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.core.constraints.Constraint;
+import org.apache.accumulo.core.data.ByteSequence;
+import org.apache.accumulo.core.data.Key;
+import org.apache.accumulo.core.data.KeyExtent;
+import org.apache.accumulo.core.data.Range;
+import org.apache.accumulo.core.data.Value;
+import org.apache.accumulo.core.iterators.IteratorUtil;
+import org.apache.accumulo.core.iterators.IteratorUtil.IteratorScope;
+import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
+import org.apache.accumulo.core.master.state.tables.TableState;
+import org.apache.accumulo.core.master.thrift.FateOperation;
+import org.apache.accumulo.core.master.thrift.MasterClientService;
+import org.apache.accumulo.core.metadata.MetadataServicer;
+import org.apache.accumulo.core.metadata.MetadataTable;
+import org.apache.accumulo.core.metadata.RootTable;
+import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection;
+import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.core.security.Credentials;
+import org.apache.accumulo.core.tabletserver.thrift.NotServingTabletException;
+import org.apache.accumulo.core.tabletserver.thrift.TabletClientService;
+import org.apache.accumulo.core.util.ArgumentChecker;
+import org.apache.accumulo.core.util.ByteBufferUtil;
+import org.apache.accumulo.core.util.CachedConfiguration;
+import org.apache.accumulo.core.util.LocalityGroupUtil;
+import org.apache.accumulo.core.util.MapCounter;
+import org.apache.accumulo.core.util.NamingThreadFactory;
+import org.apache.accumulo.core.util.OpTimer;
+import org.apache.accumulo.core.util.Pair;
+import org.apache.accumulo.core.util.StringUtil;
+import org.apache.accumulo.core.util.TextUtil;
+import org.apache.accumulo.core.util.ThriftUtil;
+import org.apache.accumulo.core.util.UtilWaitThread;
+import org.apache.accumulo.core.volume.VolumeConfiguration;
+import org.apache.accumulo.trace.instrument.Tracer;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.Text;
+import org.apache.log4j.Level;
+import org.apache.log4j.Logger;
+import org.apache.thrift.TApplicationException;
+import org.apache.thrift.TException;
+import org.apache.thrift.transport.TTransportException;
+
+public class TableOperationsImpl extends TableOperationsHelper {
+  private Instance instance;
+  private Credentials credentials;
+
+  public static final String CLONE_EXCLUDE_PREFIX = "!";
+
+  private static final Logger log = Logger.getLogger(TableOperations.class);
+
+  /**
+   * @param instance
+   *          the connection information for this instance
+   * @param credentials
+   *          the username/password for this connection
+   */
+  public TableOperationsImpl(Instance instance, Credentials credentials) {
+    ArgumentChecker.notNull(instance, credentials);
+    this.instance = instance;
+    this.credentials = credentials;
+  }
+
+  /**
+   * Retrieve a list of tables in Accumulo.
+   *
+   * @return List of tables in accumulo
+   */
+  @Override
+  public SortedSet<String> list() {
+    OpTimer opTimer = new OpTimer(log, Level.TRACE).start("Fetching list of tables...");
+    TreeSet<String> tableNames = new TreeSet<String>(Tables.getNameToIdMap(instance).keySet());
+    opTimer.stop("Fetched " + tableNames.size() + " table names in %DURATION%");
+    return tableNames;
+  }
+
+  /**
+   * A method to check if a table exists in Accumulo.
+   *
+   * @param tableName
+   *          the name of the table
+   * @return true if the table exists
+   */
+  @Override
+  public boolean exists(String tableName) {
+    ArgumentChecker.notNull(tableName);
+    if (tableName.equals(MetadataTable.NAME) || tableName.equals(RootTable.NAME))
+      return true;
+
+    OpTimer opTimer = new OpTimer(log, Level.TRACE).start("Checking if table " + tableName + " exists...");
+    boolean exists = Tables.getNameToIdMap(instance).containsKey(tableName);
+    opTimer.stop("Checked existance of " + exists + " in %DURATION%");
+    return exists;
+  }
+
+  /**
+   * Create a table with no special configuration
+   *
+   * @param tableName
+   *          the name of the table
+   * @throws AccumuloException
+   *           if a general error occurs
+   * @throws AccumuloSecurityException
+   *           if the user does not have permission
+   * @throws TableExistsException
+   *           if the table already exists
+   */
+  @Override
+  public void create(String tableName) throws AccumuloException, AccumuloSecurityException, TableExistsException {
+    create(tableName, true, TimeType.MILLIS);
+  }
+
+  /**
+   * @param tableName
+   *          the name of the table
+   * @param limitVersion
+   *          Enables/disables the versioning iterator, which will limit the number of Key versions kept.
+   */
+  @Override
+  public void create(String tableName, boolean limitVersion) throws AccumuloException, AccumuloSecurityException, TableExistsException {
+    create(tableName, limitVersion, TimeType.MILLIS);
+  }
+
+  /**
+   * @param tableName
+   *          the name of the table
+   * @param timeType
+   *          specifies logical or real-time based time recording for entries in the table
+   * @param limitVersion
+   *          Enables/disables the versioning iterator, which will limit the number of Key versions kept.
+   */
+  @Override
+  public void create(String tableName, boolean limitVersion, TimeType timeType) throws AccumuloException, AccumuloSecurityException, TableExistsException {
+    ArgumentChecker.notNull(tableName, timeType);
+
+    List<ByteBuffer> args = Arrays.asList(ByteBuffer.wrap(tableName.getBytes(Constants.UTF8)), ByteBuffer.wrap(timeType.name().getBytes(Constants.UTF8)));
+
+    Map<String,String> opts;
+    if (limitVersion)
+      opts = IteratorUtil.generateInitialTableProperties(limitVersion);
+    else
+      opts = Collections.emptyMap();
+
+    try {
+      doTableFateOperation(tableName, AccumuloException.class, FateOperation.TABLE_CREATE, args, opts);
+    } catch (TableNotFoundException e) {
+      // should not happen
+      throw new AssertionError(e);
+    }
+  }
+
+  private long beginFateOperation() throws ThriftSecurityException, TException {
+    while (true) {
+      MasterClientService.Iface client = null;
+      try {
+        client = MasterClient.getConnectionWithRetry(instance);
+        return client.beginFateOperation(Tracer.traceInfo(), credentials.toThrift(instance));
+      } catch (TTransportException tte) {
+        log.debug("Failed to call beginFateOperation(), retrying ... ", tte);
+        UtilWaitThread.sleep(100);
+      } finally {
+        MasterClient.close(client);
+      }
+    }
+  }
+
+  // This method is for retrying in the case of network failures; anything else it passes to the caller to deal with
+  private void executeFateOperation(long opid, FateOperation op, List<ByteBuffer> args, Map<String,String> opts, boolean autoCleanUp)
+      throws ThriftSecurityException, TException, ThriftTableOperationException {
+    while (true) {
+      MasterClientService.Iface client = null;
+      try {
+        client = MasterClient.getConnectionWithRetry(instance);
+        client.executeFateOperation(Tracer.traceInfo(), credentials.toThrift(instance), opid, op, args, opts, autoCleanUp);
+        break;
+      } catch (TTransportException tte) {
+        log.debug("Failed to call executeFateOperation(), retrying ... ", tte);
+        UtilWaitThread.sleep(100);
+      } finally {
+        MasterClient.close(client);
+      }
+    }
+  }
+
+  private String waitForFateOperation(long opid) throws ThriftSecurityException, TException, ThriftTableOperationException {
+    while (true) {
+      MasterClientService.Iface client = null;
+      try {
+        client = MasterClient.getConnectionWithRetry(instance);
+        return client.waitForFateOperation(Tracer.traceInfo(), credentials.toThrift(instance), opid);
+      } catch (TTransportException tte) {
+        log.debug("Failed to call waitForFateOperation(), retrying ... ", tte);
+        UtilWaitThread.sleep(100);
+      } finally {
+        MasterClient.close(client);
+      }
+    }
+  }
+
+  private void finishFateOperation(long opid) throws ThriftSecurityException, TException {
+    while (true) {
+      MasterClientService.Iface client = null;
+      try {
+        client = MasterClient.getConnectionWithRetry(instance);
+        client.finishFateOperation(Tracer.traceInfo(), credentials.toThrift(instance), opid);
+        break;
+      } catch (TTransportException tte) {
+        log.debug("Failed to call finishFateOperation(), retrying ... ", tte);
+        UtilWaitThread.sleep(100);
+      } finally {
+        MasterClient.close(client);
+      }
+    }
+  }
+
+  String doFateOperation(FateOperation op, List<ByteBuffer> args, Map<String,String> opts) throws AccumuloSecurityException, TableExistsException,
+      TableNotFoundException, AccumuloException, NamespaceExistsException, NamespaceNotFoundException {
+    return doFateOperation(op, args, opts, true);
+  }
+
+  String doFateOperation(FateOperation op, List<ByteBuffer> args, Map<String,String> opts, boolean wait) throws AccumuloSecurityException,
+      TableExistsException, TableNotFoundException, AccumuloException, NamespaceExistsException, NamespaceNotFoundException {
+    Long opid = null;
+
+    try {
+      opid = beginFateOperation();
+      executeFateOperation(opid, op, args, opts, !wait);
+      if (!wait) {
+        opid = null;
+        return null;
+      }
+      String ret = waitForFateOperation(opid);
+      return ret;
+    } catch (ThriftSecurityException e) {
+      String tableName = ByteBufferUtil.toString(args.get(0));
+      switch (e.getCode()) {
+        case TABLE_DOESNT_EXIST:
+          throw new TableNotFoundException(null, tableName, "Target table does not exist");
+        case NAMESPACE_DOESNT_EXIST:
+          throw new NamespaceNotFoundException(null, tableName, "Target namespace does not exist");
+        default:
+          String tableInfo = Tables.getPrintableTableInfoFromName(instance, tableName);
+          throw new AccumuloSecurityException(e.user, e.code, tableInfo, e);
+      }
+    } catch (ThriftTableOperationException e) {
+      switch (e.getType()) {
+        case EXISTS:
+          throw new TableExistsException(e);
+        case NOTFOUND:
+          throw new TableNotFoundException(e);
+        case NAMESPACE_EXISTS:
+          throw new NamespaceExistsException(e);
+        case NAMESPACE_NOTFOUND:
+          throw new NamespaceNotFoundException(e);
+        case OFFLINE:
+          throw new TableOfflineException(instance, null);
+        default:
+          throw new AccumuloException(e.description, e);
+      }
+    } catch (Exception e) {
+      throw new AccumuloException(e.getMessage(), e);
+    } finally {
+      Tables.clearCache(instance);
+      // always finish table op, even when exception
+      if (opid != null)
+        try {
+          finishFateOperation(opid);
+        } catch (Exception e) {
+          log.warn(e.getMessage(), e);
+        }
+    }
+  }
+
+  private static class SplitEnv {
+    private String tableName;
+    private String tableId;
+    private ExecutorService executor;
+    private CountDownLatch latch;
+    private AtomicReference<Exception> exception;
+
+    SplitEnv(String tableName, String tableId, ExecutorService executor, CountDownLatch latch, AtomicReference<Exception> exception) {
+      this.tableName = tableName;
+      this.tableId = tableId;
+      this.executor = executor;
+      this.latch = latch;
+      this.exception = exception;
+    }
+  }
+
+  private class SplitTask implements Runnable {
+
+    private List<Text> splits;
+    private SplitEnv env;
+
+    SplitTask(SplitEnv env, List<Text> splits) {
+      this.env = env;
+      this.splits = splits;
+    }
+
+    @Override
+    public void run() {
+      try {
+        if (env.exception.get() != null)
+          return;
+
+        if (splits.size() <= 2) {
+          addSplits(env.tableName, new TreeSet<Text>(splits), env.tableId);
+          for (int i = 0; i < splits.size(); i++)
+            env.latch.countDown();
+          return;
+        }
+
+        int mid = splits.size() / 2;
+
+        // split the middle split point to ensure that child task split different tablets and can therefore
+        // run in parallel
+        addSplits(env.tableName, new TreeSet<Text>(splits.subList(mid, mid + 1)), env.tableId);
+        env.latch.countDown();
+
+        env.executor.submit(new SplitTask(env, splits.subList(0, mid)));
+        env.executor.submit(new SplitTask(env, splits.subList(mid + 1, splits.size())));
+
+      } catch (Exception e) {
+        env.exception.compareAndSet(null, e);
+      }
+    }
+
+  }
+
+  /**
+   * @param tableName
+   *          the name of the table
+   * @param partitionKeys
+   *          a sorted set of row key values to pre-split the table on
+   * @throws AccumuloException
+   *           if a general error occurs
+   * @throws AccumuloSecurityException
+   *           if the user does not have permission
+   * @throws TableNotFoundException
+   *           if the table does not exist
+   */
+  @Override
+  public void addSplits(String tableName, SortedSet<Text> partitionKeys) throws TableNotFoundException, AccumuloException, AccumuloSecurityException {
+    String tableId = Tables.getTableId(instance, tableName);
+
+    List<Text> splits = new ArrayList<Text>(partitionKeys);
+    // should be sorted because we copied from a sorted set, but that makes assumptions about
+    // how the copy was done so resort to be sure.
+    Collections.sort(splits);
+
+    CountDownLatch latch = new CountDownLatch(splits.size());
+    AtomicReference<Exception> exception = new AtomicReference<Exception>(null);
+
+    ExecutorService executor = Executors.newFixedThreadPool(16, new NamingThreadFactory("addSplits"));
+    try {
+      executor.submit(new SplitTask(new SplitEnv(tableName, tableId, executor, latch, exception), splits));
+
+      while (!latch.await(100, TimeUnit.MILLISECONDS)) {
+        if (exception.get() != null) {
+          executor.shutdownNow();
+          Exception excep = exception.get();
+          if (excep instanceof TableNotFoundException)
+            throw (TableNotFoundException) excep;
+          else if (excep instanceof AccumuloException)
+            throw (AccumuloException) excep;
+          else if (excep instanceof AccumuloSecurityException)
+            throw (AccumuloSecurityException) excep;
+          else if (excep instanceof RuntimeException)
+            throw (RuntimeException) excep;
+          else
+            throw new RuntimeException(excep);
+        }
+      }
+    } catch (InterruptedException e) {
+      throw new RuntimeException(e);
+    } finally {
+      executor.shutdown();
+    }
+  }
+
+  private void addSplits(String tableName, SortedSet<Text> partitionKeys, String tableId) throws AccumuloException, AccumuloSecurityException,
+      TableNotFoundException, AccumuloServerException {
+    TabletLocator tabLocator = TabletLocator.getLocator(instance, new Text(tableId));
+
+    for (Text split : partitionKeys) {
+      boolean successful = false;
+      int attempt = 0;
+
+      while (!successful) {
+
+        if (attempt > 0)
+          UtilWaitThread.sleep(100);
+
+        attempt++;
+
+        TabletLocation tl = tabLocator.locateTablet(credentials, split, false, false);
+
+        if (tl == null) {
+          if (!Tables.exists(instance, tableId))
+            throw new TableNotFoundException(tableId, tableName, null);
+          else if (Tables.getTableState(instance, tableId) == TableState.OFFLINE)
+            throw new TableOfflineException(instance, tableId);
+          continue;
+        }
+
+        try {
+          TabletClientService.Client client = ThriftUtil.getTServerClient(tl.tablet_location, ServerConfigurationUtil.getConfiguration(instance));
+          try {
+            OpTimer opTimer = null;
+            if (log.isTraceEnabled())
+              opTimer = new OpTimer(log, Level.TRACE).start("Splitting tablet " + tl.tablet_extent + " on " + tl.tablet_location + " at " + split);
+
+            client.splitTablet(Tracer.traceInfo(), credentials.toThrift(instance), tl.tablet_extent.toThrift(), TextUtil.getByteBuffer(split));
+
+            // just split it, might as well invalidate it in the cache
+            tabLocator.invalidateCache(tl.tablet_extent);
+
+            if (opTimer != null)
+              opTimer.stop("Split tablet in %DURATION%");
+          } finally {
+            ThriftUtil.returnClient(client);
+          }
+
+        } catch (TApplicationException tae) {
+          throw new AccumuloServerException(tl.tablet_location, tae);
+        } catch (TTransportException e) {
+          tabLocator.invalidateCache(tl.tablet_location);
+          continue;
+        } catch (ThriftSecurityException e) {
+          Tables.clearCache(instance);
+          if (!Tables.exists(instance, tableId))
+            throw new TableNotFoundException(tableId, tableName, null);
+          throw new AccumuloSecurityException(e.user, e.code, e);
+        } catch (NotServingTabletException e) {
+          tabLocator.invalidateCache(tl.tablet_extent);
+          continue;
+        } catch (TException e) {
+          tabLocator.invalidateCache(tl.tablet_location);
+          continue;
+        }
+
+        successful = true;
+      }
+    }
+  }
+
+  @Override
+  public void merge(String tableName, Text start, Text end) throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
+
+    ArgumentChecker.notNull(tableName);
+    ByteBuffer EMPTY = ByteBuffer.allocate(0);
+    List<ByteBuffer> args = Arrays.asList(ByteBuffer.wrap(tableName.getBytes(Constants.UTF8)), start == null ? EMPTY : TextUtil.getByteBuffer(start),
+        end == null ? EMPTY : TextUtil.getByteBuffer(end));
+    Map<String,String> opts = new HashMap<String,String>();
+    try {
+      doTableFateOperation(tableName, TableNotFoundException.class, FateOperation.TABLE_MERGE, args, opts);
+    } catch (TableExistsException e) {
+      // should not happen
+      throw new AssertionError(e);
+    }
+  }
+
+  @Override
+  public void deleteRows(String tableName, Text start, Text end) throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
+
+    ArgumentChecker.notNull(tableName);
+    ByteBuffer EMPTY = ByteBuffer.allocate(0);
+    List<ByteBuffer> args = Arrays.asList(ByteBuffer.wrap(tableName.getBytes(Constants.UTF8)), start == null ? EMPTY : TextUtil.getByteBuffer(start),
+        end == null ? EMPTY : TextUtil.getByteBuffer(end));
+    Map<String,String> opts = new HashMap<String,String>();
+    try {
+      doTableFateOperation(tableName, TableNotFoundException.class, FateOperation.TABLE_DELETE_RANGE, args, opts);
+    } catch (TableExistsException e) {
+      // should not happen
+      throw new AssertionError(e);
+    }
+  }
+
+  /**
+   * @param tableName
+   *          the name of the table
+   * @return the split points (end-row names) for the table's current split profile
+   */
+  @Override
+  public Collection<Text> listSplits(String tableName) throws TableNotFoundException, AccumuloSecurityException {
+
+    ArgumentChecker.notNull(tableName);
+
+    String tableId = Tables.getTableId(instance, tableName);
+
+    TreeMap<KeyExtent,String> tabletLocations = new TreeMap<KeyExtent,String>();
+
+    while (true) {
+      try {
+        tabletLocations.clear();
+        // the following method throws AccumuloException for some conditions that should be retried
+        MetadataServicer.forTableId(instance, credentials, tableId).getTabletLocations(tabletLocations);
+        break;
+      } catch (AccumuloSecurityException ase) {
+        throw ase;
+      } catch (Exception e) {
+        if (!Tables.exists(instance, tableId)) {
+          throw new TableNotFoundException(tableId, tableName, null);
+        }
+
+        if (e instanceof RuntimeException && e.getCause() instanceof AccumuloSecurityException) {
+          throw (AccumuloSecurityException) e.getCause();
+        }
+
+        log.info(e.getMessage() + " ... retrying ...");
+        UtilWaitThread.sleep(3000);
+      }
+    }
+
+    ArrayList<Text> endRows = new ArrayList<Text>(tabletLocations.size());
+
+    for (KeyExtent ke : tabletLocations.keySet())
+      if (ke.getEndRow() != null)
+        endRows.add(ke.getEndRow());
+
+    return endRows;
+  }
+
+  @Deprecated
+  @Override
+  public Collection<Text> getSplits(String tableName) throws TableNotFoundException {
+    try {
+      return listSplits(tableName);
+    } catch (AccumuloSecurityException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  /**
+   * @param tableName
+   *          the name of the table
+   * @param maxSplits
+   *          specifies the maximum number of splits to return
+   * @return the split points (end-row names) for the table's current split profile, grouped into fewer splits so as not to exceed maxSplits
+   */
+  @Override
+  public Collection<Text> listSplits(String tableName, int maxSplits) throws TableNotFoundException, AccumuloSecurityException {
+    Collection<Text> endRows = listSplits(tableName);
+
+    if (endRows.size() <= maxSplits)
+      return endRows;
+
+    double r = (maxSplits + 1) / (double) (endRows.size());
+    double pos = 0;
+
+    ArrayList<Text> subset = new ArrayList<Text>(maxSplits);
+
+    int j = 0;
+    for (int i = 0; i < endRows.size() && j < maxSplits; i++) {
+      pos += r;
+      while (pos > 1) {
+        subset.add(((ArrayList<Text>) endRows).get(i));
+        j++;
+        pos -= 1;
+      }
+    }
+
+    return subset;
+  }
+
+  @Deprecated
+  @Override
+  public Collection<Text> getSplits(String tableName, int maxSplits) throws TableNotFoundException {
+    try {
+      return listSplits(tableName, maxSplits);
+    } catch (AccumuloSecurityException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  /**
+   * Delete a table
+   *
+   * @param tableName
+   *          the name of the table
+   * @throws AccumuloException
+   *           if a general error occurs
+   * @throws AccumuloSecurityException
+   *           if the user does not have permission
+   * @throws TableNotFoundException
+   *           if the table does not exist
+   */
+  @Override
+  public void delete(String tableName) throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
+    ArgumentChecker.notNull(tableName);
+
+    List<ByteBuffer> args = Arrays.asList(ByteBuffer.wrap(tableName.getBytes(Constants.UTF8)));
+    Map<String,String> opts = new HashMap<String,String>();
+
+    try {
+      doTableFateOperation(tableName, TableNotFoundException.class, FateOperation.TABLE_DELETE, args, opts);
+    } catch (TableExistsException e) {
+      // should not happen
+      throw new AssertionError(e);
+    }
+
+  }
+
+  @Override
+  public void clone(String srcTableName, String newTableName, boolean flush, Map<String,String> propertiesToSet, Set<String> propertiesToExclude)
+      throws AccumuloSecurityException, TableNotFoundException, AccumuloException, TableExistsException {
+
+    ArgumentChecker.notNull(srcTableName, newTableName);
+
+    String srcTableId = Tables.getTableId(instance, srcTableName);
+
+    if (flush)
+      _flush(srcTableId, null, null, true);
+
+    if (propertiesToExclude == null)
+      propertiesToExclude = Collections.emptySet();
+
+    if (propertiesToSet == null)
+      propertiesToSet = Collections.emptyMap();
+
+    List<ByteBuffer> args = Arrays.asList(ByteBuffer.wrap(srcTableId.getBytes(Constants.UTF8)), ByteBuffer.wrap(newTableName.getBytes(Constants.UTF8)));
+    Map<String,String> opts = new HashMap<String,String>();
+    for (Entry<String,String> entry : propertiesToSet.entrySet()) {
+      if (entry.getKey().startsWith(CLONE_EXCLUDE_PREFIX))
+        throw new IllegalArgumentException("Property can not start with " + CLONE_EXCLUDE_PREFIX);
+      opts.put(entry.getKey(), entry.getValue());
+    }
+
+    for (String prop : propertiesToExclude) {
+      opts.put(CLONE_EXCLUDE_PREFIX + prop, "");
+    }
+
+    doTableFateOperation(newTableName, AccumuloException.class, FateOperation.TABLE_CLONE, args, opts);
+  }
+
+  /**
+   * Rename a table
+   *
+   * @param oldTableName
+   *          the old table name
+   * @param newTableName
+   *          the new table name
+   * @throws AccumuloException
+   *           if a general error occurs
+   * @throws AccumuloSecurityException
+   *           if the user does not have permission
+   * @throws TableNotFoundException
+   *           if the old table name does not exist
+   * @throws TableExistsException
+   *           if the new table name already exists
+   */
+  @Override
+  public void rename(String oldTableName, String newTableName) throws AccumuloSecurityException, TableNotFoundException, AccumuloException,
+      TableExistsException {
+
+    List<ByteBuffer> args = Arrays.asList(ByteBuffer.wrap(oldTableName.getBytes(Constants.UTF8)), ByteBuffer.wrap(newTableName.getBytes(Constants.UTF8)));
+    Map<String,String> opts = new HashMap<String,String>();
+    doTableFateOperation(oldTableName, TableNotFoundException.class, FateOperation.TABLE_RENAME, args, opts);
+  }
+
+  /**
+   * @deprecated since 1.4 {@link #flush(String, Text, Text, boolean)}
+   */
+  @Override
+  @Deprecated
+  public void flush(String tableName) throws AccumuloException, AccumuloSecurityException {
+    try {
+      flush(tableName, null, null, false);
+    } catch (TableNotFoundException e) {
+      throw new AccumuloException(e.getMessage(), e);
+    }
+  }
+
+  /**
+   * Flush a table
+   *
+   * @param tableName
+   *          the name of the table
+   * @throws AccumuloException
+   *           if a general error occurs
+   * @throws AccumuloSecurityException
+   *           if the user does not have permission
+   */
+  @Override
+  public void flush(String tableName, Text start, Text end, boolean wait) throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
+    ArgumentChecker.notNull(tableName);
+
+    String tableId = Tables.getTableId(instance, tableName);
+    _flush(tableId, start, end, wait);
+  }
+
+  @Override
+  public void compact(String tableName, Text start, Text end, boolean flush, boolean wait) throws AccumuloSecurityException, TableNotFoundException,
+      AccumuloException {
+    compact(tableName, start, end, new ArrayList<IteratorSetting>(), flush, wait);
+  }
+
+  @Override
+  public void compact(String tableName, Text start, Text end, List<IteratorSetting> iterators, boolean flush, boolean wait) throws AccumuloSecurityException,
+      TableNotFoundException, AccumuloException {
+    ArgumentChecker.notNull(tableName);
+    ByteBuffer EMPTY = ByteBuffer.allocate(0);
+
+    String tableId = Tables.getTableId(instance, tableName);
+
+    if (flush)
+      _flush(tableId, start, end, true);
+
+    List<ByteBuffer> args = Arrays.asList(ByteBuffer.wrap(tableId.getBytes(Constants.UTF8)), start == null ? EMPTY : TextUtil.getByteBuffer(start),
+        end == null ? EMPTY : TextUtil.getByteBuffer(end), ByteBuffer.wrap(IteratorUtil.encodeIteratorSettings(iterators)));
+
+    Map<String,String> opts = new HashMap<String,String>();
+    try {
+      doFateOperation(FateOperation.TABLE_COMPACT, args, opts, wait);
+    } catch (TableExistsException e) {
+      // should not happen
+      throw new AssertionError(e);
+    } catch (NamespaceExistsException e) {
+      // should not happen
+      throw new AssertionError(e);
+    } catch (NamespaceNotFoundException e) {
+      throw new TableNotFoundException(null, tableName, "Namespace not found", e);
+    }
+  }
+
+  @Override
+  public void cancelCompaction(String tableName) throws AccumuloSecurityException, TableNotFoundException, AccumuloException {
+    String tableId = Tables.getTableId(instance, tableName);
+
+    List<ByteBuffer> args = Arrays.asList(ByteBuffer.wrap(tableId.getBytes(Constants.UTF8)));
+
+    Map<String,String> opts = new HashMap<String,String>();
+    try {
+      doTableFateOperation(tableName, TableNotFoundException.class, FateOperation.TABLE_CANCEL_COMPACT, args, opts);
+    } catch (TableExistsException e) {
+      // should not happen
+      throw new AssertionError(e);
+    }
+
+  }
+
+  private void _flush(String tableId, Text start, Text end, boolean wait) throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
+
+    try {
+      long flushID;
+
+      // used to pass the table name. but the tableid associated with a table name could change between calls.
+      // so pass the tableid to both calls
+
+      while (true) {
+        MasterClientService.Iface client = null;
+        try {
+          client = MasterClient.getConnectionWithRetry(instance);
+          flushID = client.initiateFlush(Tracer.traceInfo(), credentials.toThrift(instance), tableId);
+          break;
+        } catch (TTransportException tte) {
+          log.debug("Failed to call initiateFlush, retrying ... ", tte);
+          UtilWaitThread.sleep(100);
+        } finally {
+          MasterClient.close(client);
+        }
+      }
+
+      while (true) {
+        MasterClientService.Iface client = null;
+        try {
+          client = MasterClient.getConnectionWithRetry(instance);
+          client.waitForFlush(Tracer.traceInfo(), credentials.toThrift(instance), tableId, TextUtil.getByteBuffer(start), TextUtil.getByteBuffer(end), flushID,
+              wait ? Long.MAX_VALUE : 1);
+          break;
+        } catch (TTransportException tte) {
+          log.debug("Failed to call initiateFlush, retrying ... ", tte);
+          UtilWaitThread.sleep(100);
+        } finally {
+          MasterClient.close(client);
+        }
+      }
+    } catch (ThriftSecurityException e) {
+      switch (e.getCode()) {
+        case TABLE_DOESNT_EXIST:
+          throw new TableNotFoundException(tableId, null, e.getMessage(), e);
+        default:
+          log.debug("flush security exception on table id " + tableId);
+          throw new AccumuloSecurityException(e.user, e.code, e);
+      }
+    } catch (ThriftTableOperationException e) {
+      switch (e.getType()) {
+        case NOTFOUND:
+          throw new TableNotFoundException(e);
+        default:
+          throw new AccumuloException(e.description, e);
+      }
+    } catch (Exception e) {
+      throw new AccumuloException(e);
+    }
+  }
+
+  /**
+   * Sets a property on a table
+   *
+   * @param tableName
+   *          the name of the table
+   * @param property
+   *          the name of a per-table property
+   * @param value
+   *          the value to set a per-table property to
+   * @throws AccumuloException
+   *           if a general error occurs
+   * @throws AccumuloSecurityException
+   *           if the user does not have permission
+   */
+  @Override
+  public void setProperty(final String tableName, final String property, final String value) throws AccumuloException, AccumuloSecurityException {
+    ArgumentChecker.notNull(tableName, property, value);
+    try {
+      MasterClient.executeTable(instance, new ClientExec<MasterClientService.Client>() {
+        @Override
+        public void execute(MasterClientService.Client client) throws Exception {
+          client.setTableProperty(Tracer.traceInfo(), credentials.toThrift(instance), tableName, property, value);
+        }
+      });
+    } catch (TableNotFoundException e) {
+      throw new AccumuloException(e);
+    }
+  }
+
+  /**
+   * Removes a property from a table
+   *
+   * @param tableName
+   *          the name of the table
+   * @param property
+   *          the name of a per-table property
+   * @throws AccumuloException
+   *           if a general error occurs
+   * @throws AccumuloSecurityException
+   *           if the user does not have permission
+   */
+  @Override
+  public void removeProperty(final String tableName, final String property) throws AccumuloException, AccumuloSecurityException {
+    ArgumentChecker.notNull(tableName, property);
+    try {
+      MasterClient.executeTable(instance, new ClientExec<MasterClientService.Client>() {
+        @Override
+        public void execute(MasterClientService.Client client) throws Exception {
+          client.removeTableProperty(Tracer.traceInfo(), credentials.toThrift(instance), tableName, property);
+        }
+      });
+    } catch (TableNotFoundException e) {
+      throw new AccumuloException(e);
+    }
+  }
+
+  /**
+   * Gets properties of a table
+   *
+   * @param tableName
+   *          the name of the table
+   * @return all properties visible by this table (system and per-table properties)
+   * @throws TableNotFoundException
+   *           if the table does not exist
+   */
+  @Override
+  public Iterable<Entry<String,String>> getProperties(final String tableName) throws AccumuloException, TableNotFoundException {
+    ArgumentChecker.notNull(tableName);
+    try {
+      return ServerClient.executeRaw(instance, new ClientExecReturn<Map<String,String>,ClientService.Client>() {
+        @Override
+        public Map<String,String> execute(ClientService.Client client) throws Exception {
+          return client.getTableConfiguration(Tracer.traceInfo(), credentials.toThrift(instance), tableName);
+        }
+      }).entrySet();
+    } catch (ThriftTableOperationException e) {
+      switch (e.getType()) {
+        case NOTFOUND:
+          throw new TableNotFoundException(e);
+        case NAMESPACE_NOTFOUND:
+          throw new TableNotFoundException(tableName, new NamespaceNotFoundException(e));
+        default:
+          throw new AccumuloException(e.description, e);
+      }
+    } catch (AccumuloException e) {
+      throw e;
+    } catch (Exception e) {
+      throw new AccumuloException(e);
+    }
+
+  }
+
+  /**
+   * Sets a tables locality groups. A tables locality groups can be changed at any time.
+   *
+   * @param tableName
+   *          the name of the table
+   * @param groups
+   *          mapping of locality group names to column families in the locality group
+   * @throws AccumuloException
+   *           if a general error occurs
+   * @throws AccumuloSecurityException
+   *           if the user does not have permission
+   * @throws TableNotFoundException
+   *           if the table does not exist
+   */
+  @Override
+  public void setLocalityGroups(String tableName, Map<String,Set<Text>> groups) throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
+    // ensure locality groups do not overlap
+    HashSet<Text> all = new HashSet<Text>();
+    for (Entry<String,Set<Text>> entry : groups.entrySet()) {
+
+      if (!Collections.disjoint(all, entry.getValue())) {
+        throw new IllegalArgumentException("Group " + entry.getKey() + " overlaps with another group");
+      }
+
+      all.addAll(entry.getValue());
+    }
+
+    for (Entry<String,Set<Text>> entry : groups.entrySet()) {
+      Set<Text> colFams = entry.getValue();
+      String value = LocalityGroupUtil.encodeColumnFamilies(colFams);
+      setProperty(tableName, Property.TABLE_LOCALITY_GROUP_PREFIX + entry.getKey(), value);
+    }
+
+    try {
+      setProperty(tableName, Property.TABLE_LOCALITY_GROUPS.getKey(), StringUtil.join(groups.keySet(), ","));
+    } catch (AccumuloException e) {
+      if (e.getCause() instanceof TableNotFoundException)
+        throw (TableNotFoundException) e.getCause();
+      throw e;
+    }
+
+    // remove anything extraneous
+    String prefix = Property.TABLE_LOCALITY_GROUP_PREFIX.getKey();
+    for (Entry<String,String> entry : getProperties(tableName)) {
+      String property = entry.getKey();
+      if (property.startsWith(prefix)) {
+        // this property configures a locality group, find out which
+        // one:
+        String[] parts = property.split("\\.");
+        String group = parts[parts.length - 1];
+
+        if (!groups.containsKey(group)) {
+          removeProperty(tableName, property);
+        }
+      }
+    }
+  }
+
+  /**
+   *
+   * Gets the locality groups currently set for a table.
+   *
+   * @param tableName
+   *          the name of the table
+   * @return mapping of locality group names to column families in the locality group
+   * @throws AccumuloException
+   *           if a general error occurs
+   * @throws TableNotFoundException
+   *           if the table does not exist
+   */
+  @Override
+  public Map<String,Set<Text>> getLocalityGroups(String tableName) throws AccumuloException, TableNotFoundException {
+    AccumuloConfiguration conf = new ConfigurationCopy(this.getProperties(tableName));
+    Map<String,Set<ByteSequence>> groups = LocalityGroupUtil.getLocalityGroups(conf);
+
+    Map<String,Set<Text>> groups2 = new HashMap<String,Set<Text>>();
+    for (Entry<String,Set<ByteSequence>> entry : groups.entrySet()) {
+
+      HashSet<Text> colFams = new HashSet<Text>();
+
+      for (ByteSequence bs : entry.getValue()) {
+        colFams.add(new Text(bs.toArray()));
+      }
+
+      groups2.put(entry.getKey(), colFams);
+    }
+
+    return groups2;
+  }
+
+  /**
+   * @param tableName
+   *          the name of the table
+   * @param range
+   *          a range to split
+   * @param maxSplits
+   *          the maximum number of splits
+   * @return the range, split into smaller ranges that fall on boundaries of the table's split points as evenly as possible
+   * @throws AccumuloException
+   *           if a general error occurs
+   * @throws AccumuloSecurityException
+   *           if the user does not have permission
+   * @throws TableNotFoundException
+   *           if the table does not exist
+   */
+  @Override
+  public Set<Range> splitRangeByTablets(String tableName, Range range, int maxSplits) throws AccumuloException, AccumuloSecurityException,
+      TableNotFoundException {
+    ArgumentChecker.notNull(tableName, range);
+    if (maxSplits < 1)
+      throw new IllegalArgumentException("maximum splits must be >= 1");
+    if (maxSplits == 1)
+      return Collections.singleton(range);
+
+    Random random = new Random();
+    Map<String,Map<KeyExtent,List<Range>>> binnedRanges = new HashMap<String,Map<KeyExtent,List<Range>>>();
+    String tableId = Tables.getTableId(instance, tableName);
+    TabletLocator tl = TabletLocator.getLocator(instance, new Text(tableId));
+    // its possible that the cache could contain complete, but old information about a tables tablets... so clear it
+    tl.invalidateCache();
+    while (!tl.binRanges(credentials, Collections.singletonList(range), binnedRanges).isEmpty()) {
+      if (!Tables.exists(instance, tableId))
+        throw new TableDeletedException(tableId);
+      if (Tables.getTableState(instance, tableId) == TableState.OFFLINE)
+        throw new TableOfflineException(instance, tableId);
+
+      log.warn("Unable to locate bins for specified range. Retrying.");
+      // sleep randomly between 100 and 200ms
+      UtilWaitThread.sleep(100 + random.nextInt(100));
+      binnedRanges.clear();
+      tl.invalidateCache();
+    }
+
+    // group key extents to get <= maxSplits
+    LinkedList<KeyExtent> unmergedExtents = new LinkedList<KeyExtent>();
+    List<KeyExtent> mergedExtents = new ArrayList<KeyExtent>();
+
+    for (Map<KeyExtent,List<Range>> map : binnedRanges.values())
+      unmergedExtents.addAll(map.keySet());
+
+    // the sort method is efficient for linked list
+    Collections.sort(unmergedExtents);
+
+    while (unmergedExtents.size() + mergedExtents.size() > maxSplits) {
+      if (unmergedExtents.size() >= 2) {
+        KeyExtent first = unmergedExtents.removeFirst();
+        KeyExtent second = unmergedExtents.removeFirst();
+        first.setEndRow(second.getEndRow());
+        mergedExtents.add(first);
+      } else {
+        mergedExtents.addAll(unmergedExtents);
+        unmergedExtents.clear();
+        unmergedExtents.addAll(mergedExtents);
+        mergedExtents.clear();
+      }
+
+    }
+
+    mergedExtents.addAll(unmergedExtents);
+
+    Set<Range> ranges = new HashSet<Range>();
+    for (KeyExtent k : mergedExtents)
+      ranges.add(k.toDataRange().clip(range));
+
+    return ranges;
+  }
+
+  // TODO Remove deprecation warning surppression when Hadoop1 support is dropped
+  @SuppressWarnings("deprecation")
+  private Path checkPath(String dir, String kind, String type) throws IOException, AccumuloException {
+    Path ret;
+    FileSystem fs = VolumeConfiguration.getVolume(dir, CachedConfiguration.getInstance(), ServerConfigurationUtil.getConfiguration(instance)).getFileSystem();
+
+    if (dir.contains(":")) {
+      ret = new Path(dir);
+    } else {
+      ret = fs.makeQualified(new Path(dir));
+    }
+
+    if (!fs.exists(ret))
+      throw new AccumuloException(kind + " import " + type + " directory " + dir + " does not exist!");
+
+    if (!fs.getFileStatus(ret).isDir()) {
+      throw new AccumuloException(kind + " import " + type + " directory " + dir + " is not a directory!");
+    }
+
+    if (type.equals("failure")) {
+      FileStatus[] listStatus = fs.listStatus(ret);
+      if (listStatus != null && listStatus.length != 0) {
+        throw new AccumuloException("Bulk import failure directory " + ret + " is not empty");
+      }
+    }
+
+    return ret;
+  }
+
+  @Override
+  public void importDirectory(String tableName, String dir, String failureDir, boolean setTime) throws IOException, AccumuloSecurityException,
+      TableNotFoundException, AccumuloException {
+    ArgumentChecker.notNull(tableName, dir, failureDir);
+    // check for table existance
+    Tables.getTableId(instance, tableName);
+
+    Path dirPath = checkPath(dir, "Bulk", "");
+    Path failPath = checkPath(failureDir, "Bulk", "failure");
+
+    List<ByteBuffer> args = Arrays.asList(ByteBuffer.wrap(tableName.getBytes(Constants.UTF8)), ByteBuffer.wrap(dirPath.toString().getBytes(Constants.UTF8)),
+        ByteBuffer.wrap(failPath.toString().getBytes(Constants.UTF8)), ByteBuffer.wrap((setTime + "").getBytes(Constants.UTF8)));
+    Map<String,String> opts = new HashMap<String,String>();
+
+    try {
+      doTableFateOperation(tableName, TableNotFoundException.class, FateOperation.TABLE_BULK_IMPORT, args, opts);
+    } catch (TableExistsException e) {
+      // should not happen
+      throw new AssertionError(e);
+    }
+  }
+
+  private void waitForTableStateTransition(String tableId, TableState expectedState) throws AccumuloException, TableNotFoundException,
+      AccumuloSecurityException {
+
+    Text startRow = null;
+    Text lastRow = null;
+
+    while (true) {
+
+      if (Tables.getTableState(instance, tableId) != expectedState) {
+        Tables.clearCache(instance);
+        if (Tables.getTableState(instance, tableId) != expectedState) {
+          if (!Tables.exists(instance, tableId))
+            throw new TableDeletedException(tableId);
+          throw new AccumuloException("Unexpected table state " + tableId + " " + Tables.getTableState(instance, tableId) + " != " + expectedState);
+        }
+      }
+
+      Range range = new KeyExtent(new Text(tableId), null, null).toMetadataRange();
+      if (startRow == null || lastRow == null)
+        range = new KeyExtent(new Text(tableId), null, null).toMetadataRange();
+      else
+        range = new Range(startRow, lastRow);
+
+      String metaTable = MetadataTable.NAME;
+      if (tableId.equals(MetadataTable.ID))
+        metaTable = RootTable.NAME;
+      Scanner scanner = instance.getConnector(credentials.getPrincipal(), credentials.getToken()).createScanner(metaTable, Authorizations.EMPTY);
+      scanner = new IsolatedScanner(scanner);
+      TabletsSection.TabletColumnFamily.PREV_ROW_COLUMN.fetch(scanner);
+      scanner.fetchColumnFamily(TabletsSection.CurrentLocationColumnFamily.NAME);
+      scanner.setRange(range);
+
+      RowIterator rowIter = new RowIterator(scanner);
+
+      KeyExtent lastExtent = null;
+
+      int total = 0;
+      int waitFor = 0;
+      int holes = 0;
+      Text continueRow = null;
+      MapCounter<String> serverCounts = new MapCounter<String>();
+
+      while (rowIter.hasNext()) {
+        Iterator<Entry<Key,Value>> row = rowIter.next();
+
+        total++;
+
+        KeyExtent extent = null;
+        String future = null;
+        String current = null;
+
+        while (row.hasNext()) {
+          Entry<Key,Value> entry = row.next();
+          Key key = entry.getKey();
+
+          if (key.getColumnFamily().equals(TabletsSection.FutureLocationColumnFamily.NAME))
+            future = entry.getValue().toString();
+
+          if (key.getColumnFamily().equals(TabletsSection.CurrentLocationColumnFamily.NAME))
+            current = entry.getValue().toString();
+
+          if (TabletsSection.TabletColumnFamily.PREV_ROW_COLUMN.hasColumns(key))
+            extent = new KeyExtent(key.getRow(), entry.getValue());
+        }
+
+        if ((expectedState == TableState.ONLINE && current == null) || (expectedState == TableState.OFFLINE && (future != null || current != null))) {
+          if (continueRow == null)
+            continueRow = extent.getMetadataEntry();
+          waitFor++;
+          lastRow = extent.getMetadataEntry();
+
+          if (current != null)
+            serverCounts.increment(current, 1);
+          if (future != null)
+            serverCounts.increment(future, 1);
+        }
+
+        if (!extent.getTableId().toString().equals(tableId)) {
+          throw new AccumuloException("Saw unexpected table Id " + tableId + " " + extent);
+        }
+
+        if (lastExtent != null && !extent.isPreviousExtent(lastExtent)) {
+          holes++;
+        }
+
+        lastExtent = extent;
+      }
+
+      if (continueRow != null) {
+        startRow = continueRow;
+      }
+
+      if (holes > 0 || total == 0) {
+        startRow = null;
+        lastRow = null;
+      }
+
+      if (waitFor > 0 || holes > 0 || total == 0) {
+        long waitTime;
+        long maxPerServer = 0;
+        if (serverCounts.size() > 0) {
+          maxPerServer = Collections.max(serverCounts.values());
+          waitTime = maxPerServer * 10;
+        } else
+          waitTime = waitFor * 10;
+        waitTime = Math.max(100, waitTime);
+        waitTime = Math.min(5000, waitTime);
+        log.trace("Waiting for " + waitFor + "(" + maxPerServer + ") tablets, startRow = " + startRow + " lastRow = " + lastRow + ", holes=" + holes
+            + " sleeping:" + waitTime + "ms");
+        UtilWaitThread.sleep(waitTime);
+      } else {
+        break;
+      }
+
+    }
+  }
+
+  @Override
+  public void offline(String tableName) throws AccumuloSecurityException, AccumuloException, TableNotFoundException {
+    offline(tableName, false);
+  }
+
+  /**
+   *
+   * @param tableName
+   *          the table to take offline
+   * @throws AccumuloException
+   *           when there is a general accumulo error
+   * @throws AccumuloSecurityException
+   *           when the user does not have the proper permissions
+   */
+  @Override
+  public void offline(String tableName, boolean wait) throws AccumuloSecurityException, AccumuloException, TableNotFoundException {
+
+    ArgumentChecker.notNull(tableName);
+    String tableId = Tables.getTableId(instance, tableName);
+    List<ByteBuffer> args = Arrays.asList(ByteBuffer.wrap(tableId.getBytes(Constants.UTF8)));
+    Map<String,String> opts = new HashMap<String,String>();
+
+    try {
+      doTableFateOperation(tableName, TableNotFoundException.class, FateOperation.TABLE_OFFLINE, args, opts);
+    } catch (TableExistsException e) {
+      // should not happen
+      throw new AssertionError(e);
+    }
+
+    if (wait)
+      waitForTableStateTransition(tableId, TableState.OFFLINE);
+  }
+
+  @Override
+  public void online(String tableName) throws AccumuloSecurityException, AccumuloException, TableNotFoundException {
+    online(tableName, false);
+  }
+
+  /**
+   *
+   * @param tableName
+   *          the table to take online
+   * @throws AccumuloException
+   *           when there is a general accumulo error
+   * @throws AccumuloSecurityException
+   *           when the user does not have the proper permissions
+   */
+  @Override
+  public void online(String tableName, boolean wait) throws AccumuloSecurityException, AccumuloException, TableNotFoundException {
+    ArgumentChecker.notNull(tableName);
+    String tableId = Tables.getTableId(instance, tableName);
+    List<ByteBuffer> args = Arrays.asList(ByteBuffer.wrap(tableId.getBytes(Constants.UTF8)));
+    Map<String,String> opts = new HashMap<String,String>();
+
+    try {
+      doTableFateOperation(tableName, TableNotFoundException.class, FateOperation.TABLE_ONLINE, args, opts);
+    } catch (TableExistsException e) {
+      // should not happen
+      throw new AssertionError(e);
+    }
+
+    if (wait)
+      waitForTableStateTransition(tableId, TableState.ONLINE);
+  }
+
+  /**
+   * Clears the tablet locator cache for a specified table
+   *
+   * @param tableName
+   *          the name of the table
+   * @throws TableNotFoundException
+   *           if table does not exist
+   */
+  @Override
+  public void clearLocatorCache(String tableName) throws TableNotFoundException {
+    ArgumentChecker.notNull(tableName);
+    TabletLocator tabLocator = TabletLocator.getLocator(instance, new Text(Tables.getTableId(instance, tableName)));
+    tabLocator.invalidateCache();
+  }
+
+  /**
+   * Get a mapping of table name to internal table id.
+   *
+   * @return the map from table name to internal table id
+   */
+  @Override
+  public Map<String,String> tableIdMap() {
+    return Tables.getNameToIdMap(instance);
+  }
+
+  @Override
+  public Text getMaxRow(String tableName, Authorizations auths, Text startRow, boolean startInclusive, Text endRow, boolean endInclusive)
+      throws TableNotFoundException, AccumuloException, AccumuloSecurityException {
+    ArgumentChecker.notNull(tableName, auths);
+    Scanner scanner = instance.getConnector(credentials.getPrincipal(), credentials.getToken()).createScanner(tableName, auths);
+    return FindMax.findMax(scanner, startRow, startInclusive, endRow, endInclusive);
+  }
+
+  @Override
+  public List<DiskUsage> getDiskUsage(Set<String> tableNames) throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
+
+    List<TDiskUsage> diskUsages = null;
+    while (diskUsages == null) {
+      Pair<String,Client> pair = null;
+      try {
+        // this operation may us a lot of memory... its likely that connections to tabletservers hosting metadata tablets will be cached, so do not use cached
+        // connections
+        pair = ServerClient.getConnection(instance, false);
+        diskUsages = pair.getSecond().getDiskUsage(tableNames, credentials.toThrift(instance));
+      } catch (ThriftTableOperationException e) {
+        switch (e.getType()) {
+          case NOTFOUND:
+            throw new TableNotFoundException(e);
+          case NAMESPACE_NOTFOUND:
+            throw new TableNotFoundException(e.getTableName(), new NamespaceNotFoundException(e));
+          default:
+            throw new AccumuloException(e.description, e);
+        }
+      } catch (ThriftSecurityException e) {
+        throw new AccumuloSecurityException(e.getUser(), e.getCode());
+      } catch (TTransportException e) {
+        // some sort of communication error occurred, retry
+        log.debug("disk usage request failed " + pair.getFirst() + ", retrying ... ", e);
+        UtilWaitThread.sleep(100);
+      } catch (TException e) {
+        // may be a TApplicationException which indicates error on the server side
+        throw new AccumuloException(e);
+      } finally {
+        // must always return thrift connection
+        if (pair != null)
+          ServerClient.close(pair.getSecond());
+      }
+    }
+
+    List<DiskUsage> finalUsages = new ArrayList<DiskUsage>();
+    for (TDiskUsage diskUsage : diskUsages) {
+      finalUsages.add(new DiskUsage(new TreeSet<String>(diskUsage.getTables()), diskUsage.getUsage()));
+    }
+
+    return finalUsages;
+  }
+
+  public static Map<String,String> getExportedProps(FileSystem fs, Path path) throws IOException {
+    HashMap<String,String> props = new HashMap<String,String>();
+
+    ZipInputStream zis = new ZipInputStream(fs.open(path));
+    try {
+      ZipEntry zipEntry;
+      while ((zipEntry = zis.getNextEntry()) != null) {
+        if (zipEntry.getName().equals(Constants.EXPORT_TABLE_CONFIG_FILE)) {
+          BufferedReader in = new BufferedReader(new InputStreamReader(zis, Constants.UTF8));
+          try {
+            String line;
+            while ((line = in.readLine()) != null) {
+              String sa[] = line.split("=", 2);
+              props.put(sa[0], sa[1]);
+            }
+          } finally {
+            in.close();
+          }
+
+          break;
+        }
+      }
+    } finally {
+      zis.close();
+    }
+    return props;
+  }
+
+  @Override
+  public void importTable(String tableName, String importDir) throws TableExistsException, AccumuloException, AccumuloSecurityException {
+    ArgumentChecker.notNull(tableName, importDir);
+
+    try {
+      importDir = checkPath(importDir, "Table", "").toString();
+    } catch (IOException e) {
+      throw new AccumuloException(e);
+    }
+
+    try {
+      FileSystem fs = new Path(importDir).getFileSystem(CachedConfiguration.getInstance());
+      Map<String,String> props = getExportedProps(fs, new Path(importDir, Constants.EXPORT_FILE));
+
+      for (Entry<String,String> entry : props.entrySet()) {
+        if (Property.isClassProperty(entry.getKey()) && !entry.getValue().contains(Constants.CORE_PACKAGE_NAME)) {
+          Logger.getLogger(this.getClass()).info(
+              "Imported table sets '" + entry.getKey() + "' to '" + entry.getValue() + "'.  Ensure this class is on Accumulo classpath.");
+        }
+      }
+
+    } catch (IOException ioe) {
+      Logger.getLogger(this.getClass()).warn("Failed to check if imported table references external java classes : " + ioe.getMessage());
+    }
+
+    List<ByteBuffer> args = Arrays.asList(ByteBuffer.wrap(tableName.getBytes(Constants.UTF8)), ByteBuffer.wrap(importDir.getBytes(Constants.UTF8)));
+
+    Map<String,String> opts = Collections.emptyMap();
+
+    try {
+      doTableFateOperation(tableName, AccumuloException.class, FateOperation.TABLE_IMPORT, args, opts);
+    } catch (TableNotFoundException e) {
+      // should not happen
+      throw new AssertionError(e);
+    }
+
+  }
+
+  @Override
+  public void exportTable(String tableName, String exportDir) throws TableNotFoundException, AccumuloException, AccumuloSecurityException {
+    ArgumentChecker.notNull(tableName, exportDir);
+
+    List<ByteBuffer> args = Arrays.asList(ByteBuffer.wrap(tableName.getBytes(Constants.UTF8)), ByteBuffer.wrap(exportDir.getBytes(Constants.UTF8)));
+
+    Map<String,String> opts = Collections.emptyMap();
+
+    try {
+      doTableFateOperation(tableName, TableNotFoundException.class, FateOperation.TABLE_EXPORT, args, opts);
+    } catch (TableExistsException e) {
+      // should not happen
+      throw new AssertionError(e);
+    }
+  }
+
+  @Override
+  public boolean testClassLoad(final String tableName, final String className, final String asTypeName) throws TableNotFoundException, AccumuloException,
+      AccumuloSecurityException {
+    ArgumentChecker.notNull(tableName, className, asTypeName);
+
+    try {
+      return ServerClient.executeRaw(instance, new ClientExecReturn<Boolean,ClientService.Client>() {
+        @Override
+        public Boolean execute(ClientService.Client client) throws Exception {
+          return client.checkTableClass(Tracer.traceInfo(), credentials.toThrift(instance), tableName, className, asTypeName);
+        }
+      });
+    } catch (ThriftTableOperationException e) {
+      switch (e.getType()) {
+        case NOTFOUND:
+          throw new TableNotFoundException(e);
+        case NAMESPACE_NOTFOUND:
+          throw new TableNotFoundException(tableName, new NamespaceNotFoundException(e));
+        default:
+          throw new AccumuloException(e.description, e);
+      }
+    } catch (ThriftSecurityException e) {
+      throw new AccumuloSecurityException(e.user, e.code, e);
+    } catch (AccumuloException e) {
+      throw e;
+    } catch (Exception e) {
+      throw new AccumuloException(e);
+    }
+  }
+
+  @Override
+  public void attachIterator(String tableName, IteratorSetting setting, EnumSet<IteratorScope> scopes) throws AccumuloSecurityException, AccumuloException,
+      TableNotFoundException {
+    testClassLoad(tableName, setting.getIteratorClass(), SortedKeyValueIterator.class.getName());
+    super.attachIterator(tableName, setting, scopes);
+  }
+
+  @Override
+  public int addConstraint(String tableName, String constraintClassName) throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
+    testClassLoad(tableName, constraintClassName, Constraint.class.getName());
+    return super.addConstraint(tableName, constraintClassName);
+  }
+
+  private void doTableFateOperation(String tableName, Class<? extends Exception> namespaceNotFoundExceptionClass, FateOperation op, List<ByteBuffer> args,
+      Map<String,String> opts) throws AccumuloSecurityException, AccumuloException, TableExistsException, TableNotFoundException {
+    try {
+      doFateOperation(op, args, opts);
+    } catch (NamespaceExistsException e) {
+      // should not happen
+      throw new AssertionError(e);
+    } catch (NamespaceNotFoundException e) {
+      if (namespaceNotFoundExceptionClass == null) {
+        // should not happen
+        throw new AssertionError(e);
+      } else if (AccumuloException.class.isAssignableFrom(namespaceNotFoundExceptionClass)) {
+        throw new AccumuloException("Cannot create table in non-existent namespace", e);
+      } else if (TableNotFoundException.class.isAssignableFrom(namespaceNotFoundExceptionClass)) {
+        throw new TableNotFoundException(null, tableName, "Namespace not found", e);
+      } else {
+        // should not happen
+        throw new AssertionError(e);
+      }
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/f4454a06/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/impl/InputConfigurator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/impl/InputConfigurator.java b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/impl/InputConfigurator.java
index cfd9aa2..2fc606c 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/impl/InputConfigurator.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/impl/InputConfigurator.java
@@ -47,7 +47,7 @@ import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.client.impl.Tables;
 import org.apache.accumulo.core.client.impl.TabletLocator;
 import org.apache.accumulo.core.client.mapreduce.InputTableConfig;
-import org.apache.accumulo.core.client.mock.MockTabletLocator;
+import org.apache.accumulo.core.client.mock.impl.MockTabletLocator;
 import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.KeyExtent;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/f4454a06/core/src/main/java/org/apache/accumulo/core/client/mock/MockConnector.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mock/MockConnector.java b/core/src/main/java/org/apache/accumulo/core/client/mock/MockConnector.java
index 996198c..6cd6f2c 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mock/MockConnector.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mock/MockConnector.java
@@ -124,17 +124,17 @@ public class MockConnector extends Connector {
   
   @Override
   public TableOperations tableOperations() {
-    return new MockTableOperations(acu, username);
+    return new MockTableOperationsImpl(acu, username);
   }
   
   @Override
   public SecurityOperations securityOperations() {
-    return new MockSecurityOperations(acu);
+    return new MockSecurityOperationsImpl(acu);
   }
   
   @Override
   public InstanceOperations instanceOperations() {
-    return new MockInstanceOperations(acu);
+    return new MockInstanceOperationsImpl(acu);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/accumulo/blob/f4454a06/core/src/main/java/org/apache/accumulo/core/client/mock/MockInstanceOperations.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mock/MockInstanceOperations.java b/core/src/main/java/org/apache/accumulo/core/client/mock/MockInstanceOperations.java
index 15379af..58eff24 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mock/MockInstanceOperations.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mock/MockInstanceOperations.java
@@ -16,75 +16,13 @@
  */
 package org.apache.accumulo.core.client.mock;
 
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.accumulo.core.client.AccumuloException;
-import org.apache.accumulo.core.client.AccumuloSecurityException;
-import org.apache.accumulo.core.client.admin.ActiveCompaction;
-import org.apache.accumulo.core.client.admin.ActiveScan;
-import org.apache.accumulo.core.client.admin.InstanceOperations;
-import org.apache.accumulo.start.classloader.vfs.AccumuloVFSClassLoader;
-
 /**
- * 
+ * @deprecated since 1.6.0; not intended for public api and you should not use it.
  */
-public class MockInstanceOperations implements InstanceOperations {
-  MockAccumulo acu;
+@Deprecated
+public class MockInstanceOperations extends MockInstanceOperationsImpl {
   
   public MockInstanceOperations(MockAccumulo acu) {
-    this.acu = acu;
-  }
-  
-  @Override
-  public void setProperty(String property, String value) throws AccumuloException, AccumuloSecurityException {
-    acu.setProperty(property, value);
-  }
-  
-  @Override
-  public void removeProperty(String property) throws AccumuloException, AccumuloSecurityException {
-    acu.removeProperty(property);
-  }
-  
-  @Override
-  public Map<String,String> getSystemConfiguration() throws AccumuloException, AccumuloSecurityException {
-    return acu.systemProperties;
-  }
-  
-  @Override
-  public Map<String,String> getSiteConfiguration() throws AccumuloException, AccumuloSecurityException {
-    return acu.systemProperties;
-  }
-  
-  @Override
-  public List<String> getTabletServers() {
-    return new ArrayList<String>();
-  }
-  
-  @Override
-  public List<ActiveScan> getActiveScans(String tserver) throws AccumuloException, AccumuloSecurityException {
-    return new ArrayList<ActiveScan>();
-  }
-  
-  @Override
-  public boolean testClassLoad(String className, String asTypeName) throws AccumuloException, AccumuloSecurityException {
-    try {
-      AccumuloVFSClassLoader.loadClass(className, Class.forName(asTypeName));
-    } catch (ClassNotFoundException e) {
-      e.printStackTrace();
-      return false;
-    }
-    return true;
-  }
-  
-  @Override
-  public List<ActiveCompaction> getActiveCompactions(String tserver) throws AccumuloException, AccumuloSecurityException {
-    return new ArrayList<ActiveCompaction>();
-  }
-  
-  @Override
-  public void ping(String tserver) throws AccumuloException {
-    
+    super(acu);
   }
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/f4454a06/core/src/main/java/org/apache/accumulo/core/client/mock/MockInstanceOperationsImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mock/MockInstanceOperationsImpl.java b/core/src/main/java/org/apache/accumulo/core/client/mock/MockInstanceOperationsImpl.java
new file mode 100644
index 0000000..1bec226
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/client/mock/MockInstanceOperationsImpl.java
@@ -0,0 +1,87 @@
+/*
+ * 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.core.client.mock;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.accumulo.core.client.AccumuloException;
+import org.apache.accumulo.core.client.AccumuloSecurityException;
+import org.apache.accumulo.core.client.admin.ActiveCompaction;
+import org.apache.accumulo.core.client.admin.ActiveScan;
+import org.apache.accumulo.core.client.admin.InstanceOperations;
+import org.apache.accumulo.start.classloader.vfs.AccumuloVFSClassLoader;
+
+class MockInstanceOperationsImpl implements InstanceOperations {
+  MockAccumulo acu;
+
+  public MockInstanceOperationsImpl(MockAccumulo acu) {
+    this.acu = acu;
+  }
+
+  @Override
+  public void setProperty(String property, String value) throws AccumuloException, AccumuloSecurityException {
+    acu.setProperty(property, value);
+  }
+
+  @Override
+  public void removeProperty(String property) throws AccumuloException, AccumuloSecurityException {
+    acu.removeProperty(property);
+  }
+
+  @Override
+  public Map<String,String> getSystemConfiguration() throws AccumuloException, AccumuloSecurityException {
+    return acu.systemProperties;
+  }
+
+  @Override
+  public Map<String,String> getSiteConfiguration() throws AccumuloException, AccumuloSecurityException {
+    return acu.systemProperties;
+  }
+
+  @Override
+  public List<String> getTabletServers() {
+    return new ArrayList<String>();
+  }
+
+  @Override
+  public List<ActiveScan> getActiveScans(String tserver) throws AccumuloException, AccumuloSecurityException {
+    return new ArrayList<ActiveScan>();
+  }
+
+  @Override
+  public boolean testClassLoad(String className, String asTypeName) throws AccumuloException, AccumuloSecurityException {
+    try {
+      AccumuloVFSClassLoader.loadClass(className, Class.forName(asTypeName));
+    } catch (ClassNotFoundException e) {
+      e.printStackTrace();
+      return false;
+    }
+    return true;
+  }
+
+  @Override
+  public List<ActiveCompaction> getActiveCompactions(String tserver) throws AccumuloException, AccumuloSecurityException {
+    return new ArrayList<ActiveCompaction>();
+  }
+
+  @Override
+  public void ping(String tserver) throws AccumuloException {
+
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/f4454a06/core/src/main/java/org/apache/accumulo/core/client/mock/MockNamespaceOperations.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mock/MockNamespaceOperations.java b/core/src/main/java/org/apache/accumulo/core/client/mock/MockNamespaceOperations.java
index 9f0594a..77a9cb1 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mock/MockNamespaceOperations.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mock/MockNamespaceOperations.java
@@ -27,12 +27,12 @@ import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.NamespaceExistsException;
 import org.apache.accumulo.core.client.NamespaceNotEmptyException;
 import org.apache.accumulo.core.client.NamespaceNotFoundException;
-import org.apache.accumulo.core.client.admin.NamespaceOperationsHelper;
+import org.apache.accumulo.core.client.impl.NamespaceOperationsHelper;
 import org.apache.accumulo.core.client.impl.Namespaces;
 import org.apache.accumulo.core.client.impl.Tables;
 import org.apache.accumulo.start.classloader.vfs.AccumuloVFSClassLoader;
 
-public class MockNamespaceOperations extends NamespaceOperationsHelper {
+class MockNamespaceOperations extends NamespaceOperationsHelper {
 
   final private MockAccumulo acu;
   final private String username;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/f4454a06/core/src/main/java/org/apache/accumulo/core/client/mock/MockSecurityOperations.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mock/MockSecurityOperations.java b/core/src/main/java/org/apache/accumulo/core/client/mock/MockSecurityOperations.java
index 16a8e02..107c711 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mock/MockSecurityOperations.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mock/MockSecurityOperations.java
@@ -16,210 +16,9 @@
  */
 package org.apache.accumulo.core.client.mock;
 
-import java.util.EnumSet;
-import java.util.Set;
-
-import org.apache.accumulo.core.client.AccumuloException;
-import org.apache.accumulo.core.client.AccumuloSecurityException;
-import org.apache.accumulo.core.client.admin.SecurityOperations;
-import org.apache.accumulo.core.client.impl.thrift.SecurityErrorCode;
-import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
-import org.apache.accumulo.core.client.security.tokens.PasswordToken;
-import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.core.security.NamespacePermission;
-import org.apache.accumulo.core.security.SystemPermission;
-import org.apache.accumulo.core.security.TablePermission;
-
-public class MockSecurityOperations implements SecurityOperations {
-
-  final private MockAccumulo acu;
+public class MockSecurityOperations extends MockSecurityOperationsImpl {
 
   MockSecurityOperations(MockAccumulo acu) {
-    this.acu = acu;
-  }
-
-  @Deprecated
-  @Override
-  public void createUser(String user, byte[] password, Authorizations authorizations) throws AccumuloException, AccumuloSecurityException {
-    createLocalUser(user, new PasswordToken(password));
-    changeUserAuthorizations(user, authorizations);
-  }
-
-  @Override
-  public void createLocalUser(String principal, PasswordToken password) throws AccumuloException, AccumuloSecurityException {
-    this.acu.users.put(principal, new MockUser(principal, password, new Authorizations()));
-  }
-
-  @Deprecated
-  @Override
-  public void dropUser(String user) throws AccumuloException, AccumuloSecurityException {
-    dropLocalUser(user);
-  }
-
-  @Override
-  public void dropLocalUser(String principal) throws AccumuloException, AccumuloSecurityException {
-    this.acu.users.remove(principal);
-  }
-
-  @Deprecated
-  @Override
-  public boolean authenticateUser(String user, byte[] password) throws AccumuloException, AccumuloSecurityException {
-    return authenticateUser(user, new PasswordToken(password));
-  }
-
-  @Override
-  public boolean authenticateUser(String principal, AuthenticationToken token) throws AccumuloException, AccumuloSecurityException {
-    MockUser user = acu.users.get(principal);
-    if (user == null)
-      return false;
-    return user.token.equals(token);
-  }
-
-  @Deprecated
-  @Override
-  public void changeUserPassword(String user, byte[] password) throws AccumuloException, AccumuloSecurityException {
-    changeLocalUserPassword(user, new PasswordToken(password));
-  }
-
-  @Override
-  public void changeLocalUserPassword(String principal, PasswordToken token) throws AccumuloException, AccumuloSecurityException {
-    MockUser user = acu.users.get(principal);
-    if (user != null)
-      user.token = token.clone();
-    else
-      throw new AccumuloSecurityException(principal, SecurityErrorCode.USER_DOESNT_EXIST);
-  }
-
-  @Override
-  public void changeUserAuthorizations(String principal, Authorizations authorizations) throws AccumuloException, AccumuloSecurityException {
-    MockUser user = acu.users.get(principal);
-    if (user != null)
-      user.authorizations = authorizations;
-    else
-      throw new AccumuloSecurityException(principal, SecurityErrorCode.USER_DOESNT_EXIST);
-  }
-
-  @Override
-  public Authorizations getUserAuthorizations(String principal) throws AccumuloException, AccumuloSecurityException {
-    MockUser user = acu.users.get(principal);
-    if (user != null)
-      return user.authorizations;
-    else
-      throw new AccumuloSecurityException(principal, SecurityErrorCode.USER_DOESNT_EXIST);
-  }
-
-  @Override
-  public boolean hasSystemPermission(String principal, SystemPermission perm) throws AccumuloException, AccumuloSecurityException {
-    MockUser user = acu.users.get(principal);
-    if (user != null)
-      return user.permissions.contains(perm);
-    else
-      throw new AccumuloSecurityException(principal, SecurityErrorCode.USER_DOESNT_EXIST);
-  }
-
-  @Override
-  public boolean hasTablePermission(String principal, String tableName, TablePermission perm) throws AccumuloException, AccumuloSecurityException {
-    MockTable table = acu.tables.get(tableName);
-    if (table == null)
-      throw new AccumuloSecurityException(tableName, SecurityErrorCode.TABLE_DOESNT_EXIST);
-    EnumSet<TablePermission> perms = table.userPermissions.get(principal);
-    if (perms == null)
-      return false;
-    return perms.contains(perm);
-  }
-
-  @Override
-  public boolean hasNamespacePermission(String principal, String namespace, NamespacePermission permission) throws AccumuloException, AccumuloSecurityException {
-    MockNamespace mockNamespace = acu.namespaces.get(namespace);
-    if (mockNamespace == null)
-      throw new AccumuloSecurityException(namespace, SecurityErrorCode.NAMESPACE_DOESNT_EXIST);
-    EnumSet<NamespacePermission> perms = mockNamespace.userPermissions.get(principal);
-    if (perms == null)
-      return false;
-    return perms.contains(permission);
-  }
-
-  @Override
-  public void grantSystemPermission(String principal, SystemPermission permission) throws AccumuloException, AccumuloSecurityException {
-    MockUser user = acu.users.get(principal);
-    if (user != null)
-      user.permissions.add(permission);
-    else
-      throw new AccumuloSecurityException(principal, SecurityErrorCode.USER_DOESNT_EXIST);
-  }
-
-  @Override
-  public void grantTablePermission(String principal, String tableName, TablePermission permission) throws AccumuloException, AccumuloSecurityException {
-    if (acu.users.get(principal) == null)
-      throw new AccumuloSecurityException(principal, SecurityErrorCode.USER_DOESNT_EXIST);
-    MockTable table = acu.tables.get(tableName);
-    if (table == null)
-      throw new AccumuloSecurityException(tableName, SecurityErrorCode.TABLE_DOESNT_EXIST);
-    EnumSet<TablePermission> perms = table.userPermissions.get(principal);
-    if (perms == null)
-      table.userPermissions.put(principal, EnumSet.of(permission));
-    else
-      perms.add(permission);
-  }
-
-  @Override
-  public void grantNamespacePermission(String principal, String namespace, NamespacePermission permission) throws AccumuloException, AccumuloSecurityException {
-    if (acu.users.get(principal) == null)
-      throw new AccumuloSecurityException(principal, SecurityErrorCode.USER_DOESNT_EXIST);
-    MockNamespace mockNamespace = acu.namespaces.get(namespace);
-    if (mockNamespace == null)
-      throw new AccumuloSecurityException(namespace, SecurityErrorCode.NAMESPACE_DOESNT_EXIST);
-    EnumSet<NamespacePermission> perms = mockNamespace.userPermissions.get(principal);
-    if (perms == null)
-      mockNamespace.userPermissions.put(principal, EnumSet.of(permission));
-    else
-      perms.add(permission);
-  }
-
-  @Override
-  public void revokeSystemPermission(String principal, SystemPermission permission) throws AccumuloException, AccumuloSecurityException {
-    MockUser user = acu.users.get(principal);
-    if (user != null)
-      user.permissions.remove(permission);
-    else
-      throw new AccumuloSecurityException(principal, SecurityErrorCode.USER_DOESNT_EXIST);
-  }
-
-  @Override
-  public void revokeTablePermission(String principal, String tableName, TablePermission permission) throws AccumuloException, AccumuloSecurityException {
-    if (acu.users.get(principal) == null)
-      throw new AccumuloSecurityException(principal, SecurityErrorCode.USER_DOESNT_EXIST);
-    MockTable table = acu.tables.get(tableName);
-    if (table == null)
-      throw new AccumuloSecurityException(tableName, SecurityErrorCode.TABLE_DOESNT_EXIST);
-    EnumSet<TablePermission> perms = table.userPermissions.get(principal);
-    if (perms != null)
-      perms.remove(permission);
-
-  }
-
-  @Override
-  public void revokeNamespacePermission(String principal, String namespace, NamespacePermission permission) throws AccumuloException, AccumuloSecurityException {
-    if (acu.users.get(principal) == null)
-      throw new AccumuloSecurityException(principal, SecurityErrorCode.USER_DOESNT_EXIST);
-    MockNamespace mockNamespace = acu.namespaces.get(namespace);
-    if (mockNamespace == null)
-      throw new AccumuloSecurityException(namespace, SecurityErrorCode.NAMESPACE_DOESNT_EXIST);
-    EnumSet<NamespacePermission> perms = mockNamespace.userPermissions.get(principal);
-    if (perms != null)
-      perms.remove(permission);
-
-  }
-
-  @Deprecated
-  @Override
-  public Set<String> listUsers() throws AccumuloException, AccumuloSecurityException {
-    return listLocalUsers();
+    super(acu);
   }
-
-  @Override
-  public Set<String> listLocalUsers() throws AccumuloException, AccumuloSecurityException {
-    return acu.users.keySet();
-  }
-
 }


[07/17] git commit: Merge branch '1.5.2-SNAPSHOT' into 1.6.0-SNAPSHOT

Posted by bu...@apache.org.
Merge branch '1.5.2-SNAPSHOT' into 1.6.0-SNAPSHOT


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

Branch: refs/heads/master
Commit: 956f19b5f4745a6af98993ccf3929f5af02ce930
Parents: f4454a0 83ef6b8
Author: Christopher Tubbs <ct...@apache.org>
Authored: Fri Apr 25 13:43:30 2014 -0400
Committer: Christopher Tubbs <ct...@apache.org>
Committed: Fri Apr 25 13:43:30 2014 -0400

----------------------------------------------------------------------
 .../accumulo/core/client/admin/TableOperations.java  | 15 +++++++--------
 1 file changed, 7 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/956f19b5/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperations.java
----------------------------------------------------------------------


[09/17] Merge branch '1.6.0-SNAPSHOT'

Posted by bu...@apache.org.
http://git-wip-us.apache.org/repos/asf/accumulo/blob/592c8b06/mapreduce/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/impl/InputConfigurator.java
----------------------------------------------------------------------
diff --cc mapreduce/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/impl/InputConfigurator.java
index 7657c3c,0000000..69bbef2
mode 100644,000000..100644
--- a/mapreduce/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/impl/InputConfigurator.java
+++ b/mapreduce/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/impl/InputConfigurator.java
@@@ -1,796 -1,0 +1,796 @@@
 +/*
 + * 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.core.client.mapreduce.lib.impl;
 +
 +import static com.google.common.base.Preconditions.checkArgument;
 +
 +import java.io.ByteArrayInputStream;
 +import java.io.ByteArrayOutputStream;
 +import java.io.DataInputStream;
 +import java.io.DataOutputStream;
 +import java.io.IOException;
 +import java.nio.charset.StandardCharsets;
 +import java.util.ArrayList;
 +import java.util.Collection;
 +import java.util.HashMap;
 +import java.util.HashSet;
 +import java.util.Iterator;
 +import java.util.List;
 +import java.util.Map;
 +import java.util.Set;
 +import java.util.StringTokenizer;
 +
 +import org.apache.accumulo.core.client.AccumuloException;
 +import org.apache.accumulo.core.client.AccumuloSecurityException;
 +import org.apache.accumulo.core.client.ClientSideIteratorScanner;
 +import org.apache.accumulo.core.client.Connector;
 +import org.apache.accumulo.core.client.Instance;
 +import org.apache.accumulo.core.client.IsolatedScanner;
 +import org.apache.accumulo.core.client.IteratorSetting;
 +import org.apache.accumulo.core.client.RowIterator;
 +import org.apache.accumulo.core.client.Scanner;
 +import org.apache.accumulo.core.client.TableNotFoundException;
 +import org.apache.accumulo.core.client.impl.Tables;
 +import org.apache.accumulo.core.client.impl.TabletLocator;
 +import org.apache.accumulo.core.client.mapreduce.InputTableConfig;
- import org.apache.accumulo.core.client.mock.MockTabletLocator;
++import org.apache.accumulo.core.client.mock.impl.MockTabletLocator;
 +import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
 +import org.apache.accumulo.core.data.Key;
 +import org.apache.accumulo.core.data.KeyExtent;
 +import org.apache.accumulo.core.data.PartialKey;
 +import org.apache.accumulo.core.data.Range;
 +import org.apache.accumulo.core.data.Value;
 +import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
 +import org.apache.accumulo.core.master.state.tables.TableState;
 +import org.apache.accumulo.core.metadata.MetadataTable;
 +import org.apache.accumulo.core.metadata.schema.MetadataSchema;
 +import org.apache.accumulo.core.security.Authorizations;
 +import org.apache.accumulo.core.security.TablePermission;
 +import org.apache.accumulo.core.util.Pair;
 +import org.apache.accumulo.core.util.TextUtil;
 +import org.apache.commons.codec.binary.Base64;
 +import org.apache.hadoop.conf.Configuration;
 +import org.apache.hadoop.io.MapWritable;
 +import org.apache.hadoop.io.Text;
 +import org.apache.hadoop.io.Writable;
 +import org.apache.hadoop.util.StringUtils;
 +
 +import com.google.common.collect.Maps;
 +
 +/**
 + * @since 1.6.0
 + */
 +public class InputConfigurator extends ConfiguratorBase {
 +
 +  /**
 +   * Configuration keys for {@link Scanner}.
 +   * 
 +   * @since 1.6.0
 +   */
 +  public static enum ScanOpts {
 +    TABLE_NAME, AUTHORIZATIONS, RANGES, COLUMNS, ITERATORS, TABLE_CONFIGS
 +  }
 +
 +  /**
 +   * Configuration keys for various features.
 +   * 
 +   * @since 1.6.0
 +   */
 +  public static enum Features {
 +    AUTO_ADJUST_RANGES, SCAN_ISOLATION, USE_LOCAL_ITERATORS, SCAN_OFFLINE
 +  }
 +
 +  /**
 +   * Sets the name of the input table, over which this job will scan.
 +   * 
 +   * @param implementingClass
 +   *          the class whose name will be used as a prefix for the property configuration key
 +   * @param conf
 +   *          the Hadoop configuration object to configure
 +   * @param tableName
 +   *          the table to use when the tablename is null in the write call
 +   * @since 1.6.0
 +   */
 +  public static void setInputTableName(Class<?> implementingClass, Configuration conf, String tableName) {
 +    checkArgument(tableName != null, "tableName is null");
 +    conf.set(enumToConfKey(implementingClass, ScanOpts.TABLE_NAME), tableName);
 +  }
 +
 +  /**
 +   * Sets the name of the input table, over which this job will scan.
 +   * 
 +   * @param implementingClass
 +   *          the class whose name will be used as a prefix for the property configuration key
 +   * @param conf
 +   *          the Hadoop configuration object to configure
 +   * @since 1.6.0
 +   */
 +  public static String getInputTableName(Class<?> implementingClass, Configuration conf) {
 +    return conf.get(enumToConfKey(implementingClass, ScanOpts.TABLE_NAME));
 +  }
 +
 +  /**
 +   * Sets the {@link Authorizations} used to scan. Must be a subset of the user's authorization. Defaults to the empty set.
 +   * 
 +   * @param implementingClass
 +   *          the class whose name will be used as a prefix for the property configuration key
 +   * @param conf
 +   *          the Hadoop configuration object to configure
 +   * @param auths
 +   *          the user's authorizations
 +   * @since 1.6.0
 +   */
 +  public static void setScanAuthorizations(Class<?> implementingClass, Configuration conf, Authorizations auths) {
 +    if (auths != null && !auths.isEmpty())
 +      conf.set(enumToConfKey(implementingClass, ScanOpts.AUTHORIZATIONS), auths.serialize());
 +  }
 +
 +  /**
 +   * Gets the authorizations to set for the scans from the configuration.
 +   * 
 +   * @param implementingClass
 +   *          the class whose name will be used as a prefix for the property configuration key
 +   * @param conf
 +   *          the Hadoop configuration object to configure
 +   * @return the Accumulo scan authorizations
 +   * @since 1.6.0
 +   * @see #setScanAuthorizations(Class, Configuration, Authorizations)
 +   */
 +  public static Authorizations getScanAuthorizations(Class<?> implementingClass, Configuration conf) {
 +    String authString = conf.get(enumToConfKey(implementingClass, ScanOpts.AUTHORIZATIONS));
 +    return authString == null ? Authorizations.EMPTY : new Authorizations(authString.getBytes(StandardCharsets.UTF_8));
 +  }
 +
 +  /**
 +   * Sets the input ranges to scan on all input tables for this job. If not set, the entire table will be scanned.
 +   * 
 +   * @param implementingClass
 +   *          the class whose name will be used as a prefix for the property configuration key
 +   * @param conf
 +   *          the Hadoop configuration object to configure
 +   * @param ranges
 +   *          the ranges that will be mapped over
 +   * @throws IllegalArgumentException
 +   *           if the ranges cannot be encoded into base 64
 +   * @since 1.6.0
 +   */
 +  public static void setRanges(Class<?> implementingClass, Configuration conf, Collection<Range> ranges) {
 +    checkArgument(ranges != null, "ranges is null");
 +
 +    ArrayList<String> rangeStrings = new ArrayList<String>(ranges.size());
 +    try {
 +      for (Range r : ranges) {
 +        ByteArrayOutputStream baos = new ByteArrayOutputStream();
 +        r.write(new DataOutputStream(baos));
 +        rangeStrings.add(new String(Base64.encodeBase64(baos.toByteArray())));
 +      }
 +      conf.setStrings(enumToConfKey(implementingClass, ScanOpts.RANGES), rangeStrings.toArray(new String[0]));
 +    } catch (IOException ex) {
 +      throw new IllegalArgumentException("Unable to encode ranges to Base64", ex);
 +    }
 +  }
 +
 +  /**
 +   * Gets the ranges to scan over from a job.
 +   * 
 +   * @param implementingClass
 +   *          the class whose name will be used as a prefix for the property configuration key
 +   * @param conf
 +   *          the Hadoop configuration object to configure
 +   * @return the ranges
 +   * @throws IOException
 +   *           if the ranges have been encoded improperly
 +   * @since 1.6.0
 +   * @see #setRanges(Class, Configuration, Collection)
 +   */
 +  public static List<Range> getRanges(Class<?> implementingClass, Configuration conf) throws IOException {
 +
 +    Collection<String> encodedRanges = conf.getStringCollection(enumToConfKey(implementingClass, ScanOpts.RANGES));
 +    List<Range> ranges = new ArrayList<Range>();
 +    for (String rangeString : encodedRanges) {
 +      ByteArrayInputStream bais = new ByteArrayInputStream(Base64.decodeBase64(rangeString.getBytes()));
 +      Range range = new Range();
 +      range.readFields(new DataInputStream(bais));
 +      ranges.add(range);
 +    }
 +    return ranges;
 +  }
 +
 +  /**
 +   * Gets a list of the iterator settings (for iterators to apply to a scanner) from this configuration.
 +   * 
 +   * @param implementingClass
 +   *          the class whose name will be used as a prefix for the property configuration key
 +   * @param conf
 +   *          the Hadoop configuration object to configure
 +   * @return a list of iterators
 +   * @since 1.6.0
 +   * @see #addIterator(Class, Configuration, IteratorSetting)
 +   */
 +  public static List<IteratorSetting> getIterators(Class<?> implementingClass, Configuration conf) {
 +    String iterators = conf.get(enumToConfKey(implementingClass, ScanOpts.ITERATORS));
 +
 +    // If no iterators are present, return an empty list
 +    if (iterators == null || iterators.isEmpty())
 +      return new ArrayList<IteratorSetting>();
 +
 +    // Compose the set of iterators encoded in the job configuration
 +    StringTokenizer tokens = new StringTokenizer(iterators, StringUtils.COMMA_STR);
 +    List<IteratorSetting> list = new ArrayList<IteratorSetting>();
 +    try {
 +      while (tokens.hasMoreTokens()) {
 +        String itstring = tokens.nextToken();
 +        ByteArrayInputStream bais = new ByteArrayInputStream(Base64.decodeBase64(itstring.getBytes()));
 +        list.add(new IteratorSetting(new DataInputStream(bais)));
 +        bais.close();
 +      }
 +    } catch (IOException e) {
 +      throw new IllegalArgumentException("couldn't decode iterator settings");
 +    }
 +    return list;
 +  }
 +
 +  /**
 +   * Restricts the columns that will be mapped over for the single input table on this job.
 +   * 
 +   * @param implementingClass
 +   *          the class whose name will be used as a prefix for the property configuration key
 +   * @param conf
 +   *          the Hadoop configuration object to configure
 +   * @param columnFamilyColumnQualifierPairs
 +   *          a pair of {@link Text} objects corresponding to column family and column qualifier. If the column qualifier is null, the entire column family is
 +   *          selected. An empty set is the default and is equivalent to scanning the all columns.
 +   * @throws IllegalArgumentException
 +   *           if the column family is null
 +   * @since 1.6.0
 +   */
 +  public static void fetchColumns(Class<?> implementingClass, Configuration conf, Collection<Pair<Text,Text>> columnFamilyColumnQualifierPairs) {
 +    checkArgument(columnFamilyColumnQualifierPairs != null, "columnFamilyColumnQualifierPairs is null");
 +    String[] columnStrings = serializeColumns(columnFamilyColumnQualifierPairs);
 +    conf.setStrings(enumToConfKey(implementingClass, ScanOpts.COLUMNS), columnStrings);
 +  }
 +
 +  public static String[] serializeColumns(Collection<Pair<Text,Text>> columnFamilyColumnQualifierPairs) {
 +    checkArgument(columnFamilyColumnQualifierPairs != null, "columnFamilyColumnQualifierPairs is null");
 +    ArrayList<String> columnStrings = new ArrayList<String>(columnFamilyColumnQualifierPairs.size());
 +    for (Pair<Text,Text> column : columnFamilyColumnQualifierPairs) {
 +
 +      if (column.getFirst() == null)
 +        throw new IllegalArgumentException("Column family can not be null");
 +
 +      String col = new String(Base64.encodeBase64(TextUtil.getBytes(column.getFirst())), StandardCharsets.UTF_8);
 +      if (column.getSecond() != null)
 +        col += ":" + new String(Base64.encodeBase64(TextUtil.getBytes(column.getSecond())), StandardCharsets.UTF_8);
 +      columnStrings.add(col);
 +    }
 +
 +    return columnStrings.toArray(new String[0]);
 +  }
 +
 +  /**
 +   * Gets the columns to be mapped over from this job.
 +   * 
 +   * @param implementingClass
 +   *          the class whose name will be used as a prefix for the property configuration key
 +   * @param conf
 +   *          the Hadoop configuration object to configure
 +   * @return a set of columns
 +   * @since 1.6.0
 +   * @see #fetchColumns(Class, Configuration, Collection)
 +   */
 +  public static Set<Pair<Text,Text>> getFetchedColumns(Class<?> implementingClass, Configuration conf) {
 +    checkArgument(conf != null, "conf is null");
 +    String confValue = conf.get(enumToConfKey(implementingClass, ScanOpts.COLUMNS));
 +    List<String> serialized = new ArrayList<String>();
 +    if (confValue != null) {
 +      // Split and include any trailing empty strings to allow empty column families
 +      for (String val : confValue.split(",", -1)) {
 +        serialized.add(val);
 +      }
 +    }
 +    return deserializeFetchedColumns(serialized);
 +  }
 +
 +  public static Set<Pair<Text,Text>> deserializeFetchedColumns(Collection<String> serialized) {
 +    Set<Pair<Text,Text>> columns = new HashSet<Pair<Text,Text>>();
 +
 +    if (null == serialized) {
 +      return columns;
 +    }
 +
 +    for (String col : serialized) {
 +      int idx = col.indexOf(":");
 +      Text cf = new Text(idx < 0 ? Base64.decodeBase64(col.getBytes(StandardCharsets.UTF_8)) : Base64.decodeBase64(col.substring(0, idx).getBytes(
 +          StandardCharsets.UTF_8)));
 +      Text cq = idx < 0 ? null : new Text(Base64.decodeBase64(col.substring(idx + 1).getBytes(StandardCharsets.UTF_8)));
 +      columns.add(new Pair<Text,Text>(cf, cq));
 +    }
 +    return columns;
 +  }
 +
 +  /**
 +   * Encode an iterator on the input for the single input table associated with this job.
 +   * 
 +   * @param implementingClass
 +   *          the class whose name will be used as a prefix for the property configuration key
 +   * @param conf
 +   *          the Hadoop configuration object to configure
 +   * @param cfg
 +   *          the configuration of the iterator
 +   * @throws IllegalArgumentException
 +   *           if the iterator can't be serialized into the configuration
 +   * @since 1.6.0
 +   */
 +  public static void addIterator(Class<?> implementingClass, Configuration conf, IteratorSetting cfg) {
 +    ByteArrayOutputStream baos = new ByteArrayOutputStream();
 +    String newIter;
 +    try {
 +      cfg.write(new DataOutputStream(baos));
 +      newIter = new String(Base64.encodeBase64(baos.toByteArray()), StandardCharsets.UTF_8);
 +      baos.close();
 +    } catch (IOException e) {
 +      throw new IllegalArgumentException("unable to serialize IteratorSetting");
 +    }
 +
 +    String confKey = enumToConfKey(implementingClass, ScanOpts.ITERATORS);
 +    String iterators = conf.get(confKey);
 +    // No iterators specified yet, create a new string
 +    if (iterators == null || iterators.isEmpty()) {
 +      iterators = newIter;
 +    } else {
 +      // append the next iterator & reset
 +      iterators = iterators.concat(StringUtils.COMMA_STR + newIter);
 +    }
 +    // Store the iterators w/ the job
 +    conf.set(confKey, iterators);
 +  }
 +
 +  /**
 +   * Controls the automatic adjustment of ranges for this job. This feature merges overlapping ranges, then splits them to align with tablet boundaries.
 +   * Disabling this feature will cause exactly one Map task to be created for each specified range. The default setting is enabled. *
 +   * 
 +   * <p>
 +   * By default, this feature is <b>enabled</b>.
 +   * 
 +   * @param implementingClass
 +   *          the class whose name will be used as a prefix for the property configuration key
 +   * @param conf
 +   *          the Hadoop configuration object to configure
 +   * @param enableFeature
 +   *          the feature is enabled if true, disabled otherwise
 +   * @see #setRanges(Class, Configuration, Collection)
 +   * @since 1.6.0
 +   */
 +  public static void setAutoAdjustRanges(Class<?> implementingClass, Configuration conf, boolean enableFeature) {
 +    conf.setBoolean(enumToConfKey(implementingClass, Features.AUTO_ADJUST_RANGES), enableFeature);
 +  }
 +
 +  /**
 +   * Determines whether a configuration has auto-adjust ranges enabled.
 +   * 
 +   * @param implementingClass
 +   *          the class whose name will be used as a prefix for the property configuration key
 +   * @param conf
 +   *          the Hadoop configuration object to configure
 +   * @return false if the feature is disabled, true otherwise
 +   * @since 1.6.0
 +   * @see #setAutoAdjustRanges(Class, Configuration, boolean)
 +   */
 +  public static Boolean getAutoAdjustRanges(Class<?> implementingClass, Configuration conf) {
 +    return conf.getBoolean(enumToConfKey(implementingClass, Features.AUTO_ADJUST_RANGES), true);
 +  }
 +
 +  /**
 +   * Controls the use of the {@link IsolatedScanner} in this job.
 +   * 
 +   * <p>
 +   * By default, this feature is <b>disabled</b>.
 +   * 
 +   * @param implementingClass
 +   *          the class whose name will be used as a prefix for the property configuration key
 +   * @param conf
 +   *          the Hadoop configuration object to configure
 +   * @param enableFeature
 +   *          the feature is enabled if true, disabled otherwise
 +   * @since 1.6.0
 +   */
 +  public static void setScanIsolation(Class<?> implementingClass, Configuration conf, boolean enableFeature) {
 +    conf.setBoolean(enumToConfKey(implementingClass, Features.SCAN_ISOLATION), enableFeature);
 +  }
 +
 +  /**
 +   * Determines whether a configuration has isolation enabled.
 +   * 
 +   * @param implementingClass
 +   *          the class whose name will be used as a prefix for the property configuration key
 +   * @param conf
 +   *          the Hadoop configuration object to configure
 +   * @return true if the feature is enabled, false otherwise
 +   * @since 1.6.0
 +   * @see #setScanIsolation(Class, Configuration, boolean)
 +   */
 +  public static Boolean isIsolated(Class<?> implementingClass, Configuration conf) {
 +    return conf.getBoolean(enumToConfKey(implementingClass, Features.SCAN_ISOLATION), false);
 +  }
 +
 +  /**
 +   * Controls the use of the {@link ClientSideIteratorScanner} in this job. Enabling this feature will cause the iterator stack to be constructed within the Map
 +   * task, rather than within the Accumulo TServer. To use this feature, all classes needed for those iterators must be available on the classpath for the task.
 +   * 
 +   * <p>
 +   * By default, this feature is <b>disabled</b>.
 +   * 
 +   * @param implementingClass
 +   *          the class whose name will be used as a prefix for the property configuration key
 +   * @param conf
 +   *          the Hadoop configuration object to configure
 +   * @param enableFeature
 +   *          the feature is enabled if true, disabled otherwise
 +   * @since 1.6.0
 +   */
 +  public static void setLocalIterators(Class<?> implementingClass, Configuration conf, boolean enableFeature) {
 +    conf.setBoolean(enumToConfKey(implementingClass, Features.USE_LOCAL_ITERATORS), enableFeature);
 +  }
 +
 +  /**
 +   * Determines whether a configuration uses local iterators.
 +   * 
 +   * @param implementingClass
 +   *          the class whose name will be used as a prefix for the property configuration key
 +   * @param conf
 +   *          the Hadoop configuration object to configure
 +   * @return true if the feature is enabled, false otherwise
 +   * @since 1.6.0
 +   * @see #setLocalIterators(Class, Configuration, boolean)
 +   */
 +  public static Boolean usesLocalIterators(Class<?> implementingClass, Configuration conf) {
 +    return conf.getBoolean(enumToConfKey(implementingClass, Features.USE_LOCAL_ITERATORS), false);
 +  }
 +
 +  /**
 +   * <p>
 +   * Enable reading offline tables. By default, this feature is disabled and only online tables are scanned. This will make the map reduce job directly read the
 +   * table's files. If the table is not offline, then the job will fail. If the table comes online during the map reduce job, it is likely that the job will
 +   * fail.
 +   * 
 +   * <p>
 +   * To use this option, the map reduce user will need access to read the Accumulo directory in HDFS.
 +   * 
 +   * <p>
 +   * Reading the offline table will create the scan time iterator stack in the map process. So any iterators that are configured for the table will need to be
 +   * on the mapper's classpath.
 +   * 
 +   * <p>
 +   * One way to use this feature is to clone a table, take the clone offline, and use the clone as the input table for a map reduce job. If you plan to map
 +   * reduce over the data many times, it may be better to the compact the table, clone it, take it offline, and use the clone for all map reduce jobs. The
 +   * reason to do this is that compaction will reduce each tablet in the table to one file, and it is faster to read from one file.
 +   * 
 +   * <p>
 +   * There are two possible advantages to reading a tables file directly out of HDFS. First, you may see better read performance. Second, it will support
 +   * speculative execution better. When reading an online table speculative execution can put more load on an already slow tablet server.
 +   * 
 +   * <p>
 +   * By default, this feature is <b>disabled</b>.
 +   * 
 +   * @param implementingClass
 +   *          the class whose name will be used as a prefix for the property configuration key
 +   * @param conf
 +   *          the Hadoop configuration object to configure
 +   * @param enableFeature
 +   *          the feature is enabled if true, disabled otherwise
 +   * @since 1.6.0
 +   */
 +  public static void setOfflineTableScan(Class<?> implementingClass, Configuration conf, boolean enableFeature) {
 +    conf.setBoolean(enumToConfKey(implementingClass, Features.SCAN_OFFLINE), enableFeature);
 +  }
 +
 +  /**
 +   * Determines whether a configuration has the offline table scan feature enabled.
 +   * 
 +   * @param implementingClass
 +   *          the class whose name will be used as a prefix for the property configuration key
 +   * @param conf
 +   *          the Hadoop configuration object to configure
 +   * @return true if the feature is enabled, false otherwise
 +   * @since 1.6.0
 +   * @see #setOfflineTableScan(Class, Configuration, boolean)
 +   */
 +  public static Boolean isOfflineScan(Class<?> implementingClass, Configuration conf) {
 +    return conf.getBoolean(enumToConfKey(implementingClass, Features.SCAN_OFFLINE), false);
 +  }
 +
 +  /**
 +   * Sets configurations for multiple tables at a time.
 +   * 
 +   * @param implementingClass
 +   *          the class whose name will be used as a prefix for the property configuration key
 +   * @param conf
 +   *          the Hadoop configuration object to configure
 +   * @param configs
 +   *          an array of {@link InputTableConfig} objects to associate with the job
 +   * @since 1.6.0
 +   */
 +  public static void setInputTableConfigs(Class<?> implementingClass, Configuration conf, Map<String,InputTableConfig> configs) {
 +    MapWritable mapWritable = new MapWritable();
 +    for (Map.Entry<String,InputTableConfig> tableConfig : configs.entrySet())
 +      mapWritable.put(new Text(tableConfig.getKey()), tableConfig.getValue());
 +
 +    ByteArrayOutputStream baos = new ByteArrayOutputStream();
 +    try {
 +      mapWritable.write(new DataOutputStream(baos));
 +    } catch (IOException e) {
 +      throw new IllegalStateException("Table configuration could not be serialized.");
 +    }
 +
 +    String confKey = enumToConfKey(implementingClass, ScanOpts.TABLE_CONFIGS);
 +    conf.set(confKey, new String(Base64.encodeBase64(baos.toByteArray())));
 +  }
 +
 +  /**
 +   * Returns all {@link InputTableConfig} objects associated with this job.
 +   * 
 +   * @param implementingClass
 +   *          the class whose name will be used as a prefix for the property configuration key
 +   * @param conf
 +   *          the Hadoop configuration object to configure
 +   * @return all of the table query configs for the job
 +   * @since 1.6.0
 +   */
 +  public static Map<String,InputTableConfig> getInputTableConfigs(Class<?> implementingClass, Configuration conf) {
 +    Map<String,InputTableConfig> configs = new HashMap<String,InputTableConfig>();
 +    Map.Entry<String,InputTableConfig> defaultConfig = getDefaultInputTableConfig(implementingClass, conf);
 +    if (defaultConfig != null)
 +      configs.put(defaultConfig.getKey(), defaultConfig.getValue());
 +    String configString = conf.get(enumToConfKey(implementingClass, ScanOpts.TABLE_CONFIGS));
 +    MapWritable mapWritable = new MapWritable();
 +    if (configString != null) {
 +      try {
 +        byte[] bytes = Base64.decodeBase64(configString.getBytes());
 +        ByteArrayInputStream bais = new ByteArrayInputStream(bytes);
 +        mapWritable.readFields(new DataInputStream(bais));
 +        bais.close();
 +      } catch (IOException e) {
 +        throw new IllegalStateException("The table query configurations could not be deserialized from the given configuration");
 +      }
 +    }
 +    for (Map.Entry<Writable,Writable> entry : mapWritable.entrySet())
 +      configs.put(((Text) entry.getKey()).toString(), (InputTableConfig) entry.getValue());
 +
 +    return configs;
 +  }
 +
 +  /**
 +   * Returns the {@link InputTableConfig} for the given table
 +   * 
 +   * @param implementingClass
 +   *          the class whose name will be used as a prefix for the property configuration key
 +   * @param conf
 +   *          the Hadoop configuration object to configure
 +   * @param tableName
 +   *          the table name for which to fetch the table query config
 +   * @return the table query config for the given table name (if it exists) and null if it does not
 +   * @since 1.6.0
 +   */
 +  public static InputTableConfig getInputTableConfig(Class<?> implementingClass, Configuration conf, String tableName) {
 +    Map<String,InputTableConfig> queryConfigs = getInputTableConfigs(implementingClass, conf);
 +    return queryConfigs.get(tableName);
 +  }
 +
 +  /**
 +   * Initializes an Accumulo {@link TabletLocator} based on the configuration.
 +   * 
 +   * @param implementingClass
 +   *          the class whose name will be used as a prefix for the property configuration key
 +   * @param conf
 +   *          the Hadoop configuration object to configure
 +   * @param tableId
 +   *          The table id for which to initialize the {@link TabletLocator}
 +   * @return an Accumulo tablet locator
 +   * @throws TableNotFoundException
 +   *           if the table name set on the configuration doesn't exist
 +   * @since 1.6.0
 +   */
 +  public static TabletLocator getTabletLocator(Class<?> implementingClass, Configuration conf, String tableId) throws TableNotFoundException {
 +    String instanceType = conf.get(enumToConfKey(implementingClass, InstanceOpts.TYPE));
 +    if ("MockInstance".equals(instanceType))
 +      return new MockTabletLocator();
 +    Instance instance = getInstance(implementingClass, conf);
 +    return TabletLocator.getLocator(instance, new Text(tableId));
 +  }
 +
 +  // InputFormat doesn't have the equivalent of OutputFormat's checkOutputSpecs(JobContext job)
 +  /**
 +   * Check whether a configuration is fully configured to be used with an Accumulo {@link org.apache.hadoop.mapreduce.InputFormat}.
 +   * 
 +   * @param implementingClass
 +   *          the class whose name will be used as a prefix for the property configuration key
 +   * @param conf
 +   *          the Hadoop configuration object to configure
 +   * @throws IOException
 +   *           if the context is improperly configured
 +   * @since 1.6.0
 +   */
 +  public static void validateOptions(Class<?> implementingClass, Configuration conf) throws IOException {
 +
 +    Map<String,InputTableConfig> inputTableConfigs = getInputTableConfigs(implementingClass, conf);
 +    if (!isConnectorInfoSet(implementingClass, conf))
 +      throw new IOException("Input info has not been set.");
 +    String instanceKey = conf.get(enumToConfKey(implementingClass, InstanceOpts.TYPE));
 +    if (!"MockInstance".equals(instanceKey) && !"ZooKeeperInstance".equals(instanceKey))
 +      throw new IOException("Instance info has not been set.");
 +    // validate that we can connect as configured
 +    try {
 +      String principal = getPrincipal(implementingClass, conf);
 +      AuthenticationToken token = getAuthenticationToken(implementingClass, conf);
 +      Connector c = getInstance(implementingClass, conf).getConnector(principal, token);
 +      if (!c.securityOperations().authenticateUser(principal, token))
 +        throw new IOException("Unable to authenticate user");
 +
 +      if (getInputTableConfigs(implementingClass, conf).size() == 0)
 +        throw new IOException("No table set.");
 +
 +      for (Map.Entry<String,InputTableConfig> tableConfig : inputTableConfigs.entrySet()) {
 +        if (!c.securityOperations().hasTablePermission(getPrincipal(implementingClass, conf), tableConfig.getKey(), TablePermission.READ))
 +          throw new IOException("Unable to access table");
 +      }
 +      for (Map.Entry<String,InputTableConfig> tableConfigEntry : inputTableConfigs.entrySet()) {
 +        InputTableConfig tableConfig = tableConfigEntry.getValue();
 +        if (!tableConfig.shouldUseLocalIterators()) {
 +          if (tableConfig.getIterators() != null) {
 +            for (IteratorSetting iter : tableConfig.getIterators()) {
 +              if (!c.tableOperations().testClassLoad(tableConfigEntry.getKey(), iter.getIteratorClass(), SortedKeyValueIterator.class.getName()))
 +                throw new AccumuloException("Servers are unable to load " + iter.getIteratorClass() + " as a " + SortedKeyValueIterator.class.getName());
 +            }
 +          }
 +        }
 +      }
 +    } catch (AccumuloException e) {
 +      throw new IOException(e);
 +    } catch (AccumuloSecurityException e) {
 +      throw new IOException(e);
 +    } catch (TableNotFoundException e) {
 +      throw new IOException(e);
 +    }
 +  }
 +
 +  /**
 +   * Returns the {@link org.apache.accumulo.core.client.mapreduce.InputTableConfig} for the configuration based on the properties set using the single-table
 +   * input methods.
 +   * 
 +   * @param implementingClass
 +   *          the class whose name will be used as a prefix for the property configuration key
 +   * @param conf
 +   *          the Hadoop instance for which to retrieve the configuration
 +   * @return the config object built from the single input table properties set on the job
 +   * @since 1.6.0
 +   */
 +  protected static Map.Entry<String,InputTableConfig> getDefaultInputTableConfig(Class<?> implementingClass, Configuration conf) {
 +    String tableName = getInputTableName(implementingClass, conf);
 +    if (tableName != null) {
 +      InputTableConfig queryConfig = new InputTableConfig();
 +      List<IteratorSetting> itrs = getIterators(implementingClass, conf);
 +      if (itrs != null)
 +        queryConfig.setIterators(itrs);
 +      Set<Pair<Text,Text>> columns = getFetchedColumns(implementingClass, conf);
 +      if (columns != null)
 +        queryConfig.fetchColumns(columns);
 +      List<Range> ranges = null;
 +      try {
 +        ranges = getRanges(implementingClass, conf);
 +      } catch (IOException e) {
 +        throw new RuntimeException(e);
 +      }
 +      if (ranges != null)
 +        queryConfig.setRanges(ranges);
 +
 +      queryConfig.setAutoAdjustRanges(getAutoAdjustRanges(implementingClass, conf)).setUseIsolatedScanners(isIsolated(implementingClass, conf))
 +          .setUseLocalIterators(usesLocalIterators(implementingClass, conf)).setOfflineScan(isOfflineScan(implementingClass, conf));
 +      return Maps.immutableEntry(tableName, queryConfig);
 +    }
 +    return null;
 +  }
 +
 +  public static Map<String,Map<KeyExtent,List<Range>>> binOffline(String tableId, List<Range> ranges, Instance instance, Connector conn)
 +      throws AccumuloException, TableNotFoundException {
 +    Map<String,Map<KeyExtent,List<Range>>> binnedRanges = new HashMap<String,Map<KeyExtent,List<Range>>>();
 +
 +    if (Tables.getTableState(instance, tableId) != TableState.OFFLINE) {
 +      Tables.clearCache(instance);
 +      if (Tables.getTableState(instance, tableId) != TableState.OFFLINE) {
 +        throw new AccumuloException("Table is online tableId:" + tableId + " cannot scan table in offline mode ");
 +      }
 +    }
 +
 +    for (Range range : ranges) {
 +      Text startRow;
 +
 +      if (range.getStartKey() != null)
 +        startRow = range.getStartKey().getRow();
 +      else
 +        startRow = new Text();
 +
 +      Range metadataRange = new Range(new KeyExtent(new Text(tableId), startRow, null).getMetadataEntry(), true, null, false);
 +      Scanner scanner = conn.createScanner(MetadataTable.NAME, Authorizations.EMPTY);
 +      MetadataSchema.TabletsSection.TabletColumnFamily.PREV_ROW_COLUMN.fetch(scanner);
 +      scanner.fetchColumnFamily(MetadataSchema.TabletsSection.LastLocationColumnFamily.NAME);
 +      scanner.fetchColumnFamily(MetadataSchema.TabletsSection.CurrentLocationColumnFamily.NAME);
 +      scanner.fetchColumnFamily(MetadataSchema.TabletsSection.FutureLocationColumnFamily.NAME);
 +      scanner.setRange(metadataRange);
 +
 +      RowIterator rowIter = new RowIterator(scanner);
 +      KeyExtent lastExtent = null;
 +      while (rowIter.hasNext()) {
 +        Iterator<Map.Entry<Key,Value>> row = rowIter.next();
 +        String last = "";
 +        KeyExtent extent = null;
 +        String location = null;
 +
 +        while (row.hasNext()) {
 +          Map.Entry<Key,Value> entry = row.next();
 +          Key key = entry.getKey();
 +
 +          if (key.getColumnFamily().equals(MetadataSchema.TabletsSection.LastLocationColumnFamily.NAME)) {
 +            last = entry.getValue().toString();
 +          }
 +
 +          if (key.getColumnFamily().equals(MetadataSchema.TabletsSection.CurrentLocationColumnFamily.NAME)
 +              || key.getColumnFamily().equals(MetadataSchema.TabletsSection.FutureLocationColumnFamily.NAME)) {
 +            location = entry.getValue().toString();
 +          }
 +
 +          if (MetadataSchema.TabletsSection.TabletColumnFamily.PREV_ROW_COLUMN.hasColumns(key)) {
 +            extent = new KeyExtent(key.getRow(), entry.getValue());
 +          }
 +
 +        }
 +
 +        if (location != null)
 +          return null;
 +
 +        if (!extent.getTableId().toString().equals(tableId)) {
 +          throw new AccumuloException("Saw unexpected table Id " + tableId + " " + extent);
 +        }
 +
 +        if (lastExtent != null && !extent.isPreviousExtent(lastExtent)) {
 +          throw new AccumuloException(" " + lastExtent + " is not previous extent " + extent);
 +        }
 +
 +        Map<KeyExtent,List<Range>> tabletRanges = binnedRanges.get(last);
 +        if (tabletRanges == null) {
 +          tabletRanges = new HashMap<KeyExtent,List<Range>>();
 +          binnedRanges.put(last, tabletRanges);
 +        }
 +
 +        List<Range> rangeList = tabletRanges.get(extent);
 +        if (rangeList == null) {
 +          rangeList = new ArrayList<Range>();
 +          tabletRanges.put(extent, rangeList);
 +        }
 +
 +        rangeList.add(range);
 +
 +        if (extent.getEndRow() == null || range.afterEndKey(new Key(extent.getEndRow()).followingKey(PartialKey.ROW))) {
 +          break;
 +        }
 +
 +        lastExtent = extent;
 +      }
 +
 +    }
 +    return binnedRanges;
 +  }
 +}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/592c8b06/server/base/src/main/java/org/apache/accumulo/server/security/SecurityOperation.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/accumulo/blob/592c8b06/server/master/src/main/java/org/apache/accumulo/master/FateServiceHandler.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/accumulo/blob/592c8b06/server/master/src/main/java/org/apache/accumulo/master/tableOps/ImportTable.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/accumulo/blob/592c8b06/shell/src/main/java/org/apache/accumulo/shell/mock/MockShell.java
----------------------------------------------------------------------
diff --cc shell/src/main/java/org/apache/accumulo/shell/mock/MockShell.java
index 151648a,0000000..38bb501
mode 100644,000000..100644
--- a/shell/src/main/java/org/apache/accumulo/shell/mock/MockShell.java
+++ b/shell/src/main/java/org/apache/accumulo/shell/mock/MockShell.java
@@@ -1,143 -1,0 +1,182 @@@
 +/*
 + * 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.mock;
 +
 +import java.io.ByteArrayInputStream;
 +import java.io.IOException;
 +import java.io.InputStream;
 +import java.io.OutputStream;
 +import java.nio.charset.StandardCharsets;
++import java.io.Writer;
 +
++import org.apache.commons.io.output.WriterOutputStream;
++
++import org.apache.commons.cli.CommandLine;
 +import jline.console.ConsoleReader;
 +
 +import org.apache.accumulo.core.client.mock.MockInstance;
 +import org.apache.accumulo.shell.Shell;
 +import org.apache.accumulo.shell.ShellOptionsJC;
 +
 +/**
 + * An Accumulo Shell implementation that allows a developer to attach an InputStream and Writer to the Shell for testing purposes.
 + */
 +public class MockShell extends Shell {
 +  private static final String NEWLINE = "\n";
 +  
 +  protected InputStream in;
 +  protected OutputStream out;
-   
++
++  /**
++   * Will only be set if you use either the Writer constructor or the setWriter(Writer) method
++   * @deprecated since 1.6.0; use out
++   */
++  @Deprecated
++  protected Writer writer = null;
++
 +  public MockShell(InputStream in, OutputStream out) throws IOException {
 +    super();
 +    this.in = in;
 +    this.out = out;
++    // we presume they don't use the writer field unless they use the other constructor.
++  }
++
++  /**
++   * @deprecated since 1.6.0; use OutputStream version
++   */
++  @Deprecated
++  public MockShell(InputStream in, Writer out) throws IOException {
++    this(in, new WriterOutputStream(out, Constants.UTF8.name()));
++    this.writer = out;
 +  }
 +  
 +  public boolean config(String... args) {
 +    configError = super.config(args);
 +    
 +    // Update the ConsoleReader with the input and output "redirected"
 +    try {
 +      this.reader = new ConsoleReader(in, out);
 +    } catch (Exception e) {
 +      printException(e);
 +      configError = true;
 +    }
 +    
 +    // Don't need this for testing purposes
 +    this.reader.setHistoryEnabled(false);
 +    this.reader.setPaginationEnabled(false);
 +    
 +    // Make the parsing from the client easier;
 +    this.verbose = false;
 +    return configError;
 +  }
 +  
 +  @Override
 +  protected void setInstance(ShellOptionsJC options) {
 +    // We always want a MockInstance for this test
 +    instance = new MockInstance();
 +  }
++
++  /**
++   * @deprecated since 1.6.0; use ShellOptionsJC version
++   */
++  @Deprecated
++  protected void setInstance(CommandLine cl) {
++    // same result as in previous version
++    setInstance((ShellOptionsJC)null);
++  }
 +  
 +  public int start() throws IOException {
 +    if (configError)
 +      return 1;
 +    
 +    String input;
 +    if (isVerbose())
 +      printInfo();
 +    
 +    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) {
 +      if (hasExited())
 +        return exitCode;
 +      
 +      reader.setPrompt(getDefaultPrompt());
 +      input = reader.readLine();
 +      if (input == null) {
 +        reader.println();
 +        return exitCode;
 +      } // user canceled
 +      
 +      execCommand(input, false, false);
 +    }
 +  }
 +  
 +  /**
 +   * @param in
 +   *          the in to set
 +   */
 +  public void setConsoleInputStream(InputStream in) {
 +    this.in = in;
 +  }
 +  
 +  /**
 +   * @param out
 +   *          the output stream to set
 +   */
 +  public void setConsoleWriter(OutputStream out) {
 +    this.out = out;
 +  }
++
++  /**
++   * @deprecated since 1.6.0; use the OutputStream version
++   */
++  @Deprecated
++  public void setConsoleWriter(Writer out) {
++    setConsoleWriter(new WriterOutputStream(out, Constants.UTF8.name()));
++    this.writer = out;
++  }
 +  
 +  /**
 +   * Convenience method to create the byte-array to hand to the console
 +   * 
 +   * @param commands
 +   *          An array of commands to run
 +   * @return A byte[] input stream which can be handed to the console.
 +   */
 +  public static ByteArrayInputStream makeCommands(String... commands) {
 +    StringBuilder sb = new StringBuilder(commands.length * 8);
 +    
 +    for (String command : commands) {
 +      sb.append(command).append(NEWLINE);
 +    }
 +    
 +    return new ByteArrayInputStream(sb.toString().getBytes(StandardCharsets.UTF_8));
 +  }
 +}


[06/17] git commit: ACCUMULO-2734 Fix trivial javadoc bug

Posted by bu...@apache.org.
ACCUMULO-2734 Fix trivial javadoc bug


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

Branch: refs/heads/master
Commit: 83ef6b8433e25477993308cb5a2acc30d9c26246
Parents: 700bcf5
Author: Christopher Tubbs <ct...@apache.org>
Authored: Fri Apr 25 13:42:41 2014 -0400
Committer: Christopher Tubbs <ct...@apache.org>
Committed: Fri Apr 25 13:42:41 2014 -0400

----------------------------------------------------------------------
 .../accumulo/core/client/admin/TableOperations.java  | 15 +++++++--------
 1 file changed, 7 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/83ef6b84/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperations.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperations.java b/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperations.java
index 0823656..e06baae 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperations.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperations.java
@@ -134,16 +134,15 @@ public interface TableOperations {
    * Note that while the documentation for Text specifies that its bytestream should be UTF-8, the encoding is not enforced by operations that work with byte arrays.
    * <p>
    * For example, you can create 256 evenly-sliced splits via the following code sample even though the given byte sequences are not valid UTF-8.
+   *
    * <pre>
-   * {@code
-   *  TableOperations tableOps = connector.tableOperations();
-   *  TreeSet<Text> splits = new TreeSet<Text>();
-   *  for (int i = 0; i < 256; i++) {
-   *    byte[] bytes = { (byte) i };
-   *    splits.add(new Text(bytes));
-   *  }
-   *  tableOps.addSplits(TABLE_NAME, splits);
+   * TableOperations tableOps = connector.tableOperations();
+   * TreeSet&lt;Text&gt; splits = new TreeSet&lt;Text&gt;();
+   * for (int i = 0; i &lt; 256; i++) {
+   *   byte[] bytes = {(byte) i};
+   *   splits.add(new Text(bytes));
    * }
+   * tableOps.addSplits(TABLE_NAME, splits);
    * </pre>
    *
    * @param tableName


[12/17] git commit: ACCUMULO-2171 Update CHANGES for 1.6.0-RC4

Posted by bu...@apache.org.
ACCUMULO-2171 Update CHANGES for 1.6.0-RC4

  This task also removes some previously added issues which were not actually
  addressed in that version.


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

Branch: refs/heads/master
Commit: 448e757807816bc36b43fb622df60c1186a347f3
Parents: 9bdad26
Author: Christopher Tubbs <ct...@apache.org>
Authored: Fri Apr 25 15:22:07 2014 -0400
Committer: Christopher Tubbs <ct...@apache.org>
Committed: Fri Apr 25 15:49:31 2014 -0400

----------------------------------------------------------------------
 CHANGES | 73 ++++++++++++++++++++----------------------------------------
 1 file changed, 24 insertions(+), 49 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/448e7578/CHANGES
----------------------------------------------------------------------
diff --git a/CHANGES b/CHANGES
index 04d4b63..df5e11f 100644
--- a/CHANGES
+++ b/CHANGES
@@ -22,6 +22,7 @@ Release Notes - Accumulo - Version 1.6.0
     * [ACCUMULO-1029] - Remove unnecessary svn:ignore items
     * [ACCUMULO-1132] - Use a concrete implementation of AuthenticationToken for !SYSTEM user
     * [ACCUMULO-1218] - document the recovery from a failed zookeeper
+    * [ACCUMULO-1219] - document recovery when a file is missing
     * [ACCUMULO-1220] - document steps to recover when a WAL is corrupted or missing
     * [ACCUMULO-1270] - Require license checks on every build
     * [ACCUMULO-1367] - Instance name and ports in MiniAccumuloCluster should be configurable
@@ -30,7 +31,6 @@ Release Notes - Accumulo - Version 1.6.0
     * [ACCUMULO-1383] - Add javadoc for lexicoders
     * [ACCUMULO-1388] - Allow users to configure memory used by MiniAccumulo processes
     * [ACCUMULO-1390] - Document how to use single node Accumulo
-    * [ACCUMULO-1391] - Create a MACConfig object to make tweaking the MAC easier
     * [ACCUMULO-1409] - Make MiniAccumuloCluster support native maps
     * [ACCUMULO-1412] - Allow shell password to come from a variety of sources
     * [ACCUMULO-1431] - Clean-up/Simplify .gitignore
@@ -59,7 +59,6 @@ Release Notes - Accumulo - Version 1.6.0
     * [ACCUMULO-1627] - Add hashcode() and equals() to ConditionalMutation
     * [ACCUMULO-1654] - Bug in encryption-at-rest causes periodic IOExceptions
     * [ACCUMULO-1674] - Use Credentials to clean up internals of mapreduce configuration
-    * [ACCUMULO-1684] - WAL encryption beaks WAL
     * [ACCUMULO-1688] - Ensure ZooLock.watchParent is properly set in case of Connection loss
     * [ACCUMULO-1712] - Remove special characters from path names of metadata tables
     * [ACCUMULO-1713] - defer deleting _tmp files to recovery-likely scenarios
@@ -115,7 +114,6 @@ Release Notes - Accumulo - Version 1.6.0
     * [ACCUMULO-2438] - deleting all table rows caused tablet load failure
     * [ACCUMULO-2444] - Add unit tests to o.a.a.core.security
     * [ACCUMULO-2451] - updateAccumuloVersion only updates the data version on a single configured volume
-    * [ACCUMULO-2452] - DfsLogger.write(LogFileKey, LogFileValue) flushes with every call
     * [ACCUMULO-2456] - NamespacesIT.verifyIteratorInheritance
     * [ACCUMULO-2457] - SimpleProxyIT fails to find a scanner using {{getActiveScans}}
     * [ACCUMULO-2458] - BloomFilterIT fails to find improved performance
@@ -136,11 +134,16 @@ Release Notes - Accumulo - Version 1.6.0
     * [ACCUMULO-2547] - LargeRowIT.basicTest times out
     * [ACCUMULO-2551] - Add read-only access to Fate
     * [ACCUMULO-2555] - DeleterFormatterTest.testNoConfirm fails
+    * [ACCUMULO-2569] - ExamplesIT.testAgeoffFilter times out
     * [ACCUMULO-2584] - ShellServerIT.listscans fails
     * [ACCUMULO-2585] - WriteAheadLogIT.test times out
     * [ACCUMULO-2605] - TServerUtilsTest.testStopTServer_ES fails with class not found
     * [ACCUMULO-2620] - org.apache.accumulo.core.client.TableExistsException: Table run exists
     * [ACCUMULO-2680] - BackupMasterIT timing too tight
+    * [ACCUMULO-2715] - [FindBugs] Bad comparison of int with 0x7fffffffffffffffL
+    * [ACCUMULO-2718] - [FindBugs] Confusing both setup() and setUp() methods in AuditMessageIT
+    * [ACCUMULO-2720] - [FindBugs] HTTP response splitting vulnerabilities in the OperationServlet
+    * [ACCUMULO-2728] - [FindBugs] Bad random number generation in VolumeUtil
 
 ** Bug
     * [ACCUMULO-324] - System/site constraints and iterators should NOT affect the METADATA table
@@ -157,7 +160,6 @@ Release Notes - Accumulo - Version 1.6.0
     * [ACCUMULO-1326] - ShellTest.authsTests failing in trunk.
     * [ACCUMULO-1345] - Provide feedback that a compaction is "stuck"
     * [ACCUMULO-1356] - shell tables command does not paginate
-    * [ACCUMULO-1370] - tweak rat check excludes to work on a configured workspace
     * [ACCUMULO-1379] - PermGen leak
     * [ACCUMULO-1382] - Pair incorrectly assumes its components are Comparable
     * [ACCUMULO-1407] - Fix documentation for deleterows
@@ -196,7 +198,6 @@ Release Notes - Accumulo - Version 1.6.0
     * [ACCUMULO-1605] - stack overflow MilliSpan.java:113
     * [ACCUMULO-1616] - Incorrect use of getCanonicalName in BloomFilterLayer code (and others)
     * [ACCUMULO-1618] - No ability to disable trash for root tablet & WAL
-    * [ACCUMULO-1624] - Delete table running slow in clone integration test
     * [ACCUMULO-1626] - Mutation equals() method does not always work
     * [ACCUMULO-1629] - Files in cloned table disappear when source deleted
     * [ACCUMULO-1630] - Accumulo gc cannot identify as non localhost address
@@ -222,7 +223,6 @@ Release Notes - Accumulo - Version 1.6.0
     * [ACCUMULO-1734] - Mutation is hard to check in unit tests
     * [ACCUMULO-1735] - test ShutdownIT#stopDuringStart sometimes times out
     * [ACCUMULO-1743] - Seeing unexpected trace output
-    * [ACCUMULO-1745] - Merge broken on cloned tables
     * [ACCUMULO-1752] - MAC start() method should timeout when it can't connect to Zookeeper
     * [ACCUMULO-1774] - DevNull iterator documentation is incorrect
     * [ACCUMULO-1775] - zooCacheTest fails due to missing /tmp/zcTest-42
@@ -302,7 +302,6 @@ Release Notes - Accumulo - Version 1.6.0
     * [ACCUMULO-1994] - proxy does not handle Key timestamps correctly
     * [ACCUMULO-1997] - Perform validation on ACCUMULO_HOME
     * [ACCUMULO-1998] - Encrypted WALogs seem to be excessively buffering
-    * [ACCUMULO-2006] - MetadataSplitIT fails
     * [ACCUMULO-2009] - ExamplesIT fails with Hadoop 1.2.1
     * [ACCUMULO-2012] - Cloning the metadata table fails with faulty error
     * [ACCUMULO-2013] - Cloning tables results in errors from the master
@@ -316,7 +315,6 @@ Release Notes - Accumulo - Version 1.6.0
     * [ACCUMULO-2036] - Mappers are not running locally
     * [ACCUMULO-2037] - Tablets not assigned to last location
     * [ACCUMULO-2042] - Scalability test does not run
-    * [ACCUMULO-2045] - Accumulo fails to init when using viewfs://
     * [ACCUMULO-2047] - Failures when using two namenodes and viewfs://
     * [ACCUMULO-2054] - Concurrent random walk test fails
     * [ACCUMULO-2056] - NotSerializableException: com.google.common.net.HostAndPort
@@ -353,7 +351,6 @@ Release Notes - Accumulo - Version 1.6.0
     * [ACCUMULO-2127] - example accumulo-site.xmls put Hadoop 2 jars ahead of Accumulo jars on classpath
     * [ACCUMULO-2128] - Provide resource cleanup via static utility rather than Instance.close
     * [ACCUMULO-2134] - SimpleProxyIT fails on TableNotFoundException
-    * [ACCUMULO-2140] - Race conditions between client operations and upgrade
     * [ACCUMULO-2142] - GC doesn't advertise its port
     * [ACCUMULO-2146] - typo in continuous-env.sh.example
     * [ACCUMULO-2154] - NoNodeException error in master
@@ -373,9 +370,9 @@ Release Notes - Accumulo - Version 1.6.0
     * [ACCUMULO-2194] - Back-to-back Security randomwalk tests fail
     * [ACCUMULO-2196] - ConfigurableMacIT tests should each use their own dirs
     * [ACCUMULO-2198] - Concurrent randomwalk fails with unbalanced servers
-    * [ACCUMULO-2199] - FileNotFound exception attempting to run continuous ingest verification
     * [ACCUMULO-2202] - Bloom loader missleading ERROR
     * [ACCUMULO-2206] - close consistency check failure
+    * [ACCUMULO-2208] - Provide better error handling when unable to cleanly close log
     * [ACCUMULO-2209] - Rename DefaultKeyEncryptionStrategy to something more descriptive
     * [ACCUMULO-2210] - socket should be closed in MiniAccumuloClusterImpl#start()
     * [ACCUMULO-2211] - Security randomwalk fails "User doesn't exist and they SHOULD"
@@ -398,12 +395,12 @@ Release Notes - Accumulo - Version 1.6.0
     * [ACCUMULO-2266] - TServer should ensure wal settings are valid for underlying FS
     * [ACCUMULO-2270] - bulk import thread pool initialized in non-static method.
     * [ACCUMULO-2289] - Correct Tables.getNamespace to clarify namespace name or ID
+    * [ACCUMULO-2293] - AccumuloSecurityException might be thrown instead of TableNotFoundException on flush or clone
     * [ACCUMULO-2295] - Various ITs timing too sensitive
     * [ACCUMULO-2297] - Additional unnecessary member on InputFormatBase
     * [ACCUMULO-2298] - ExamplesIT should be chunked into several smaller tests
     * [ACCUMULO-2306] - SslWithClientAuthIT.bulk consistently fails in parallel
     * [ACCUMULO-2312] - NPE running Concurrent randomwalk
-    * [ACCUMULO-2315] - Concurrent random walk test needs better handling of Namespace/table not found
     * [ACCUMULO-2316] - Table based Security operations should throw Table not found exceptions over namespace not found
     * [ACCUMULO-2318] - Renaming table don't require namespace
     * [ACCUMULO-2319] - TabletServerResourceManager MemoryManagementFramework constructor starts threads
@@ -449,6 +446,7 @@ Release Notes - Accumulo - Version 1.6.0
     * [ACCUMULO-2515] - NamespaceConfiguration doesn't invalidateCache
     * [ACCUMULO-2519] - FATE operation failed across upgrade
     * [ACCUMULO-2520] - Garbage collector deleted everything when given bad input
+    * [ACCUMULO-2523] - TabletTime.maxMetadataTime NPE if both arguments are null
     * [ACCUMULO-2524] - ProblemReport needs to check for nulls on construction
     * [ACCUMULO-2526] - Top level files in tarball should be 644, not 755
     * [ACCUMULO-2539] - SummingCombiner does not adhere to Combiner#COLUMNS_OPTION
@@ -471,6 +469,14 @@ Release Notes - Accumulo - Version 1.6.0
     * [ACCUMULO-2690] - DfsLogger outputstream short circuiting is never satisfied.
     * [ACCUMULO-2695] - Failure while running conditional RW test w/ agitation
     * [ACCUMULO-2700] - SecurityOperation.authenticateSystemUser fails to properly validate system user
+    * [ACCUMULO-2713] - Instance secret written out with other configuration items to RFiles and WALogs when encryption is turned on
+    * [ACCUMULO-2714] - Integration test classpath issues with Guava
+    * [ACCUMULO-2716] - Duplicate connection loss logging in Writer
+    * [ACCUMULO-2717] - multiTableTest is timing out
+    * [ACCUMULO-2723] - orphaned monitor test
+    * [ACCUMULO-2729] - Conditional RW test not checking status
+    * [ACCUMULO-2730] - assigned tablets in DELETING state are not deleted
+    * [ACCUMULO-2733] - Credentials is missing deserializer to match Thrift serializer
 
 ** Improvement
     * [ACCUMULO-112] - Investigate partitioning in memory map by locality group
@@ -497,7 +503,6 @@ Release Notes - Accumulo - Version 1.6.0
     * [ACCUMULO-1313] - Use annotations to filter/hide sensitive configuration properties when logging
     * [ACCUMULO-1335] - DU command should be available through the java API
     * [ACCUMULO-1347] - Accumulo Shell Does Not support comments
-    * [ACCUMULO-1349] - Enable tests with a suffix of IT to run in a profile
     * [ACCUMULO-1357] - Make debugging MiniAccumuloCluster easier
     * [ACCUMULO-1371] - Improve message in config.sh when HADOOP_PREFIX is not a directory.
     * [ACCUMULO-1375] - Update README files in proxy module.
@@ -586,8 +591,6 @@ Release Notes - Accumulo - Version 1.6.0
     * [ACCUMULO-2454] - Aggregate Cobertura reports
     * [ACCUMULO-2482] - Improve error message on ZK retries
     * [ACCUMULO-2499] - User manual should include list of default ports
-    * [ACCUMULO-2521] - Expose FATE admin utility by name under bin/accumulo
-    * [ACCUMULO-2528] - OfflineScanner is not volume-aware
     * [ACCUMULO-2529] - bake desired warnings into normal maven build
     * [ACCUMULO-2592] - Add new interfaces that better allow extension/implementations of Accumulo clusters
 
@@ -642,8 +645,13 @@ Release Notes - Accumulo - Version 1.6.0
     * [ACCUMULO-2616] - Bump plugin versions and parent POM version
     * [ACCUMULO-2652] - Build warnings from missing plugin version
     * [ACCUMULO-2653] - Remove useless javadocs
+    * [ACCUMULO-2654] - means to create empty rfile for recovery
     * [ACCUMULO-2670] - Corrupt license header in shelltest.txt
     * [ACCUMULO-2697] - Automatically run findbugs to detect serious issues in the build
+    * [ACCUMULO-2707] - Remove build-helper task: regex-property from build
+    * [ACCUMULO-2708] - Lower the rank threshold for findbugs execution in the build
+    * [ACCUMULO-2726] - Maintain binary backwards compat for non-deprecated API components for 1.5 -> 1.6
+    * [ACCUMULO-2734] - javadoc bug with multi-line @code tag
 
 ** Test
     * [ACCUMULO-1981] - BloomFilterLayerLookupTest does not run
@@ -652,12 +660,14 @@ Release Notes - Accumulo - Version 1.6.0
     * [ACCUMULO-2392] - SimpleMacIT should document which MAC it is using
     * [ACCUMULO-2470] - Improve unit test coverage for server base classes
     * [ACCUMULO-2488] - Concurrent randomwalk balance check needs refinement
+    * [ACCUMULO-2498] - Improve unit tests coverage for core module
     * [ACCUMULO-2508] - Security randomwalk fails "BAD_CREDENTIALS - Username or Password is Invalid"
     * [ACCUMULO-2543] - cacheBlock should be closed in BlockIndexTest
     * [ACCUMULO-2558] - Improve unit test coverage for server gc classes
     * [ACCUMULO-2621] - Masters not restarting during concurrent randomwalk
     * [ACCUMULO-2657] - Shard randomwalk fails for 1.6.0
     * [ACCUMULO-2666] - provide a scaling factor for timeouts on ITs
+    * [ACCUMULO-2735] - Add specific test to ensure merging cloned tables are functional
 
 Release Notes - Accumulo - Version 1.5.0
 
@@ -705,22 +715,18 @@ Release Notes - Accumulo - Version 1.5.0
     * [ACCUMULO-993] - Create unit tests for mapred package
     * [ACCUMULO-996] - explore exposing accumulo token in proxy
     * [ACCUMULO-1001] - Fix problems with TransformingIterator
-    * [ACCUMULO-1003] - what is the point of SecuritySerDe?
     * [ACCUMULO-1004] - functional tests are failing
     * [ACCUMULO-1007] - Ruby example for proxy is busted
     * [ACCUMULO-1008] - Lift gen-{cpp,java,py,rb} from proxy's build.dir
-    * [ACCUMULO-1019] - Continuous ingest README wrong after package move
     * [ACCUMULO-1024] - Deprecate built-in user management utilities
     * [ACCUMULO-1025] - Don't deprecate AuthInfo
     * [ACCUMULO-1026] - Don't use @Deprecated to mean "Not for Client Use"
-    * [ACCUMULO-1027] - Consolidate the thrift objects for Authentication information
     * [ACCUMULO-1028] - Distinguish the user principal from the authentication token
     * [ACCUMULO-1041] - Generic interface for arbitrary token handling
     * [ACCUMULO-1047] - test 1.5.0
     * [ACCUMULO-1061] - Shutdown thread commented out
     * [ACCUMULO-1073] - Update website for 1.5.0 release
     * [ACCUMULO-1074] - Can not run as multiple users
-    * [ACCUMULO-1080] - MAster is not seeing tablet server go away
     * [ACCUMULO-1087] - master doesn't retry log recovery
     * [ACCUMULO-1089] - Missing word in Accumulo documentation
     * [ACCUMULO-1090] - Inconsistent text in example in Accumulo documentation
@@ -749,7 +755,6 @@ Release Notes - Accumulo - Version 1.5.0
     * [ACCUMULO-1185] - document running accumulo from HDFS
     * [ACCUMULO-1187] - review user and developer manuals for old information
     * [ACCUMULO-1194] - re-document deprecated getConnector() API's
-    * [ACCUMULO-1241] - Update ASF copyrights?
     * [ACCUMULO-1247] - Make master handle tablet with multiple locations
     * [ACCUMULO-1248] - Metadata cache should handle tablets with multiple locations
     * [ACCUMULO-1271] - Make apache-release profile work for us
@@ -778,7 +783,6 @@ Release Notes - Accumulo - Version 1.5.0
     * [ACCUMULO-108] - simple.gc.GCLotsOfCandidatesTest does not use the TestUtils settings properly
     * [ACCUMULO-160] - Makefile needs play nice on 64 and 32 bit systems
     * [ACCUMULO-189] - RegExFilter deepCopy NullPointerException
-    * [ACCUMULO-198] - Can't build RPM
     * [ACCUMULO-209] - RegExFilter does not properly regex when using multi-byte characters
     * [ACCUMULO-226] - BatchScanner iterator implementation erroneously returns true for hasNext upon subsequent hasNext calls
     * [ACCUMULO-228] - Default map does not accurately count bytes in memory
@@ -789,7 +793,6 @@ Release Notes - Accumulo - Version 1.5.0
     * [ACCUMULO-333] - Shell 'help' should wrap more cleanly
     * [ACCUMULO-360] - Bulk importing Keys with invalid ColumnVisibility doesn't fail until scan
     * [ACCUMULO-369] - master retries failed bulk imports too quickly
-    * [ACCUMULO-392] - Accumulo does not start up nicely in provisioned hadoop installations
     * [ACCUMULO-396] - Shell in mock mode doesn't use instance name
     * [ACCUMULO-398] - Table tablets not evenly spread.
     * [ACCUMULO-408] - Large bulk imports timing out
@@ -810,7 +813,6 @@ Release Notes - Accumulo - Version 1.5.0
     * [ACCUMULO-533] - make system iterators thread-safer
     * [ACCUMULO-534] - simple.logicalTime.LogicalTimeTest is timing out
     * [ACCUMULO-535] - shell functional test is failing
-    * [ACCUMULO-536] - bulk split optimization test is failing
     * [ACCUMULO-537] - SimpleBalancerFairness functional test is failing
     * [ACCUMULO-538] - examples functional test is failing
     * [ACCUMULO-539] - fix warning about deprecated md5 module when running functional tests
@@ -822,14 +824,12 @@ Release Notes - Accumulo - Version 1.5.0
     * [ACCUMULO-566] - monitor should display zero tablet servers as red, even if there's only one tablet server configured
     * [ACCUMULO-568] - Test creates a core/null directory
     * [ACCUMULO-570] - dirlist example README is a little inconsistent
-    * [ACCUMULO-575] - Potential data loss when datanode fails immediately after minor compaction
     * [ACCUMULO-589] - Unable to set system properties in shell
     * [ACCUMULO-590] - NPE in TableLoadBalancer
     * [ACCUMULO-591] - Need to clear tablet location cache before computing input splits in input format
     * [ACCUMULO-596] - Shell doesn't accept hex characters with nonzero first bit
     * [ACCUMULO-597] - need timezone label to clarify the times on the monitoring graphs
     * [ACCUMULO-601] - client fails to resolve master hostname when not using fully qualified domain name
-    * [ACCUMULO-604] - Running Low On Memory messages in monitor is a bit overzealous
     * [ACCUMULO-605] - NPE on MasterMonitorInfo also spammy
     * [ACCUMULO-606] - Shell needs add aggregator ability restored
     * [ACCUMULO-616] - Table setting table.scan.max.memory ignored
@@ -868,7 +868,6 @@ Release Notes - Accumulo - Version 1.5.0
     * [ACCUMULO-701] - No longer seeing TApplicationException
     * [ACCUMULO-702] - build on ubuntu hangs without required dependencies
     * [ACCUMULO-709] - AddFilesWithMissingEntries adds tmp files
-    * [ACCUMULO-721] - Reloading of lib/ext classloader causes LinkageError
     * [ACCUMULO-727] - Bulk Import retry time needs to be longer/configurable
     * [ACCUMULO-733] - add description of the BatchWriter behavior
     * [ACCUMULO-737] - The PrintInfo.printMetaBlockInfo method does not close a BCFile.Reader
@@ -888,7 +887,6 @@ Release Notes - Accumulo - Version 1.5.0
     * [ACCUMULO-779] - ZooStore can starve completed Repos
     * [ACCUMULO-781] - Move README_UBUNTU to docs directory.
     * [ACCUMULO-782] - MockConnector returns a MockInstance that is not equivalent to it's parent MockInstance
-    * [ACCUMULO-783] - start-agitator.sh starts both agitator and magitator but stop-agitator.sh only stops agitator
     * [ACCUMULO-784] - agitator maxkill/minkill issue
     * [ACCUMULO-785] - Need age off finished fate operations
     * [ACCUMULO-787] - Debian packages rely on old hadoop-zookeeper package
@@ -905,7 +903,6 @@ Release Notes - Accumulo - Version 1.5.0
     * [ACCUMULO-827] - CloudStone Code Has Issues
     * [ACCUMULO-832] - Create directory for debug logs
     * [ACCUMULO-834] - thrift server responds slowly to large numbers of requests
-    * [ACCUMULO-837] - NetUtils.getInputStream() yeilds NoSuchMethodError
     * [ACCUMULO-843] - Mock does not implement locality groups or merging
     * [ACCUMULO-844] - VisibilityFilter does not catch BadArgumentException
     * [ACCUMULO-845] - Handle method.include calls consistently in TraceWrap class.
@@ -937,9 +934,7 @@ Release Notes - Accumulo - Version 1.5.0
     * [ACCUMULO-926] - Extraneous vfs output should be logged (and breaks functional tests)
     * [ACCUMULO-928] - The TableLoadBalancer.getTableOperations method should not rely on Zookeeper.
     * [ACCUMULO-929] - Ignore JetBrains .idea and .iml files
-    * [ACCUMULO-939] - Writing to HDFS gets stuck
     * [ACCUMULO-940] - Generated code needs license headers
-    * [ACCUMULO-943] - Continuous Ingest graphs are broken by VFS extra output
     * [ACCUMULO-944] - start-here doesn't work if you define your roles by IP
     * [ACCUMULO-954] - ZooLock watcher can stop watching
     * [ACCUMULO-965] - Zookeeper session ids created as unsigned long, parsed in ZooUtils.java as signed long
@@ -980,12 +975,10 @@ Release Notes - Accumulo - Version 1.5.0
     * [ACCUMULO-1086] - Configuration secrets exposed via thrift RPC with no authentication
     * [ACCUMULO-1088] - master not balancing because balance information is out-of-date
     * [ACCUMULO-1104] - improve documentation
-    * [ACCUMULO-1106] - MiniAccumuloTest fails on hadoop 2.0
     * [ACCUMULO-1107] - Rpm fails to install on Centos 6.3 x86_64 using yum
     * [ACCUMULO-1110] - deadlock found in major compaction
     * [ACCUMULO-1111] - stack overflow in random walk test
     * [ACCUMULO-1112] - master state is UNLOAD_METADATA_TABLE and goal state is NORMAL: stuck
-    * [ACCUMULO-1113] - consistency check failure on !METADATA table
     * [ACCUMULO-1114] - class cast exception during Security random walk
     * [ACCUMULO-1115] - Config node in concurrent random walk doesn't work for tables
     * [ACCUMULO-1117] - noisy logs during random walk
@@ -1028,7 +1021,6 @@ Release Notes - Accumulo - Version 1.5.0
     * [ACCUMULO-1214] - Proxy SimpleTest fails reading stale zookeeper data
     * [ACCUMULO-1227] - simple.largeRow.LargeRowTest FAILED due to out of range exception
     * [ACCUMULO-1231] - running FindOfflineTablets with no arguments throws NPE
-    * [ACCUMULO-1232] - table_info tablet offline for a long time
     * [ACCUMULO-1233] - race condition looking at zookeeper nodes
     * [ACCUMULO-1234] - setscaniter can cause iterator class warning to appear in monitor
     * [ACCUMULO-1235] - tablet fails to load after split fix
@@ -1040,14 +1032,12 @@ Release Notes - Accumulo - Version 1.5.0
     * [ACCUMULO-1250] - Fail the build when thrift profile activated and thrift fails
     * [ACCUMULO-1251] - TFramedTransport frame size limited on client side
     * [ACCUMULO-1253] - docs/config.html generation broken
-    * [ACCUMULO-1255] - Occasional errors in vfs2 unit tests
     * [ACCUMULO-1257] - strange updates to metadata table
     * [ACCUMULO-1261] - System tests fail from clean checkout
     * [ACCUMULO-1264] - !METADATA table merge hangs
     * [ACCUMULO-1265] - copy dependencies breaks "mvn compile"
     * [ACCUMULO-1276] - problems running generate_monitor_certificate.sh
     * [ACCUMULO-1277] - Race condition between master and tserver when acquiring tserver lock
-    * [ACCUMULO-1279] - Accumulo example configs run out of heap space
     * [ACCUMULO-1282] - Monitor requires jumping through hadoop permissions hoops (and granting accumulo broad permissions)
     * [ACCUMULO-1283] - new configuration test in random walk sets timeout values much too high
     * [ACCUMULO-1284] - FindOfflineTablets uses a batch scanner over the ROOT and META tablet types
@@ -1062,7 +1052,6 @@ Release Notes - Accumulo - Version 1.5.0
     * [ACCUMULO-1348] - Accumulo Shell does not respect 'exit' when executing file
     * [ACCUMULO-1358] - Shell's setiter is not informative when using a bad class name
     * [ACCUMULO-1365] - proxy SimpleTest fails trying to get a port
-    * [ACCUMULO-1374] - Sudden Death of master, gc, and tservers
     * [ACCUMULO-1384] - bin/config.sh has hard-coded version that isn't updated with the build
     * [ACCUMULO-1389] - RPM has dependency on "jdk", it should be "jre"
     * [ACCUMULO-1392] - kerberos directions in README
@@ -1094,8 +1083,6 @@ Release Notes - Accumulo - Version 1.5.0
     * [ACCUMULO-161] - Add ability to maven to build in memory map code
     * [ACCUMULO-173] - Integrate SVN build number into packaging
     * [ACCUMULO-174] - Create maven goal/profile for documentation
-    * [ACCUMULO-187] - Maven goal/profile for javadocs
-    * [ACCUMULO-190] - Cleanup pom version numbers
     * [ACCUMULO-227] - Improve in memory map counts to provide cell level uniqueness for repeated columns in  mutation
     * [ACCUMULO-241] - Visibility labels should allow more characters
     * [ACCUMULO-259] - ZKAuthentictor needs to be split up
@@ -1106,7 +1093,6 @@ Release Notes - Accumulo - Version 1.5.0
     * [ACCUMULO-339] - Add lib and target directories to .gitignore file.
     * [ACCUMULO-348] - Adding splits to table via the shell with addsplits is very slow when adding a lot of split points
     * [ACCUMULO-350] - Support loading (reloading?) jars out of hdfs
-    * [ACCUMULO-358] - Create examples for trial users
     * [ACCUMULO-362] - Integrate automatic CHANGES into build
     * [ACCUMULO-376] - Shell command list needs orginization
     * [ACCUMULO-377] - Shell scan command needs support for multiple columns
@@ -1120,7 +1106,6 @@ Release Notes - Accumulo - Version 1.5.0
     * [ACCUMULO-442] - Please add comments to src/trace/thift.sh
     * [ACCUMULO-451] - Change byte val[] to byte[] val in Mutation.java per PMD suggestion.
     * [ACCUMULO-453] - Profile and optimize server activity visualization
-    * [ACCUMULO-455] - Reintroduce support for running external jars on the accumulo classpath
     * [ACCUMULO-467] - Allow the properties for AccumuloFileOutputFormat to be set in a mapreduce job
     * [ACCUMULO-473] - Support binary search within RFile blocks
     * [ACCUMULO-498] - Key.getColumnVisibility() should return a ColumnVisibility, not a Text
@@ -1226,7 +1211,6 @@ Release Notes - Accumulo - Version 1.5.0
     * [ACCUMULO-19] - Debian packaging support
     * [ACCUMULO-49] - optionally monitor swappiness on every server
     * [ACCUMULO-196] - Add command and control features to the Monitor
-    * [ACCUMULO-347] - some more features should be added to screen
     * [ACCUMULO-404] - Support running on-top of Kerberos-enabled HDFS
     * [ACCUMULO-420] - Allow per compaction iterator settings
     * [ACCUMULO-456] - Need utility for exporting and importing tables
@@ -1249,7 +1233,6 @@ Release Notes - Accumulo - Version 1.5.0
     * [ACCUMULO-152] - Purge old utils
     * [ACCUMULO-242] - Make appropriate references to Apache Accumulo
     * [ACCUMULO-288] - Remove MyMapFile
-    * [ACCUMULO-323] - Double check constraint permissions
     * [ACCUMULO-353] - "mvn" should not build tgz
     * [ACCUMULO-438] - Rename cloudtrace package to org.apache.accumulo.cloudtrace
     * [ACCUMULO-457] - make mvn site useful
@@ -1272,10 +1255,7 @@ Release Notes - Accumulo - Version 1.5.0
     * [ACCUMULO-755] - Run findbugs for 1.5
     * [ACCUMULO-786] - Look into walog space usage
     * [ACCUMULO-796] - purge MLock
-    * [ACCUMULO-883] - evaluate different thrift components for performance
-    * [ACCUMULO-946] - Accumulo web page should list contributor organizations
     * [ACCUMULO-951] - Remove obsolete branches in subversion
-    * [ACCUMULO-953] - Deprecate the "setiter" and replace with "setiterator"
     * [ACCUMULO-957] - Modify codestyle formatter to allow turning off the formatter
     * [ACCUMULO-960] - We don't need logger options anymore
     * [ACCUMULO-970] - Release 1.5.0
@@ -1370,7 +1350,6 @@ Release Notes - Accumulo - Version 1.4.0
     * [ACCUMULO-299] - tracing not working
     * [ACCUMULO-300] - monitor page warns if the number of tablets goes into the hundreds of thousands: this is no longer a significant limitation to scalability
     * [ACCUMULO-301] - TApplicationException running example in README.shard
-    * [ACCUMULO-304] - shell not connecting
     * [ACCUMULO-308] - Isolation example is broken
     * [ACCUMULO-310] - AccumuloInputFormat and AccumuloOutputFormat configuration methods don't match
     * [ACCUMULO-314] - Re-queue tablets immediately after major compaction if there is more work
@@ -1392,7 +1371,6 @@ Release Notes - Accumulo - Version 1.4.0
     * [ACCUMULO-368] - tablet had location but was not loaded
     * [ACCUMULO-373] - file missing during a major compaction
     * [ACCUMULO-374] - wikisearch-ingest stop list should be removed
-    * [ACCUMULO-380] - UnsupportedOperation exception on wikisearch example
     * [ACCUMULO-393] - Master not balancing after agitation
     * [ACCUMULO-395] - Map reduce reading from Accumulo is not running mapper locally
     * [ACCUMULO-405] - References to accumulo-examples still exist
@@ -1427,7 +1405,6 @@ Release Notes - Accumulo - Version 1.4.0
     * [ACCUMULO-50] - Add Security Randomwalk to All Randomwalk test
     * [ACCUMULO-67] - zookeeper session id encoded in metadata in two different ways
     * [ACCUMULO-68] - Document new 1.4 features in user manual
-    * [ACCUMULO-69] - Document how Accumulo uses Zookeeper and HDFS
     * [ACCUMULO-87] - Accumulo needs a logo
     * [ACCUMULO-90] - add ":" to the legal character making up the term in a visibility expression
     * [ACCUMULO-101] - Include tserver.memory.maps.max in the example xml file
@@ -1439,7 +1416,6 @@ Release Notes - Accumulo - Version 1.4.0
     * [ACCUMULO-139] - Add Snappy support
     * [ACCUMULO-153] - Iterator options for input formats can't contain certain characters
     * [ACCUMULO-154] - Combiner configuration is confusing
-    * [ACCUMULO-156] - Refactor Trie
     * [ACCUMULO-162] - TimestampFilter requires both start and end timestamps
     * [ACCUMULO-167] - Add static configuration methods to all user iterators
     * [ACCUMULO-188] - Need functional test to test reloading iterators
@@ -1449,7 +1425,6 @@ Release Notes - Accumulo - Version 1.4.0
     * [ACCUMULO-248] - In user manual, please define 'Mutation' before using it.
     * [ACCUMULO-251] - Add wording to README.bloom about reason for flushing.
     * [ACCUMULO-252] - Improve start-up scripts to avoid common errors
-    * [ACCUMULO-256] - Monitor page needs uptime
     * [ACCUMULO-265] - Fix iterator priority conflict in README.combiner
     * [ACCUMULO-273] - Report stuck random walk test
     * [ACCUMULO-274] - Add descriptions to README.filedata


[15/17] ACCUMULO-2726 cleanup for master's branch.

Posted by bu...@apache.org.
http://git-wip-us.apache.org/repos/asf/accumulo/blob/f277fea9/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperationsImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperationsImpl.java b/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperationsImpl.java
deleted file mode 100644
index 95ba3d8..0000000
--- a/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperationsImpl.java
+++ /dev/null
@@ -1,1581 +0,0 @@
-/*
- * 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.core.client.admin;
-
-import static com.google.common.base.Preconditions.checkArgument;
-
-import java.io.BufferedReader;
-import java.io.IOException;
-import java.io.InputStreamReader;
-import java.nio.ByteBuffer;
-import java.nio.charset.StandardCharsets;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.EnumSet;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Random;
-import java.util.Set;
-import java.util.SortedSet;
-import java.util.TreeMap;
-import java.util.TreeSet;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicReference;
-import java.util.zip.ZipEntry;
-import java.util.zip.ZipInputStream;
-
-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.Instance;
-import org.apache.accumulo.core.security.Credentials;
-import org.apache.accumulo.core.tabletserver.thrift.NotServingTabletException;
-import org.apache.accumulo.core.tabletserver.thrift.TabletClientService;
-import org.apache.accumulo.core.util.ByteBufferUtil;
-import org.apache.accumulo.core.util.CachedConfiguration;
-import org.apache.accumulo.core.util.LocalityGroupUtil;
-import org.apache.accumulo.core.util.MapCounter;
-import org.apache.accumulo.core.util.NamingThreadFactory;
-import org.apache.accumulo.core.util.OpTimer;
-import org.apache.accumulo.core.util.Pair;
-import org.apache.accumulo.core.util.TextUtil;
-import org.apache.accumulo.core.util.ThriftUtil;
-import org.apache.accumulo.core.util.UtilWaitThread;
-import org.apache.accumulo.core.volume.VolumeConfiguration;
-import org.apache.accumulo.trace.instrument.Tracer;
-import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.Text;
-import org.apache.log4j.Level;
-import org.apache.log4j.Logger;
-import org.apache.thrift.TApplicationException;
-import org.apache.thrift.TException;
-import org.apache.thrift.transport.TTransportException;
-import org.apache.accumulo.core.security.thrift.TCredentials;
-
-import com.google.common.base.Joiner;
-
-public class TableOperationsImpl extends TableOperationsHelper {
-  private Instance instance;
-  private Credentials credentials;
-
-  public static final String CLONE_EXCLUDE_PREFIX = "!";
-
-  private static final Logger log = Logger.getLogger(TableOperations.class);
-
-/**
- * @deprecated since 1.6.0; not intended for public api and you should not use it.
- */
-@Deprecated
-public class TableOperationsImpl extends org.apache.accumulo.core.client.impl.TableOperationsImpl {
-  /**
-   * @param instance
-   *          the connection information for this instance
-   * @param credentials
-   *          the username/password for this connection
-   */
-  public TableOperationsImpl(Instance instance, Credentials credentials) {
-    checkArgument(instance != null, "instance is null");
-    checkArgument(credentials != null, "credentials is null");
-    this.instance = instance;
-    this.credentials = credentials;
-  }
-
-  /**
-   * Retrieve a list of tables in Accumulo.
-   * 
-   * @return List of tables in accumulo
-   */
-  @Override
-  public SortedSet<String> list() {
-    OpTimer opTimer = new OpTimer(log, Level.TRACE).start("Fetching list of tables...");
-    TreeSet<String> tableNames = new TreeSet<String>(Tables.getNameToIdMap(instance).keySet());
-    opTimer.stop("Fetched " + tableNames.size() + " table names in %DURATION%");
-    return tableNames;
-  }
-
-  /**
-   * A method to check if a table exists in Accumulo.
-   * 
-   * @param tableName
-   *          the name of the table
-   * @return true if the table exists
-   */
-  @Override
-  public boolean exists(String tableName) {
-    checkArgument(tableName != null, "tableName is null");
-    if (tableName.equals(MetadataTable.NAME) || tableName.equals(RootTable.NAME))
-      return true;
-
-    OpTimer opTimer = new OpTimer(log, Level.TRACE).start("Checking if table " + tableName + " exists...");
-    boolean exists = Tables.getNameToIdMap(instance).containsKey(tableName);
-    opTimer.stop("Checked existance of " + exists + " in %DURATION%");
-    return exists;
-  }
-
-  /**
-   * Create a table with no special configuration
-   * 
-   * @param tableName
-   *          the name of the table
-   * @throws AccumuloException
-   *           if a general error occurs
-   * @throws AccumuloSecurityException
-   *           if the user does not have permission
-   * @throws TableExistsException
-   *           if the table already exists
-   */
-  @Override
-  public void create(String tableName) throws AccumuloException, AccumuloSecurityException, TableExistsException {
-    create(tableName, true, TimeType.MILLIS);
-  }
-
-  /**
-   * @param tableName
-   *          the name of the table
-   * @param limitVersion
-   *          Enables/disables the versioning iterator, which will limit the number of Key versions kept.
-   */
-  @Override
-  public void create(String tableName, boolean limitVersion) throws AccumuloException, AccumuloSecurityException, TableExistsException {
-    create(tableName, limitVersion, TimeType.MILLIS);
-  }
-
-  /**
-   * @param tableName
-   *          the name of the table
-   * @param timeType
-   *          specifies logical or real-time based time recording for entries in the table
-   * @param limitVersion
-   *          Enables/disables the versioning iterator, which will limit the number of Key versions kept.
-   */
-  @Override
-  public void create(String tableName, boolean limitVersion, TimeType timeType) throws AccumuloException, AccumuloSecurityException, TableExistsException {
-    checkArgument(tableName != null, "tableName is null");
-    checkArgument(timeType != null, "timeType is null");
-
-    List<ByteBuffer> args = Arrays.asList(ByteBuffer.wrap(tableName.getBytes(StandardCharsets.UTF_8)), ByteBuffer.wrap(timeType.name().getBytes(StandardCharsets.UTF_8)));
-
-    Map<String,String> opts;
-    if (limitVersion)
-      opts = IteratorUtil.generateInitialTableProperties(limitVersion);
-    else
-      opts = Collections.emptyMap();
-
-    try {
-      doTableFateOperation(tableName, AccumuloException.class, FateOperation.TABLE_CREATE, args, opts);
-    } catch (TableNotFoundException e) {
-      // should not happen
-      throw new AssertionError(e);
-    }
-  }
-
-  private long beginFateOperation() throws ThriftSecurityException, TException {
-    while (true) {
-      MasterClientService.Iface client = null;
-      try {
-        client = MasterClient.getConnectionWithRetry(instance);
-        return client.beginFateOperation(Tracer.traceInfo(), credentials.toThrift(instance));
-      } catch (TTransportException tte) {
-        log.debug("Failed to call beginFateOperation(), retrying ... ", tte);
-        UtilWaitThread.sleep(100);
-      } finally {
-        MasterClient.close(client);
-      }
-    }
-  }
-
-  // This method is for retrying in the case of network failures; anything else it passes to the caller to deal with
-  private void executeFateOperation(long opid, FateOperation op, List<ByteBuffer> args, Map<String,String> opts, boolean autoCleanUp)
-      throws ThriftSecurityException, TException, ThriftTableOperationException {
-    while (true) {
-      MasterClientService.Iface client = null;
-      try {
-        client = MasterClient.getConnectionWithRetry(instance);
-        client.executeFateOperation(Tracer.traceInfo(), credentials.toThrift(instance), opid, op, args, opts, autoCleanUp);
-        break;
-      } catch (TTransportException tte) {
-        log.debug("Failed to call executeFateOperation(), retrying ... ", tte);
-        UtilWaitThread.sleep(100);
-      } finally {
-        MasterClient.close(client);
-      }
-    }
-  }
-
-  private String waitForFateOperation(long opid) throws ThriftSecurityException, TException, ThriftTableOperationException {
-    while (true) {
-      MasterClientService.Iface client = null;
-      try {
-        client = MasterClient.getConnectionWithRetry(instance);
-        return client.waitForFateOperation(Tracer.traceInfo(), credentials.toThrift(instance), opid);
-      } catch (TTransportException tte) {
-        log.debug("Failed to call waitForFateOperation(), retrying ... ", tte);
-        UtilWaitThread.sleep(100);
-      } finally {
-        MasterClient.close(client);
-      }
-    }
-  }
-
-  private void finishFateOperation(long opid) throws ThriftSecurityException, TException {
-    while (true) {
-      MasterClientService.Iface client = null;
-      try {
-        client = MasterClient.getConnectionWithRetry(instance);
-        client.finishFateOperation(Tracer.traceInfo(), credentials.toThrift(instance), opid);
-        break;
-      } catch (TTransportException tte) {
-        log.debug("Failed to call finishFateOperation(), retrying ... ", tte);
-        UtilWaitThread.sleep(100);
-      } finally {
-        MasterClient.close(client);
-      }
-    }
-  }
-
-  String doFateOperation(FateOperation op, List<ByteBuffer> args, Map<String,String> opts) throws AccumuloSecurityException, TableExistsException,
-      TableNotFoundException, AccumuloException, NamespaceExistsException, NamespaceNotFoundException {
-    return doFateOperation(op, args, opts, true);
-  }
-
-  String doFateOperation(FateOperation op, List<ByteBuffer> args, Map<String,String> opts, boolean wait) throws AccumuloSecurityException,
-      TableExistsException, TableNotFoundException, AccumuloException, NamespaceExistsException, NamespaceNotFoundException {
-    Long opid = null;
-
-    try {
-      opid = beginFateOperation();
-      executeFateOperation(opid, op, args, opts, !wait);
-      if (!wait) {
-        opid = null;
-        return null;
-      }
-      String ret = waitForFateOperation(opid);
-      return ret;
-    } catch (ThriftSecurityException e) {
-      String tableName = ByteBufferUtil.toString(args.get(0));
-      switch (e.getCode()) {
-        case TABLE_DOESNT_EXIST:
-          throw new TableNotFoundException(null, tableName, "Target table does not exist");
-        case NAMESPACE_DOESNT_EXIST:
-          throw new NamespaceNotFoundException(null, tableName, "Target namespace does not exist");
-        default:
-          String tableInfo = Tables.getPrintableTableInfoFromName(instance, tableName);
-          throw new AccumuloSecurityException(e.user, e.code, tableInfo, e);
-      }
-    } catch (ThriftTableOperationException e) {
-      switch (e.getType()) {
-        case EXISTS:
-          throw new TableExistsException(e);
-        case NOTFOUND:
-          throw new TableNotFoundException(e);
-        case NAMESPACE_EXISTS:
-          throw new NamespaceExistsException(e);
-        case NAMESPACE_NOTFOUND:
-          throw new NamespaceNotFoundException(e);
-        case OFFLINE:
-          throw new TableOfflineException(instance, null);
-        default:
-          throw new AccumuloException(e.description, e);
-      }
-    } catch (Exception e) {
-      throw new AccumuloException(e.getMessage(), e);
-    } finally {
-      Tables.clearCache(instance);
-      // always finish table op, even when exception
-      if (opid != null)
-        try {
-          finishFateOperation(opid);
-        } catch (Exception e) {
-          log.warn(e.getMessage(), e);
-        }
-    }
-  }
-
-  private static class SplitEnv {
-    private String tableName;
-    private String tableId;
-    private ExecutorService executor;
-    private CountDownLatch latch;
-    private AtomicReference<Exception> exception;
-
-    SplitEnv(String tableName, String tableId, ExecutorService executor, CountDownLatch latch, AtomicReference<Exception> exception) {
-      this.tableName = tableName;
-      this.tableId = tableId;
-      this.executor = executor;
-      this.latch = latch;
-      this.exception = exception;
-    }
-  }
-
-  private class SplitTask implements Runnable {
-
-    private List<Text> splits;
-    private SplitEnv env;
-
-    SplitTask(SplitEnv env, List<Text> splits) {
-      this.env = env;
-      this.splits = splits;
-    }
-
-    @Override
-    public void run() {
-      try {
-        if (env.exception.get() != null)
-          return;
-
-        if (splits.size() <= 2) {
-          addSplits(env.tableName, new TreeSet<Text>(splits), env.tableId);
-          for (int i = 0; i < splits.size(); i++)
-            env.latch.countDown();
-          return;
-        }
-
-        int mid = splits.size() / 2;
-
-        // split the middle split point to ensure that child task split different tablets and can therefore
-        // run in parallel
-        addSplits(env.tableName, new TreeSet<Text>(splits.subList(mid, mid + 1)), env.tableId);
-        env.latch.countDown();
-
-        env.executor.submit(new SplitTask(env, splits.subList(0, mid)));
-        env.executor.submit(new SplitTask(env, splits.subList(mid + 1, splits.size())));
-
-      } catch (Exception e) {
-        env.exception.compareAndSet(null, e);
-      }
-    }
-
-  }
-
-  /**
-   * @param tableName
-   *          the name of the table
-   * @param partitionKeys
-   *          a sorted set of row key values to pre-split the table on
-   * @throws AccumuloException
-   *           if a general error occurs
-   * @throws AccumuloSecurityException
-   *           if the user does not have permission
-   * @throws TableNotFoundException
-   *           if the table does not exist
-   */
-  @Override
-  public void addSplits(String tableName, SortedSet<Text> partitionKeys) throws TableNotFoundException, AccumuloException, AccumuloSecurityException {
-    String tableId = Tables.getTableId(instance, tableName);
-
-    List<Text> splits = new ArrayList<Text>(partitionKeys);
-    // should be sorted because we copied from a sorted set, but that makes assumptions about
-    // how the copy was done so resort to be sure.
-    Collections.sort(splits);
-
-    CountDownLatch latch = new CountDownLatch(splits.size());
-    AtomicReference<Exception> exception = new AtomicReference<Exception>(null);
-
-    ExecutorService executor = Executors.newFixedThreadPool(16, new NamingThreadFactory("addSplits"));
-    try {
-      executor.submit(new SplitTask(new SplitEnv(tableName, tableId, executor, latch, exception), splits));
-
-      while (!latch.await(100, TimeUnit.MILLISECONDS)) {
-        if (exception.get() != null) {
-          executor.shutdownNow();
-          Exception excep = exception.get();
-          if (excep instanceof TableNotFoundException)
-            throw (TableNotFoundException) excep;
-          else if (excep instanceof AccumuloException)
-            throw (AccumuloException) excep;
-          else if (excep instanceof AccumuloSecurityException)
-            throw (AccumuloSecurityException) excep;
-          else if (excep instanceof RuntimeException)
-            throw (RuntimeException) excep;
-          else
-            throw new RuntimeException(excep);
-        }
-      }
-    } catch (InterruptedException e) {
-      throw new RuntimeException(e);
-    } finally {
-      executor.shutdown();
-    }
-  }
-
-  private void addSplits(String tableName, SortedSet<Text> partitionKeys, String tableId) throws AccumuloException, AccumuloSecurityException,
-      TableNotFoundException, AccumuloServerException {
-    TabletLocator tabLocator = TabletLocator.getLocator(instance, new Text(tableId));
-
-    for (Text split : partitionKeys) {
-      boolean successful = false;
-      int attempt = 0;
-
-      while (!successful) {
-
-        if (attempt > 0)
-          UtilWaitThread.sleep(100);
-
-        attempt++;
-
-        TabletLocation tl = tabLocator.locateTablet(credentials, split, false, false);
-
-        if (tl == null) {
-          if (!Tables.exists(instance, tableId))
-            throw new TableNotFoundException(tableId, tableName, null);
-          else if (Tables.getTableState(instance, tableId) == TableState.OFFLINE)
-            throw new TableOfflineException(instance, tableId);
-          continue;
-        }
-
-        try {
-          TabletClientService.Client client = ThriftUtil.getTServerClient(tl.tablet_location, ServerConfigurationUtil.getConfiguration(instance));
-          try {
-            OpTimer opTimer = null;
-            if (log.isTraceEnabled())
-              opTimer = new OpTimer(log, Level.TRACE).start("Splitting tablet " + tl.tablet_extent + " on " + tl.tablet_location + " at " + split);
-
-            client.splitTablet(Tracer.traceInfo(), credentials.toThrift(instance), tl.tablet_extent.toThrift(), TextUtil.getByteBuffer(split));
-
-            // just split it, might as well invalidate it in the cache
-            tabLocator.invalidateCache(tl.tablet_extent);
-
-            if (opTimer != null)
-              opTimer.stop("Split tablet in %DURATION%");
-          } finally {
-            ThriftUtil.returnClient(client);
-          }
-
-        } catch (TApplicationException tae) {
-          throw new AccumuloServerException(tl.tablet_location, tae);
-        } catch (TTransportException e) {
-          tabLocator.invalidateCache(tl.tablet_location);
-          continue;
-        } catch (ThriftSecurityException e) {
-          Tables.clearCache(instance);
-          if (!Tables.exists(instance, tableId))
-            throw new TableNotFoundException(tableId, tableName, null);
-          throw new AccumuloSecurityException(e.user, e.code, e);
-        } catch (NotServingTabletException e) {
-          tabLocator.invalidateCache(tl.tablet_extent);
-          continue;
-        } catch (TException e) {
-          tabLocator.invalidateCache(tl.tablet_location);
-          continue;
-        }
-
-        successful = true;
-      }
-    }
-  }
-
-  @Override
-  public void merge(String tableName, Text start, Text end) throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
-
-    checkArgument(tableName != null, "tableName is null");
-    ByteBuffer EMPTY = ByteBuffer.allocate(0);
-    List<ByteBuffer> args = Arrays.asList(ByteBuffer.wrap(tableName.getBytes(StandardCharsets.UTF_8)), start == null ? EMPTY : TextUtil.getByteBuffer(start),
-        end == null ? EMPTY : TextUtil.getByteBuffer(end));
-    Map<String,String> opts = new HashMap<String,String>();
-    try {
-      doTableFateOperation(tableName, TableNotFoundException.class, FateOperation.TABLE_MERGE, args, opts);
-    } catch (TableExistsException e) {
-      // should not happen
-      throw new AssertionError(e);
-    }
-  }
-
-  @Override
-  public void deleteRows(String tableName, Text start, Text end) throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
-
-    checkArgument(tableName != null, "tableName is null");
-    ByteBuffer EMPTY = ByteBuffer.allocate(0);
-    List<ByteBuffer> args = Arrays.asList(ByteBuffer.wrap(tableName.getBytes(StandardCharsets.UTF_8)), start == null ? EMPTY : TextUtil.getByteBuffer(start),
-        end == null ? EMPTY : TextUtil.getByteBuffer(end));
-    Map<String,String> opts = new HashMap<String,String>();
-    try {
-      doTableFateOperation(tableName, TableNotFoundException.class, FateOperation.TABLE_DELETE_RANGE, args, opts);
-    } catch (TableExistsException e) {
-      // should not happen
-      throw new AssertionError(e);
-    }
-  }
-
-  /**
-   * @param tableName
-   *          the name of the table
-   * @return the split points (end-row names) for the table's current split profile
-   */
-  @Override
-  public Collection<Text> listSplits(String tableName) throws TableNotFoundException, AccumuloSecurityException {
-
-    checkArgument(tableName != null, "tableName is null");
-
-    String tableId = Tables.getTableId(instance, tableName);
-
-    TreeMap<KeyExtent,String> tabletLocations = new TreeMap<KeyExtent,String>();
-
-    while (true) {
-      try {
-        tabletLocations.clear();
-        // the following method throws AccumuloException for some conditions that should be retried
-        MetadataServicer.forTableId(instance, credentials, tableId).getTabletLocations(tabletLocations);
-        break;
-      } catch (AccumuloSecurityException ase) {
-        throw ase;
-      } catch (Exception e) {
-        if (!Tables.exists(instance, tableId)) {
-          throw new TableNotFoundException(tableId, tableName, null);
-        }
-
-        if (e instanceof RuntimeException && e.getCause() instanceof AccumuloSecurityException) {
-          throw (AccumuloSecurityException) e.getCause();
-        }
-
-        log.info(e.getMessage() + " ... retrying ...");
-        UtilWaitThread.sleep(3000);
-      }
-    }
-
-    ArrayList<Text> endRows = new ArrayList<Text>(tabletLocations.size());
-
-    for (KeyExtent ke : tabletLocations.keySet())
-      if (ke.getEndRow() != null)
-        endRows.add(ke.getEndRow());
-
-    return endRows;
-  }
-
-  @Deprecated
-  @Override
-  public Collection<Text> getSplits(String tableName) throws TableNotFoundException {
-    try {
-      return listSplits(tableName);
-    } catch (AccumuloSecurityException e) {
-      throw new RuntimeException(e);
-    }
-  }
-
-  /**
-   * @param tableName
-   *          the name of the table
-   * @param maxSplits
-   *          specifies the maximum number of splits to return
-   * @return the split points (end-row names) for the table's current split profile, grouped into fewer splits so as not to exceed maxSplits
-   */
-  @Override
-  public Collection<Text> listSplits(String tableName, int maxSplits) throws TableNotFoundException, AccumuloSecurityException {
-    Collection<Text> endRows = listSplits(tableName);
-
-    if (endRows.size() <= maxSplits)
-      return endRows;
-
-    double r = (maxSplits + 1) / (double) (endRows.size());
-    double pos = 0;
-
-    ArrayList<Text> subset = new ArrayList<Text>(maxSplits);
-
-    int j = 0;
-    for (int i = 0; i < endRows.size() && j < maxSplits; i++) {
-      pos += r;
-      while (pos > 1) {
-        subset.add(((ArrayList<Text>) endRows).get(i));
-        j++;
-        pos -= 1;
-      }
-    }
-
-    return subset;
-  }
-
-  @Deprecated
-  @Override
-  public Collection<Text> getSplits(String tableName, int maxSplits) throws TableNotFoundException {
-    try {
-      return listSplits(tableName, maxSplits);
-    } catch (AccumuloSecurityException e) {
-      throw new RuntimeException(e);
-    }
-  }
-
-  /**
-   * Delete a table
-   * 
-   * @param tableName
-   *          the name of the table
-   * @throws AccumuloException
-   *           if a general error occurs
-   * @throws AccumuloSecurityException
-   *           if the user does not have permission
-   * @throws TableNotFoundException
-   *           if the table does not exist
-   */
-  @Override
-  public void delete(String tableName) throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
-    checkArgument(tableName != null, "tableName is null");
-
-    List<ByteBuffer> args = Arrays.asList(ByteBuffer.wrap(tableName.getBytes(StandardCharsets.UTF_8)));
-    Map<String,String> opts = new HashMap<String,String>();
-
-    try {
-      doTableFateOperation(tableName, TableNotFoundException.class, FateOperation.TABLE_DELETE, args, opts);
-    } catch (TableExistsException e) {
-      // should not happen
-      throw new AssertionError(e);
-    }
-
-  }
-
-  @Override
-  public void clone(String srcTableName, String newTableName, boolean flush, Map<String,String> propertiesToSet, Set<String> propertiesToExclude)
-      throws AccumuloSecurityException, TableNotFoundException, AccumuloException, TableExistsException {
-
-    checkArgument(srcTableName != null, "srcTableName is null");
-    checkArgument(newTableName != null, "newTableName is null");
-
-    String srcTableId = Tables.getTableId(instance, srcTableName);
-
-    if (flush)
-      _flush(srcTableId, null, null, true);
-
-    if (propertiesToExclude == null)
-      propertiesToExclude = Collections.emptySet();
-
-    if (propertiesToSet == null)
-      propertiesToSet = Collections.emptyMap();
-
-    List<ByteBuffer> args = Arrays.asList(ByteBuffer.wrap(srcTableId.getBytes(StandardCharsets.UTF_8)), ByteBuffer.wrap(newTableName.getBytes(StandardCharsets.UTF_8)));
-    Map<String,String> opts = new HashMap<String,String>();
-    for (Entry<String,String> entry : propertiesToSet.entrySet()) {
-      if (entry.getKey().startsWith(CLONE_EXCLUDE_PREFIX))
-        throw new IllegalArgumentException("Property can not start with " + CLONE_EXCLUDE_PREFIX);
-      opts.put(entry.getKey(), entry.getValue());
-    }
-
-    for (String prop : propertiesToExclude) {
-      opts.put(CLONE_EXCLUDE_PREFIX + prop, "");
-    }
-
-    doTableFateOperation(newTableName, AccumuloException.class, FateOperation.TABLE_CLONE, args, opts);
-  }
-
-  /**
-   * Rename a table
-   * 
-   * @param oldTableName
-   *          the old table name
-   * @param newTableName
-   *          the new table name
-   * @throws AccumuloException
-   *           if a general error occurs
-   * @throws AccumuloSecurityException
-   *           if the user does not have permission
-   * @throws TableNotFoundException
-   *           if the old table name does not exist
-   * @throws TableExistsException
-   *           if the new table name already exists
-   */
-  @Override
-  public void rename(String oldTableName, String newTableName) throws AccumuloSecurityException, TableNotFoundException, AccumuloException,
-      TableExistsException {
-
-    List<ByteBuffer> args = Arrays.asList(ByteBuffer.wrap(oldTableName.getBytes(StandardCharsets.UTF_8)), ByteBuffer.wrap(newTableName.getBytes(StandardCharsets.UTF_8)));
-    Map<String,String> opts = new HashMap<String,String>();
-    doTableFateOperation(oldTableName, TableNotFoundException.class, FateOperation.TABLE_RENAME, args, opts);
-  }
-
-  /**
-   * @deprecated since 1.4 {@link #flush(String, Text, Text, boolean)}
-   */
-  @Override
-  @Deprecated
-  public void flush(String tableName) throws AccumuloException, AccumuloSecurityException {
-    try {
-      flush(tableName, null, null, false);
-    } catch (TableNotFoundException e) {
-      throw new AccumuloException(e.getMessage(), e);
-    }
-  }
-
-  /**
-   * Flush a table
-   * 
-   * @param tableName
-   *          the name of the table
-   * @throws AccumuloException
-   *           if a general error occurs
-   * @throws AccumuloSecurityException
-   *           if the user does not have permission
-   */
-  @Override
-  public void flush(String tableName, Text start, Text end, boolean wait) throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
-    checkArgument(tableName != null, "tableName is null");
-
-    String tableId = Tables.getTableId(instance, tableName);
-    _flush(tableId, start, end, wait);
-  }
-
-  @Override
-  public void compact(String tableName, Text start, Text end, boolean flush, boolean wait) throws AccumuloSecurityException, TableNotFoundException,
-      AccumuloException {
-    compact(tableName, start, end, new ArrayList<IteratorSetting>(), flush, wait);
-  }
-
-  @Override
-  public void compact(String tableName, Text start, Text end, List<IteratorSetting> iterators, boolean flush, boolean wait) throws AccumuloSecurityException,
-      TableNotFoundException, AccumuloException {
-    checkArgument(tableName != null, "tableName is null");
-    ByteBuffer EMPTY = ByteBuffer.allocate(0);
-
-    String tableId = Tables.getTableId(instance, tableName);
-
-    if (flush)
-      _flush(tableId, start, end, true);
-
-    List<ByteBuffer> args = Arrays.asList(ByteBuffer.wrap(tableId.getBytes(StandardCharsets.UTF_8)), start == null ? EMPTY : TextUtil.getByteBuffer(start),
-        end == null ? EMPTY : TextUtil.getByteBuffer(end), ByteBuffer.wrap(IteratorUtil.encodeIteratorSettings(iterators)));
-
-    Map<String,String> opts = new HashMap<String,String>();
-    try {
-      doFateOperation(FateOperation.TABLE_COMPACT, args, opts, wait);
-    } catch (TableExistsException e) {
-      // should not happen
-      throw new AssertionError(e);
-    } catch (NamespaceExistsException e) {
-      // should not happen
-      throw new AssertionError(e);
-    } catch (NamespaceNotFoundException e) {
-      throw new TableNotFoundException(null, tableName, "Namespace not found", e);
-    }
-  }
-
-  @Override
-  public void cancelCompaction(String tableName) throws AccumuloSecurityException, TableNotFoundException, AccumuloException {
-    String tableId = Tables.getTableId(instance, tableName);
-
-    List<ByteBuffer> args = Arrays.asList(ByteBuffer.wrap(tableId.getBytes(StandardCharsets.UTF_8)));
-
-    Map<String,String> opts = new HashMap<String,String>();
-    try {
-      doTableFateOperation(tableName, TableNotFoundException.class, FateOperation.TABLE_CANCEL_COMPACT, args, opts);
-    } catch (TableExistsException e) {
-      // should not happen
-      throw new AssertionError(e);
-    }
-
-  }
-
-  private void _flush(String tableId, Text start, Text end, boolean wait) throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
-
-    try {
-      long flushID;
-
-      // used to pass the table name. but the tableid associated with a table name could change between calls.
-      // so pass the tableid to both calls
-
-      while (true) {
-        MasterClientService.Iface client = null;
-        try {
-          client = MasterClient.getConnectionWithRetry(instance);
-          flushID = client.initiateFlush(Tracer.traceInfo(), credentials.toThrift(instance), tableId);
-          break;
-        } catch (TTransportException tte) {
-          log.debug("Failed to call initiateFlush, retrying ... ", tte);
-          UtilWaitThread.sleep(100);
-        } finally {
-          MasterClient.close(client);
-        }
-      }
-
-      while (true) {
-        MasterClientService.Iface client = null;
-        try {
-          client = MasterClient.getConnectionWithRetry(instance);
-          client.waitForFlush(Tracer.traceInfo(), credentials.toThrift(instance), tableId, TextUtil.getByteBuffer(start), TextUtil.getByteBuffer(end), flushID,
-              wait ? Long.MAX_VALUE : 1);
-          break;
-        } catch (TTransportException tte) {
-          log.debug("Failed to call initiateFlush, retrying ... ", tte);
-          UtilWaitThread.sleep(100);
-        } finally {
-          MasterClient.close(client);
-        }
-      }
-    } catch (ThriftSecurityException e) {
-      switch (e.getCode()) {
-        case TABLE_DOESNT_EXIST:
-          throw new TableNotFoundException(tableId, null, e.getMessage(), e);
-        default:
-          log.debug("flush security exception on table id " + tableId);
-          throw new AccumuloSecurityException(e.user, e.code, e);
-      }
-    } catch (ThriftTableOperationException e) {
-      switch (e.getType()) {
-        case NOTFOUND:
-          throw new TableNotFoundException(e);
-        default:
-          throw new AccumuloException(e.description, e);
-      }
-    } catch (Exception e) {
-      throw new AccumuloException(e);
-    }
-  }
-
-  /**
-   * Sets a property on a table
-   * 
-   * @param tableName
-   *          the name of the table
-   * @param property
-   *          the name of a per-table property
-   * @param value
-   *          the value to set a per-table property to
-   * @throws AccumuloException
-   *           if a general error occurs
-   * @throws AccumuloSecurityException
-   *           if the user does not have permission
-   */
-  @Override
-  public void setProperty(final String tableName, final String property, final String value) throws AccumuloException, AccumuloSecurityException {
-    checkArgument(tableName != null, "tableName is null");
-    checkArgument(property != null, "property is null");
-    checkArgument(value != null, "value is null");
-    try {
-      MasterClient.executeTable(instance, new ClientExec<MasterClientService.Client>() {
-        @Override
-        public void execute(MasterClientService.Client client) throws Exception {
-          client.setTableProperty(Tracer.traceInfo(), credentials.toThrift(instance), tableName, property, value);
-        }
-      });
-    } catch (TableNotFoundException e) {
-      throw new AccumuloException(e);
-    }
-  }
-
-  /**
-   * Removes a property from a table
-   * 
-   * @param tableName
-   *          the name of the table
-   * @param property
-   *          the name of a per-table property
-   * @throws AccumuloException
-   *           if a general error occurs
-   * @throws AccumuloSecurityException
-   *           if the user does not have permission
-   */
-  @Override
-  public void removeProperty(final String tableName, final String property) throws AccumuloException, AccumuloSecurityException {
-    checkArgument(tableName != null, "tableName is null");
-    checkArgument(property != null, "property is null");
-    try {
-      MasterClient.executeTable(instance, new ClientExec<MasterClientService.Client>() {
-        @Override
-        public void execute(MasterClientService.Client client) throws Exception {
-          client.removeTableProperty(Tracer.traceInfo(), credentials.toThrift(instance), tableName, property);
-        }
-      });
-    } catch (TableNotFoundException e) {
-      throw new AccumuloException(e);
-    }
-  }
-
-  /**
-   * Gets properties of a table
-   * 
-   * @param tableName
-   *          the name of the table
-   * @return all properties visible by this table (system and per-table properties)
-   * @throws TableNotFoundException
-   *           if the table does not exist
-   */
-  @Override
-  public Iterable<Entry<String,String>> getProperties(final String tableName) throws AccumuloException, TableNotFoundException {
-    checkArgument(tableName != null, "tableName is null");
-    try {
-      return ServerClient.executeRaw(instance, new ClientExecReturn<Map<String,String>,ClientService.Client>() {
-        @Override
-        public Map<String,String> execute(ClientService.Client client) throws Exception {
-          return client.getTableConfiguration(Tracer.traceInfo(), credentials.toThrift(instance), tableName);
-        }
-      }).entrySet();
-    } catch (ThriftTableOperationException e) {
-      switch (e.getType()) {
-        case NOTFOUND:
-          throw new TableNotFoundException(e);
-        case NAMESPACE_NOTFOUND:
-          throw new TableNotFoundException(tableName, new NamespaceNotFoundException(e));
-        default:
-          throw new AccumuloException(e.description, e);
-      }
-    } catch (AccumuloException e) {
-      throw e;
-    } catch (Exception e) {
-      throw new AccumuloException(e);
-    }
-
-  }
-
-  /**
-   * Sets a tables locality groups. A tables locality groups can be changed at any time.
-   * 
-   * @param tableName
-   *          the name of the table
-   * @param groups
-   *          mapping of locality group names to column families in the locality group
-   * @throws AccumuloException
-   *           if a general error occurs
-   * @throws AccumuloSecurityException
-   *           if the user does not have permission
-   * @throws TableNotFoundException
-   *           if the table does not exist
-   */
-  @Override
-  public void setLocalityGroups(String tableName, Map<String,Set<Text>> groups) throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
-    // ensure locality groups do not overlap
-    HashSet<Text> all = new HashSet<Text>();
-    for (Entry<String,Set<Text>> entry : groups.entrySet()) {
-
-      if (!Collections.disjoint(all, entry.getValue())) {
-        throw new IllegalArgumentException("Group " + entry.getKey() + " overlaps with another group");
-      }
-
-      all.addAll(entry.getValue());
-    }
-
-    for (Entry<String,Set<Text>> entry : groups.entrySet()) {
-      Set<Text> colFams = entry.getValue();
-      String value = LocalityGroupUtil.encodeColumnFamilies(colFams);
-      setProperty(tableName, Property.TABLE_LOCALITY_GROUP_PREFIX + entry.getKey(), value);
-    }
-
-    try {
-      setProperty(tableName, Property.TABLE_LOCALITY_GROUPS.getKey(), Joiner.on(",").join(groups.keySet()));
-    } catch (AccumuloException e) {
-      if (e.getCause() instanceof TableNotFoundException)
-        throw (TableNotFoundException) e.getCause();
-      throw e;
-    }
-
-    // remove anything extraneous
-    String prefix = Property.TABLE_LOCALITY_GROUP_PREFIX.getKey();
-    for (Entry<String,String> entry : getProperties(tableName)) {
-      String property = entry.getKey();
-      if (property.startsWith(prefix)) {
-        // this property configures a locality group, find out which
-        // one:
-        String[] parts = property.split("\\.");
-        String group = parts[parts.length - 1];
-
-        if (!groups.containsKey(group)) {
-          removeProperty(tableName, property);
-        }
-      }
-    }
-  }
-
-  /**
-   * 
-   * Gets the locality groups currently set for a table.
-   * 
-   * @param tableName
-   *          the name of the table
-   * @return mapping of locality group names to column families in the locality group
-   * @throws AccumuloException
-   *           if a general error occurs
-   * @throws TableNotFoundException
-   *           if the table does not exist
-   */
-  @Override
-  public Map<String,Set<Text>> getLocalityGroups(String tableName) throws AccumuloException, TableNotFoundException {
-    AccumuloConfiguration conf = new ConfigurationCopy(this.getProperties(tableName));
-    Map<String,Set<ByteSequence>> groups = LocalityGroupUtil.getLocalityGroups(conf);
-
-    Map<String,Set<Text>> groups2 = new HashMap<String,Set<Text>>();
-    for (Entry<String,Set<ByteSequence>> entry : groups.entrySet()) {
-
-      HashSet<Text> colFams = new HashSet<Text>();
-
-      for (ByteSequence bs : entry.getValue()) {
-        colFams.add(new Text(bs.toArray()));
-      }
-
-      groups2.put(entry.getKey(), colFams);
-    }
-
-    return groups2;
-  }
-
-  /**
-   * @param tableName
-   *          the name of the table
-   * @param range
-   *          a range to split
-   * @param maxSplits
-   *          the maximum number of splits
-   * @return the range, split into smaller ranges that fall on boundaries of the table's split points as evenly as possible
-   * @throws AccumuloException
-   *           if a general error occurs
-   * @throws AccumuloSecurityException
-   *           if the user does not have permission
-   * @throws TableNotFoundException
-   *           if the table does not exist
-   */
-  @Override
-  public Set<Range> splitRangeByTablets(String tableName, Range range, int maxSplits) throws AccumuloException, AccumuloSecurityException,
-      TableNotFoundException {
-    checkArgument(tableName != null, "tableName is null");
-    checkArgument(range != null, "range is null");
-    if (maxSplits < 1)
-      throw new IllegalArgumentException("maximum splits must be >= 1");
-    if (maxSplits == 1)
-      return Collections.singleton(range);
-
-    Random random = new Random();
-    Map<String,Map<KeyExtent,List<Range>>> binnedRanges = new HashMap<String,Map<KeyExtent,List<Range>>>();
-    String tableId = Tables.getTableId(instance, tableName);
-    TabletLocator tl = TabletLocator.getLocator(instance, new Text(tableId));
-    // its possible that the cache could contain complete, but old information about a tables tablets... so clear it
-    tl.invalidateCache();
-    while (!tl.binRanges(credentials, Collections.singletonList(range), binnedRanges).isEmpty()) {
-      if (!Tables.exists(instance, tableId))
-        throw new TableDeletedException(tableId);
-      if (Tables.getTableState(instance, tableId) == TableState.OFFLINE)
-        throw new TableOfflineException(instance, tableId);
-
-      log.warn("Unable to locate bins for specified range. Retrying.");
-      // sleep randomly between 100 and 200ms
-      UtilWaitThread.sleep(100 + random.nextInt(100));
-      binnedRanges.clear();
-      tl.invalidateCache();
-    }
-
-    // group key extents to get <= maxSplits
-    LinkedList<KeyExtent> unmergedExtents = new LinkedList<KeyExtent>();
-    List<KeyExtent> mergedExtents = new ArrayList<KeyExtent>();
-
-    for (Map<KeyExtent,List<Range>> map : binnedRanges.values())
-      unmergedExtents.addAll(map.keySet());
-
-    // the sort method is efficient for linked list
-    Collections.sort(unmergedExtents);
-
-    while (unmergedExtents.size() + mergedExtents.size() > maxSplits) {
-      if (unmergedExtents.size() >= 2) {
-        KeyExtent first = unmergedExtents.removeFirst();
-        KeyExtent second = unmergedExtents.removeFirst();
-        first.setEndRow(second.getEndRow());
-        mergedExtents.add(first);
-      } else {
-        mergedExtents.addAll(unmergedExtents);
-        unmergedExtents.clear();
-        unmergedExtents.addAll(mergedExtents);
-        mergedExtents.clear();
-      }
-
-    }
-
-    mergedExtents.addAll(unmergedExtents);
-
-    Set<Range> ranges = new HashSet<Range>();
-    for (KeyExtent k : mergedExtents)
-      ranges.add(k.toDataRange().clip(range));
-
-    return ranges;
-  }
-
-  // TODO Remove deprecation warning surppression when Hadoop1 support is dropped
-  @SuppressWarnings("deprecation")
-  private Path checkPath(String dir, String kind, String type) throws IOException, AccumuloException {
-    Path ret;
-    FileSystem fs = VolumeConfiguration.getVolume(dir, CachedConfiguration.getInstance(), ServerConfigurationUtil.getConfiguration(instance)).getFileSystem();
-
-    if (dir.contains(":")) {
-      ret = new Path(dir);
-    } else {
-      ret = fs.makeQualified(new Path(dir));
-    }
-
-    if (!fs.exists(ret))
-      throw new AccumuloException(kind + " import " + type + " directory " + dir + " does not exist!");
-
-    if (!fs.getFileStatus(ret).isDir()) {
-      throw new AccumuloException(kind + " import " + type + " directory " + dir + " is not a directory!");
-    }
-
-    if (type.equals("failure")) {
-      FileStatus[] listStatus = fs.listStatus(ret);
-      if (listStatus != null && listStatus.length != 0) {
-        throw new AccumuloException("Bulk import failure directory " + ret + " is not empty");
-      }
-    }
-
-    return ret;
-  }
-
-  @Override
-  public void importDirectory(String tableName, String dir, String failureDir, boolean setTime) throws IOException, AccumuloSecurityException,
-      TableNotFoundException, AccumuloException {
-    checkArgument(tableName != null, "tableName is null");
-    checkArgument(dir != null, "dir is null");
-    checkArgument(failureDir != null, "failureDir is null");
-    // check for table existance
-    Tables.getTableId(instance, tableName);
-
-    Path dirPath = checkPath(dir, "Bulk", "");
-    Path failPath = checkPath(failureDir, "Bulk", "failure");
-
-    List<ByteBuffer> args = Arrays.asList(ByteBuffer.wrap(tableName.getBytes(StandardCharsets.UTF_8)), ByteBuffer.wrap(dirPath.toString().getBytes(StandardCharsets.UTF_8)),
-        ByteBuffer.wrap(failPath.toString().getBytes(StandardCharsets.UTF_8)), ByteBuffer.wrap((setTime + "").getBytes(StandardCharsets.UTF_8)));
-    Map<String,String> opts = new HashMap<String,String>();
-
-    try {
-      doTableFateOperation(tableName, TableNotFoundException.class, FateOperation.TABLE_BULK_IMPORT, args, opts);
-    } catch (TableExistsException e) {
-      // should not happen
-      throw new AssertionError(e);
-    }
-  }
-
-  private void waitForTableStateTransition(String tableId, TableState expectedState) throws AccumuloException, TableNotFoundException,
-      AccumuloSecurityException {
-
-    Text startRow = null;
-    Text lastRow = null;
-
-    while (true) {
-
-      if (Tables.getTableState(instance, tableId) != expectedState) {
-        Tables.clearCache(instance);
-        if (Tables.getTableState(instance, tableId) != expectedState) {
-          if (!Tables.exists(instance, tableId))
-            throw new TableDeletedException(tableId);
-          throw new AccumuloException("Unexpected table state " + tableId + " " + Tables.getTableState(instance, tableId) + " != " + expectedState);
-        }
-      }
-
-      Range range = new KeyExtent(new Text(tableId), null, null).toMetadataRange();
-      if (startRow == null || lastRow == null)
-        range = new KeyExtent(new Text(tableId), null, null).toMetadataRange();
-      else
-        range = new Range(startRow, lastRow);
-
-      String metaTable = MetadataTable.NAME;
-      if (tableId.equals(MetadataTable.ID))
-        metaTable = RootTable.NAME;
-      Scanner scanner = instance.getConnector(credentials.getPrincipal(), credentials.getToken()).createScanner(metaTable, Authorizations.EMPTY);
-      scanner = new IsolatedScanner(scanner);
-      TabletsSection.TabletColumnFamily.PREV_ROW_COLUMN.fetch(scanner);
-      scanner.fetchColumnFamily(TabletsSection.CurrentLocationColumnFamily.NAME);
-      scanner.setRange(range);
-
-      RowIterator rowIter = new RowIterator(scanner);
-
-      KeyExtent lastExtent = null;
-
-      int total = 0;
-      int waitFor = 0;
-      int holes = 0;
-      Text continueRow = null;
-      MapCounter<String> serverCounts = new MapCounter<String>();
-
-      while (rowIter.hasNext()) {
-        Iterator<Entry<Key,Value>> row = rowIter.next();
-
-        total++;
-
-        KeyExtent extent = null;
-        String future = null;
-        String current = null;
-
-        while (row.hasNext()) {
-          Entry<Key,Value> entry = row.next();
-          Key key = entry.getKey();
-
-          if (key.getColumnFamily().equals(TabletsSection.FutureLocationColumnFamily.NAME))
-            future = entry.getValue().toString();
-
-          if (key.getColumnFamily().equals(TabletsSection.CurrentLocationColumnFamily.NAME))
-            current = entry.getValue().toString();
-
-          if (TabletsSection.TabletColumnFamily.PREV_ROW_COLUMN.hasColumns(key))
-            extent = new KeyExtent(key.getRow(), entry.getValue());
-        }
-
-        if ((expectedState == TableState.ONLINE && current == null) || (expectedState == TableState.OFFLINE && (future != null || current != null))) {
-          if (continueRow == null)
-            continueRow = extent.getMetadataEntry();
-          waitFor++;
-          lastRow = extent.getMetadataEntry();
-
-          if (current != null)
-            serverCounts.increment(current, 1);
-          if (future != null)
-            serverCounts.increment(future, 1);
-        }
-
-        if (!extent.getTableId().toString().equals(tableId)) {
-          throw new AccumuloException("Saw unexpected table Id " + tableId + " " + extent);
-        }
-
-        if (lastExtent != null && !extent.isPreviousExtent(lastExtent)) {
-          holes++;
-        }
-
-        lastExtent = extent;
-      }
-
-      if (continueRow != null) {
-        startRow = continueRow;
-      }
-
-      if (holes > 0 || total == 0) {
-        startRow = null;
-        lastRow = null;
-      }
-
-      if (waitFor > 0 || holes > 0 || total == 0) {
-        long waitTime;
-        long maxPerServer = 0;
-        if (serverCounts.size() > 0) {
-          maxPerServer = Collections.max(serverCounts.values());
-          waitTime = maxPerServer * 10;
-        } else
-          waitTime = waitFor * 10;
-        waitTime = Math.max(100, waitTime);
-        waitTime = Math.min(5000, waitTime);
-        log.trace("Waiting for " + waitFor + "(" + maxPerServer + ") tablets, startRow = " + startRow + " lastRow = " + lastRow + ", holes=" + holes
-            + " sleeping:" + waitTime + "ms");
-        UtilWaitThread.sleep(waitTime);
-      } else {
-        break;
-      }
-
-    }
-  }
-
-  @Override
-  public void offline(String tableName) throws AccumuloSecurityException, AccumuloException, TableNotFoundException {
-    offline(tableName, false);
-  }
-
-  /**
-   * 
-   * @param tableName
-   *          the table to take offline
-   * @throws AccumuloException
-   *           when there is a general accumulo error
-   * @throws AccumuloSecurityException
-   *           when the user does not have the proper permissions
-   */
-  @Override
-  public void offline(String tableName, boolean wait) throws AccumuloSecurityException, AccumuloException, TableNotFoundException {
-
-    checkArgument(tableName != null, "tableName is null");
-    String tableId = Tables.getTableId(instance, tableName);
-    List<ByteBuffer> args = Arrays.asList(ByteBuffer.wrap(tableId.getBytes(StandardCharsets.UTF_8)));
-    Map<String,String> opts = new HashMap<String,String>();
-
-    try {
-      doTableFateOperation(tableName, TableNotFoundException.class, FateOperation.TABLE_OFFLINE, args, opts);
-    } catch (TableExistsException e) {
-      // should not happen
-      throw new AssertionError(e);
-    }
-
-    if (wait)
-      waitForTableStateTransition(tableId, TableState.OFFLINE);
-  }
-
-  @Override
-  public void online(String tableName) throws AccumuloSecurityException, AccumuloException, TableNotFoundException {
-    online(tableName, false);
-  }
-
-  /**
-   * 
-   * @param tableName
-   *          the table to take online
-   * @throws AccumuloException
-   *           when there is a general accumulo error
-   * @throws AccumuloSecurityException
-   *           when the user does not have the proper permissions
-   */
-  @Override
-  public void online(String tableName, boolean wait) throws AccumuloSecurityException, AccumuloException, TableNotFoundException {
-    checkArgument(tableName != null, "tableName is null");
-    String tableId = Tables.getTableId(instance, tableName);
-    List<ByteBuffer> args = Arrays.asList(ByteBuffer.wrap(tableId.getBytes(StandardCharsets.UTF_8)));
-    Map<String,String> opts = new HashMap<String,String>();
-
-    try {
-      doTableFateOperation(tableName, TableNotFoundException.class, FateOperation.TABLE_ONLINE, args, opts);
-    } catch (TableExistsException e) {
-      // should not happen
-      throw new AssertionError(e);
-    }
-
-    if (wait)
-      waitForTableStateTransition(tableId, TableState.ONLINE);
-  }
-
-  /**
-   * Clears the tablet locator cache for a specified table
-   * 
-   * @param tableName
-   *          the name of the table
-   * @throws TableNotFoundException
-   *           if table does not exist
-   */
-  @Override
-  public void clearLocatorCache(String tableName) throws TableNotFoundException {
-    checkArgument(tableName != null, "tableName is null");
-    TabletLocator tabLocator = TabletLocator.getLocator(instance, new Text(Tables.getTableId(instance, tableName)));
-    tabLocator.invalidateCache();
-  }
-
-  /**
-   * Get a mapping of table name to internal table id.
-   * 
-   * @return the map from table name to internal table id
-   */
-  @Override
-  public Map<String,String> tableIdMap() {
-    return Tables.getNameToIdMap(instance);
-  }
-
-  @Override
-  public Text getMaxRow(String tableName, Authorizations auths, Text startRow, boolean startInclusive, Text endRow, boolean endInclusive)
-      throws TableNotFoundException, AccumuloException, AccumuloSecurityException {
-    checkArgument(tableName != null, "tableName is null");
-    checkArgument(auths != null, "auths is null");
-    Scanner scanner = instance.getConnector(credentials.getPrincipal(), credentials.getToken()).createScanner(tableName, auths);
-    return FindMax.findMax(scanner, startRow, startInclusive, endRow, endInclusive);
-  }
-
-  @Override
-  public List<DiskUsage> getDiskUsage(Set<String> tableNames) throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
-
-    List<TDiskUsage> diskUsages = null;
-    while (diskUsages == null) {
-      Pair<String,Client> pair = null;
-      try {
-        // this operation may us a lot of memory... its likely that connections to tabletservers hosting metadata tablets will be cached, so do not use cached
-        // connections
-        pair = ServerClient.getConnection(instance, false);
-        diskUsages = pair.getSecond().getDiskUsage(tableNames, credentials.toThrift(instance));
-      } catch (ThriftTableOperationException e) {
-        switch (e.getType()) {
-          case NOTFOUND:
-            throw new TableNotFoundException(e);
-          case NAMESPACE_NOTFOUND:
-            throw new TableNotFoundException(e.getTableName(), new NamespaceNotFoundException(e));
-          default:
-            throw new AccumuloException(e.description, e);
-        }
-      } catch (ThriftSecurityException e) {
-        throw new AccumuloSecurityException(e.getUser(), e.getCode());
-      } catch (TTransportException e) {
-        // some sort of communication error occurred, retry
-        log.debug("disk usage request failed " + pair.getFirst() + ", retrying ... ", e);
-        UtilWaitThread.sleep(100);
-      } catch (TException e) {
-        // may be a TApplicationException which indicates error on the server side
-        throw new AccumuloException(e);
-      } finally {
-        // must always return thrift connection
-        if (pair != null)
-          ServerClient.close(pair.getSecond());
-      }
-    }
-
-    List<DiskUsage> finalUsages = new ArrayList<DiskUsage>();
-    for (TDiskUsage diskUsage : diskUsages) {
-      finalUsages.add(new DiskUsage(new TreeSet<String>(diskUsage.getTables()), diskUsage.getUsage()));
-    }
-
-    return finalUsages;
-  }
-
-  public static Map<String,String> getExportedProps(FileSystem fs, Path path) throws IOException {
-    HashMap<String,String> props = new HashMap<String,String>();
-
-    ZipInputStream zis = new ZipInputStream(fs.open(path));
-    try {
-      ZipEntry zipEntry;
-      while ((zipEntry = zis.getNextEntry()) != null) {
-        if (zipEntry.getName().equals(Constants.EXPORT_TABLE_CONFIG_FILE)) {
-          BufferedReader in = new BufferedReader(new InputStreamReader(zis, StandardCharsets.UTF_8));
-          try {
-            String line;
-            while ((line = in.readLine()) != null) {
-              String sa[] = line.split("=", 2);
-              props.put(sa[0], sa[1]);
-            }
-          } finally {
-            in.close();
-          }
-
-          break;
-        }
-      }
-    } finally {
-      zis.close();
-    }
-    return props;
-  }
-
-  @Override
-  public void importTable(String tableName, String importDir) throws TableExistsException, AccumuloException, AccumuloSecurityException {
-    checkArgument(tableName != null, "tableName is null");
-    checkArgument(importDir != null, "importDir is null");
-
-    try {
-      importDir = checkPath(importDir, "Table", "").toString();
-    } catch (IOException e) {
-      throw new AccumuloException(e);
-    }
-
-    try {
-      FileSystem fs = new Path(importDir).getFileSystem(CachedConfiguration.getInstance());
-      Map<String,String> props = getExportedProps(fs, new Path(importDir, Constants.EXPORT_FILE));
-
-      for (Entry<String,String> entry : props.entrySet()) {
-        if (Property.isClassProperty(entry.getKey()) && !entry.getValue().contains(Constants.CORE_PACKAGE_NAME)) {
-          Logger.getLogger(this.getClass()).info(
-              "Imported table sets '" + entry.getKey() + "' to '" + entry.getValue() + "'.  Ensure this class is on Accumulo classpath.");
-        }
-      }
-
-    } catch (IOException ioe) {
-      Logger.getLogger(this.getClass()).warn("Failed to check if imported table references external java classes : " + ioe.getMessage());
-    }
-
-    List<ByteBuffer> args = Arrays.asList(ByteBuffer.wrap(tableName.getBytes(StandardCharsets.UTF_8)), ByteBuffer.wrap(importDir.getBytes(StandardCharsets.UTF_8)));
-
-    Map<String,String> opts = Collections.emptyMap();
-
-    try {
-      doTableFateOperation(tableName, AccumuloException.class, FateOperation.TABLE_IMPORT, args, opts);
-    } catch (TableNotFoundException e) {
-      // should not happen
-      throw new AssertionError(e);
-    }
-
-  }
-
-  @Override
-  public void exportTable(String tableName, String exportDir) throws TableNotFoundException, AccumuloException, AccumuloSecurityException {
-    checkArgument(tableName != null, "tableName is null");
-    checkArgument(exportDir != null, "exportDir is null");
-
-    List<ByteBuffer> args = Arrays.asList(ByteBuffer.wrap(tableName.getBytes(StandardCharsets.UTF_8)), ByteBuffer.wrap(exportDir.getBytes(StandardCharsets.UTF_8)));
-
-    Map<String,String> opts = Collections.emptyMap();
-
-    try {
-      doTableFateOperation(tableName, TableNotFoundException.class, FateOperation.TABLE_EXPORT, args, opts);
-    } catch (TableExistsException e) {
-      // should not happen
-      throw new AssertionError(e);
-    }
-  }
-
-  @Override
-  public boolean testClassLoad(final String tableName, final String className, final String asTypeName) throws TableNotFoundException, AccumuloException,
-      AccumuloSecurityException {
-    checkArgument(tableName != null, "tableName is null");
-    checkArgument(className != null, "className is null");
-    checkArgument(asTypeName != null, "asTypeName is null");
-
-    try {
-      return ServerClient.executeRaw(instance, new ClientExecReturn<Boolean,ClientService.Client>() {
-        @Override
-        public Boolean execute(ClientService.Client client) throws Exception {
-          return client.checkTableClass(Tracer.traceInfo(), credentials.toThrift(instance), tableName, className, asTypeName);
-        }
-      });
-    } catch (ThriftTableOperationException e) {
-      switch (e.getType()) {
-        case NOTFOUND:
-          throw new TableNotFoundException(e);
-        case NAMESPACE_NOTFOUND:
-          throw new TableNotFoundException(tableName, new NamespaceNotFoundException(e));
-        default:
-          throw new AccumuloException(e.description, e);
-      }
-    } catch (ThriftSecurityException e) {
-      throw new AccumuloSecurityException(e.user, e.code, e);
-    } catch (AccumuloException e) {
-      throw e;
-    } catch (Exception e) {
-      throw new AccumuloException(e);
-    }
-  }
-
-  @Override
-  public void attachIterator(String tableName, IteratorSetting setting, EnumSet<IteratorScope> scopes) throws AccumuloSecurityException, AccumuloException,
-      TableNotFoundException {
-    testClassLoad(tableName, setting.getIteratorClass(), SortedKeyValueIterator.class.getName());
-    super.attachIterator(tableName, setting, scopes);
-  }
-
-  @Override
-  public int addConstraint(String tableName, String constraintClassName) throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
-    testClassLoad(tableName, constraintClassName, Constraint.class.getName());
-    return super.addConstraint(tableName, constraintClassName);
-  }
-
-  private void doTableFateOperation(String tableName, Class<? extends Exception> namespaceNotFoundExceptionClass, FateOperation op, List<ByteBuffer> args,
-      Map<String,String> opts) throws AccumuloSecurityException, AccumuloException, TableExistsException, TableNotFoundException {
-    try {
-      doFateOperation(op, args, opts);
-    } catch (NamespaceExistsException e) {
-      // should not happen
-      throw new AssertionError(e);
-    } catch (NamespaceNotFoundException e) {
-      if (namespaceNotFoundExceptionClass == null) {
-        // should not happen
-        throw new AssertionError(e);
-      } else if (AccumuloException.class.isAssignableFrom(namespaceNotFoundExceptionClass)) {
-        throw new AccumuloException("Cannot create table in non-existent namespace", e);
-      } else if (TableNotFoundException.class.isAssignableFrom(namespaceNotFoundExceptionClass)) {
-        throw new TableNotFoundException(null, tableName, "Namespace not found", e);
-      } else {
-        // should not happen
-        throw new AssertionError(e);
-      }
-    }
-  }
-
-  /**
-   * @param instance
-   *          the connection information for this instance
-   * @param credentials
-   *          the username/password for this connection
-   */
-  public TableOperationsImpl(Instance instance, TCredentials credentials) {
-    this(instance, Credentials.fromThrift(credentials));
-  }
-}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/f277fea9/core/src/main/java/org/apache/accumulo/core/client/impl/ConnectorImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/ConnectorImpl.java b/core/src/main/java/org/apache/accumulo/core/client/impl/ConnectorImpl.java
index 6be6273..0df35f6 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/impl/ConnectorImpl.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/ConnectorImpl.java
@@ -37,10 +37,6 @@ import org.apache.accumulo.core.client.admin.InstanceOperations;
 import org.apache.accumulo.core.client.admin.NamespaceOperations;
 import org.apache.accumulo.core.client.admin.SecurityOperations;
 import org.apache.accumulo.core.client.admin.TableOperations;
-import org.apache.accumulo.core.client.impl.InstanceOperationsImpl;
-import org.apache.accumulo.core.client.impl.NamespaceOperationsImpl;
-import org.apache.accumulo.core.client.impl.SecurityOperationsImpl;
-import org.apache.accumulo.core.client.impl.TableOperationsImpl;
 import org.apache.accumulo.core.client.impl.thrift.ClientService;
 import org.apache.accumulo.core.client.impl.thrift.SecurityErrorCode;
 import org.apache.accumulo.core.master.state.tables.TableState;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/f277fea9/core/src/main/java/org/apache/accumulo/core/client/impl/InstanceOperationsImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/InstanceOperationsImpl.java b/core/src/main/java/org/apache/accumulo/core/client/impl/InstanceOperationsImpl.java
index 12e4912..6fb2a41 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/impl/InstanceOperationsImpl.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/InstanceOperationsImpl.java
@@ -16,6 +16,9 @@
  */
 package org.apache.accumulo.core.client.impl;
 
+import static com.google.common.base.Preconditions.checkArgument;
+
+import java.nio.charset.StandardCharsets;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.List;
@@ -25,10 +28,10 @@ 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.Instance;
-import org.apache.accumulo.core.client.TableNotFoundException;
-import org.apache.accumulo.core.client.admin.ActiveScan;
 import org.apache.accumulo.core.client.admin.ActiveCompaction;
+import org.apache.accumulo.core.client.admin.ActiveScan;
 import org.apache.accumulo.core.client.admin.InstanceOperations;
+import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.client.impl.thrift.ClientService;
 import org.apache.accumulo.core.client.impl.thrift.ConfigurationType;
 import org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException;
@@ -37,7 +40,6 @@ import org.apache.accumulo.core.security.Credentials;
 import org.apache.accumulo.core.tabletserver.thrift.TabletClientService;
 import org.apache.accumulo.core.tabletserver.thrift.TabletClientService.Client;
 import org.apache.accumulo.core.util.AddressUtil;
-import org.apache.accumulo.core.util.ArgumentChecker;
 import org.apache.accumulo.core.util.ThriftUtil;
 import org.apache.accumulo.core.zookeeper.ZooUtil;
 import org.apache.accumulo.fate.zookeeper.ZooCache;
@@ -60,14 +62,16 @@ public class InstanceOperationsImpl implements InstanceOperations {
    *          the Credential, containing principal and Authentication Token
    */
   public InstanceOperationsImpl(Instance instance, Credentials credentials) {
-    ArgumentChecker.notNull(instance, credentials);
+    checkArgument(instance != null, "instance is null");
+    checkArgument(credentials != null, "credentials is null");
     this.instance = instance;
     this.credentials = credentials;
   }
 
   @Override
   public void setProperty(final String property, final String value) throws AccumuloException, AccumuloSecurityException {
-    ArgumentChecker.notNull(property, value);
+    checkArgument(property != null, "property is null");
+    checkArgument(value != null, "value is null");
     MasterClient.execute(instance, new ClientExec<MasterClientService.Client>() {
       @Override
       public void execute(MasterClientService.Client client) throws Exception {
@@ -78,7 +82,7 @@ public class InstanceOperationsImpl implements InstanceOperations {
 
   @Override
   public void removeProperty(final String property) throws AccumuloException, AccumuloSecurityException {
-    ArgumentChecker.notNull(property);
+    checkArgument(property != null, "property is null");
     MasterClient.execute(instance, new ClientExec<MasterClientService.Client>() {
       @Override
       public void execute(MasterClientService.Client client) throws Exception {
@@ -118,7 +122,7 @@ public class InstanceOperationsImpl implements InstanceOperations {
         List<String> copy = new ArrayList<String>(children);
         Collections.sort(copy);
         byte[] data = cache.get(path + "/" + candidate + "/" + copy.get(0));
-        if (data != null && !"master".equals(new String(data, Constants.UTF8))) {
+        if (data != null && !"master".equals(new String(data, StandardCharsets.UTF_8))) {
           results.add(candidate);
         }
       }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/f277fea9/core/src/main/java/org/apache/accumulo/core/client/impl/SecurityOperationsImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/SecurityOperationsImpl.java b/core/src/main/java/org/apache/accumulo/core/client/impl/SecurityOperationsImpl.java
index 6d8c59b..e9c057e 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/impl/SecurityOperationsImpl.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/SecurityOperationsImpl.java
@@ -16,6 +16,7 @@
  */
 package org.apache.accumulo.core.client.impl;
 
+import static com.google.common.base.Preconditions.checkArgument;
 import java.nio.ByteBuffer;
 import java.util.Set;
 
@@ -35,7 +36,6 @@ import org.apache.accumulo.core.security.Credentials;
 import org.apache.accumulo.core.security.NamespacePermission;
 import org.apache.accumulo.core.security.SystemPermission;
 import org.apache.accumulo.core.security.TablePermission;
-import org.apache.accumulo.core.util.ArgumentChecker;
 import org.apache.accumulo.core.util.ByteBufferUtil;
 import org.apache.accumulo.trace.instrument.Tracer;
 
@@ -85,7 +85,8 @@ public class SecurityOperationsImpl implements SecurityOperations {
   }
 
   public SecurityOperationsImpl(Instance instance, Credentials credentials) {
-    ArgumentChecker.notNull(instance, credentials);
+    checkArgument(instance != null, "instance is null");
+    checkArgument(credentials != null, "credentials is null");
     this.instance = instance;
     this.credentials = credentials;
   }
@@ -99,7 +100,8 @@ public class SecurityOperationsImpl implements SecurityOperations {
 
   @Override
   public void createLocalUser(final String principal, final PasswordToken password) throws AccumuloException, AccumuloSecurityException {
-    ArgumentChecker.notNull(principal, password);
+    checkArgument(principal != null, "principal is null");
+    checkArgument(password != null, "password is null");
     execute(new ClientExec<ClientService.Client>() {
       @Override
       public void execute(ClientService.Client client) throws Exception {
@@ -116,7 +118,7 @@ public class SecurityOperationsImpl implements SecurityOperations {
 
   @Override
   public void dropLocalUser(final String principal) throws AccumuloException, AccumuloSecurityException {
-    ArgumentChecker.notNull(principal);
+    checkArgument(principal != null, "principal is null");
     execute(new ClientExec<ClientService.Client>() {
       @Override
       public void execute(ClientService.Client client) throws Exception {
@@ -133,7 +135,8 @@ public class SecurityOperationsImpl implements SecurityOperations {
 
   @Override
   public boolean authenticateUser(final String principal, final AuthenticationToken token) throws AccumuloException, AccumuloSecurityException {
-    ArgumentChecker.notNull(principal, token);
+    checkArgument(principal != null, "principal is null");
+    checkArgument(token != null, "token is null");
     final Credentials toAuth = new Credentials(principal, token);
     return execute(new ClientExecReturn<Boolean,ClientService.Client>() {
       @Override
@@ -151,7 +154,8 @@ public class SecurityOperationsImpl implements SecurityOperations {
 
   @Override
   public void changeLocalUserPassword(final String principal, final PasswordToken token) throws AccumuloException, AccumuloSecurityException {
-    ArgumentChecker.notNull(principal, token);
+    checkArgument(principal != null, "principal is null");
+    checkArgument(token != null, "token is null");
     final Credentials toChange = new Credentials(principal, token);
     execute(new ClientExec<ClientService.Client>() {
       @Override
@@ -166,7 +170,8 @@ public class SecurityOperationsImpl implements SecurityOperations {
 
   @Override
   public void changeUserAuthorizations(final String principal, final Authorizations authorizations) throws AccumuloException, AccumuloSecurityException {
-    ArgumentChecker.notNull(principal, authorizations);
+    checkArgument(principal != null, "principal is null");
+    checkArgument(authorizations != null, "authorizations is null");
     execute(new ClientExec<ClientService.Client>() {
       @Override
       public void execute(ClientService.Client client) throws Exception {
@@ -178,7 +183,7 @@ public class SecurityOperationsImpl implements SecurityOperations {
 
   @Override
   public Authorizations getUserAuthorizations(final String principal) throws AccumuloException, AccumuloSecurityException {
-    ArgumentChecker.notNull(principal);
+    checkArgument(principal != null, "principal is null");
     return execute(new ClientExecReturn<Authorizations,ClientService.Client>() {
       @Override
       public Authorizations execute(ClientService.Client client) throws Exception {
@@ -189,7 +194,8 @@ public class SecurityOperationsImpl implements SecurityOperations {
 
   @Override
   public boolean hasSystemPermission(final String principal, final SystemPermission perm) throws AccumuloException, AccumuloSecurityException {
-    ArgumentChecker.notNull(principal, perm);
+    checkArgument(principal != null, "principal is null");
+    checkArgument(perm != null, "perm is null");
     return execute(new ClientExecReturn<Boolean,ClientService.Client>() {
       @Override
       public Boolean execute(ClientService.Client client) throws Exception {
@@ -200,7 +206,9 @@ public class SecurityOperationsImpl implements SecurityOperations {
 
   @Override
   public boolean hasTablePermission(final String principal, final String table, final TablePermission perm) throws AccumuloException, AccumuloSecurityException {
-    ArgumentChecker.notNull(principal, table, perm);
+    checkArgument(principal != null, "principal is null");
+    checkArgument(table != null, "table is null");
+    checkArgument(perm != null, "perm is null");
     try {
       return execute(new ClientExecReturn<Boolean,ClientService.Client>() {
         @Override
@@ -219,7 +227,9 @@ public class SecurityOperationsImpl implements SecurityOperations {
   @Override
   public boolean hasNamespacePermission(final String principal, final String namespace, final NamespacePermission permission) throws AccumuloException,
       AccumuloSecurityException {
-    ArgumentChecker.notNull(principal, namespace, permission);
+    checkArgument(principal != null, "principal is null");
+    checkArgument(namespace != null, "namespace is null");
+    checkArgument(permission != null, "permission is null");
     return execute(new ClientExecReturn<Boolean,ClientService.Client>() {
       @Override
       public Boolean execute(ClientService.Client client) throws Exception {
@@ -230,7 +240,8 @@ public class SecurityOperationsImpl implements SecurityOperations {
 
   @Override
   public void grantSystemPermission(final String principal, final SystemPermission permission) throws AccumuloException, AccumuloSecurityException {
-    ArgumentChecker.notNull(principal, permission);
+    checkArgument(principal != null, "principal is null");
+    checkArgument(permission != null, "permission is null");
     execute(new ClientExec<ClientService.Client>() {
       @Override
       public void execute(ClientService.Client client) throws Exception {
@@ -242,7 +253,9 @@ public class SecurityOperationsImpl implements SecurityOperations {
   @Override
   public void grantTablePermission(final String principal, final String table, final TablePermission permission) throws AccumuloException,
       AccumuloSecurityException {
-    ArgumentChecker.notNull(principal, table, permission);
+    checkArgument(principal != null, "principal is null");
+    checkArgument(table != null, "table is null");
+    checkArgument(permission != null, "permission is null");
     try {
       execute(new ClientExec<ClientService.Client>() {
         @Override
@@ -261,7 +274,9 @@ public class SecurityOperationsImpl implements SecurityOperations {
   @Override
   public void grantNamespacePermission(final String principal, final String namespace, final NamespacePermission permission) throws AccumuloException,
       AccumuloSecurityException {
-    ArgumentChecker.notNull(principal, namespace, permission);
+    checkArgument(principal != null, "principal is null");
+    checkArgument(namespace != null, "namespace is null");
+    checkArgument(permission != null, "permission is null");
     execute(new ClientExec<ClientService.Client>() {
       @Override
       public void execute(ClientService.Client client) throws Exception {
@@ -272,7 +287,8 @@ public class SecurityOperationsImpl implements SecurityOperations {
 
   @Override
   public void revokeSystemPermission(final String principal, final SystemPermission permission) throws AccumuloException, AccumuloSecurityException {
-    ArgumentChecker.notNull(principal, permission);
+    checkArgument(principal != null, "principal is null");
+    checkArgument(permission != null, "permission is null");
     execute(new ClientExec<ClientService.Client>() {
       @Override
       public void execute(ClientService.Client client) throws Exception {
@@ -284,7 +300,9 @@ public class SecurityOperationsImpl implements SecurityOperations {
   @Override
   public void revokeTablePermission(final String principal, final String table, final TablePermission permission) throws AccumuloException,
       AccumuloSecurityException {
-    ArgumentChecker.notNull(principal, table, permission);
+    checkArgument(principal != null, "principal is null");
+    checkArgument(table != null, "table is null");
+    checkArgument(permission != null, "permission is null");
     try {
       execute(new ClientExec<ClientService.Client>() {
         @Override
@@ -303,7 +321,9 @@ public class SecurityOperationsImpl implements SecurityOperations {
   @Override
   public void revokeNamespacePermission(final String principal, final String namespace, final NamespacePermission permission) throws AccumuloException,
       AccumuloSecurityException {
-    ArgumentChecker.notNull(principal, namespace, permission);
+    checkArgument(principal != null, "principal is null");
+    checkArgument(namespace != null, "namespace is null");
+    checkArgument(permission != null, "permission is null");
     execute(new ClientExec<ClientService.Client>() {
       @Override
       public void execute(ClientService.Client client) throws Exception {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/f277fea9/core/src/main/java/org/apache/accumulo/core/client/impl/TableOperationsHelper.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/TableOperationsHelper.java b/core/src/main/java/org/apache/accumulo/core/client/impl/TableOperationsHelper.java
index 58f1a42..cea934d 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/impl/TableOperationsHelper.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/TableOperationsHelper.java
@@ -16,6 +16,7 @@
  */
 package org.apache.accumulo.core.client.impl;
 
+import static com.google.common.base.Preconditions.checkArgument;
 import java.util.EnumSet;
 import java.util.HashMap;
 import java.util.Map;
@@ -30,7 +31,6 @@ import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.client.admin.TableOperations;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.iterators.IteratorUtil.IteratorScope;
-import org.apache.accumulo.core.util.ArgumentChecker;
 
 public abstract class TableOperationsHelper implements TableOperations {
 
@@ -42,7 +42,9 @@ public abstract class TableOperationsHelper implements TableOperations {
   @Override
   public void attachIterator(String tableName, IteratorSetting setting, EnumSet<IteratorScope> scopes) throws AccumuloSecurityException, AccumuloException,
       TableNotFoundException {
-    ArgumentChecker.notNull(tableName, setting, scopes);
+    checkArgument(tableName != null, "tableName is null");
+    checkArgument(setting != null, "setting is null");
+    checkArgument(scopes != null, "scopes is null");
     checkIteratorConflicts(tableName, setting, scopes);
     for (IteratorScope scope : scopes) {
       String root = String.format("%s%s.%s", Property.TABLE_ITERATOR_PREFIX, scope.name().toLowerCase(), setting.getName());
@@ -72,7 +74,9 @@ public abstract class TableOperationsHelper implements TableOperations {
   @Override
   public IteratorSetting getIteratorSetting(String tableName, String name, IteratorScope scope) throws AccumuloSecurityException, AccumuloException,
       TableNotFoundException {
-    ArgumentChecker.notNull(tableName, name, scope);
+    checkArgument(tableName != null, "tableName is null");
+    checkArgument(name != null, "name is null");
+    checkArgument(scope != null, "scope is null");
     int priority = -1;
     String classname = null;
     Map<String,String> settings = new HashMap<String,String>();
@@ -117,7 +121,9 @@ public abstract class TableOperationsHelper implements TableOperations {
 
   @Override
   public void checkIteratorConflicts(String tableName, IteratorSetting setting, EnumSet<IteratorScope> scopes) throws AccumuloException, TableNotFoundException {
-    ArgumentChecker.notNull(tableName, setting, scopes);
+    checkArgument(tableName != null, "tableName is null");
+    checkArgument(setting != null, "setting is null");
+    checkArgument(scopes != null, "scopes is null");
     for (IteratorScope scope : scopes) {
       String scopeStr = String.format("%s%s", Property.TABLE_ITERATOR_PREFIX, scope.name().toLowerCase());
       String nameStr = String.format("%s.%s", scopeStr, setting.getName());


[11/17] git commit: Merge branch '1.6.0-SNAPSHOT'

Posted by bu...@apache.org.
Merge branch '1.6.0-SNAPSHOT'

Conflicts:
	core/src/main/java/org/apache/accumulo/core/client/admin/InstanceOperationsImpl.java
	core/src/main/java/org/apache/accumulo/core/client/admin/SecurityOperationsImpl.java
	core/src/main/java/org/apache/accumulo/core/client/admin/TableOperationsHelper.java
	core/src/main/java/org/apache/accumulo/core/client/admin/TableOperationsImpl.java
	shell/src/main/java/org/apache/accumulo/shell/mock/MockShell.java


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

Branch: refs/heads/master
Commit: 592c8b062404da9f4f2faa333d8c393b2f6187f1
Parents: 6ac5c5f 9bdad26
Author: Sean Busbey <bu...@cloudera.com>
Authored: Fri Apr 25 12:10:46 2014 -0500
Committer: Sean Busbey <bu...@cloudera.com>
Committed: Fri Apr 25 14:47:15 2014 -0500

----------------------------------------------------------------------
 .../core/client/admin/ActiveCompaction.java     |   79 +-
 .../accumulo/core/client/admin/ActiveScan.java  |  112 +-
 .../client/admin/InstanceOperationsImpl.java    |   27 +-
 .../client/admin/NamespaceOperationsHelper.java |  214 ---
 .../client/admin/NamespaceOperationsImpl.java   |  254 ---
 .../client/admin/SecurityOperationsImpl.java    |   78 +-
 .../core/client/admin/TableOperations.java      |   15 +-
 .../client/admin/TableOperationsHelper.java     |    6 +-
 .../core/client/admin/TableOperationsImpl.java  |   59 +-
 .../core/client/impl/ActiveCompactionImpl.java  |  109 ++
 .../core/client/impl/ActiveScanImpl.java        |  146 ++
 .../core/client/impl/ConnectorImpl.java         |    8 +-
 .../client/impl/InstanceOperationsImpl.java     |  209 +++
 .../client/impl/NamespaceOperationsHelper.java  |  214 +++
 .../client/impl/NamespaceOperationsImpl.java    |  250 +++
 .../client/impl/SecurityOperationsImpl.java     |  331 ++++
 .../core/client/impl/TableOperationsHelper.java |  199 +++
 .../core/client/impl/TableOperationsImpl.java   | 1589 ++++++++++++++++++
 .../core/client/mock/MockConnector.java         |    6 +-
 .../client/mock/MockInstanceOperations.java     |   70 +-
 .../client/mock/MockInstanceOperationsImpl.java |   87 +
 .../client/mock/MockNamespaceOperations.java    |    4 +-
 .../client/mock/MockSecurityOperations.java     |  209 +--
 .../client/mock/MockSecurityOperationsImpl.java |  225 +++
 .../core/client/mock/MockTableOperations.java   |  431 +----
 .../client/mock/MockTableOperationsImpl.java    |  447 +++++
 .../core/client/mock/MockTabletLocator.java     |   45 +-
 .../client/mock/impl/MockTabletLocator.java     |   70 +
 .../client/admin/TableOperationsHelperTest.java |   90 +-
 .../client/impl/TableOperationsHelperTest.java  |  305 ++++
 .../mapreduce/lib/impl/InputConfigurator.java   |    2 +-
 .../server/security/SecurityOperation.java      |    2 +-
 .../accumulo/master/FateServiceHandler.java     |    2 +-
 .../accumulo/master/tableOps/ImportTable.java   |    2 +-
 .../apache/accumulo/shell/mock/MockShell.java   |   41 +-
 .../japi-compliance/japi-accumulo-1.5.xml       |    1 -
 .../japi-compliance/japi-accumulo-1.6.xml       |    3 +-
 37 files changed, 4351 insertions(+), 1590 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/592c8b06/core/src/main/java/org/apache/accumulo/core/client/admin/InstanceOperationsImpl.java
----------------------------------------------------------------------
diff --cc core/src/main/java/org/apache/accumulo/core/client/admin/InstanceOperationsImpl.java
index df2bb55,11e389f..d3a6285
--- a/core/src/main/java/org/apache/accumulo/core/client/admin/InstanceOperationsImpl.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/admin/InstanceOperationsImpl.java
@@@ -16,39 -16,9 +16,30 @@@
   */
  package org.apache.accumulo.core.client.admin;
  
 +import static com.google.common.base.Preconditions.checkArgument;
 +
 +import java.nio.charset.StandardCharsets;
 +import java.util.ArrayList;
 +import java.util.Collections;
 +import java.util.List;
 +import java.util.Map;
 +
 +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.Instance;
- import org.apache.accumulo.core.client.TableNotFoundException;
- import org.apache.accumulo.core.client.impl.ClientExec;
- import org.apache.accumulo.core.client.impl.ClientExecReturn;
- import org.apache.accumulo.core.client.impl.MasterClient;
- import org.apache.accumulo.core.client.impl.ServerClient;
- import org.apache.accumulo.core.client.impl.ServerConfigurationUtil;
- import org.apache.accumulo.core.client.impl.thrift.ClientService;
- import org.apache.accumulo.core.client.impl.thrift.ConfigurationType;
- import org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException;
- import org.apache.accumulo.core.master.thrift.MasterClientService;
  import org.apache.accumulo.core.security.Credentials;
 +import org.apache.accumulo.core.tabletserver.thrift.TabletClientService;
 +import org.apache.accumulo.core.tabletserver.thrift.TabletClientService.Client;
 +import org.apache.accumulo.core.util.AddressUtil;
 +import org.apache.accumulo.core.util.ThriftUtil;
 +import org.apache.accumulo.core.zookeeper.ZooUtil;
 +import org.apache.accumulo.fate.zookeeper.ZooCache;
 +import org.apache.accumulo.trace.instrument.Tracer;
 +import org.apache.thrift.TException;
 +import org.apache.thrift.transport.TTransport;
 +import org.apache.thrift.transport.TTransportException;
+ import org.apache.accumulo.core.security.thrift.TCredentials;
  
  /**
   * Provides a class for administering the accumulo instance
@@@ -63,153 -33,17 +54,163 @@@ public class InstanceOperationsImpl ext
     * @param credentials
     *          the Credential, containing principal and Authentication Token
     */
 -  private InstanceOperationsImpl(Instance instance, Credentials credentials) {
 -    super(instance, credentials);
 +  public InstanceOperationsImpl(Instance instance, Credentials credentials) {
 +    checkArgument(instance != null, "instance is null");
 +    checkArgument(credentials != null, "credentials is null");
 +    this.instance = instance;
 +    this.credentials = credentials;
 +  }
 +  
 +  @Override
 +  public void setProperty(final String property, final String value) throws AccumuloException, AccumuloSecurityException {
 +    checkArgument(property != null, "property is null");
 +    checkArgument(value != null, "value is null");
 +    MasterClient.execute(instance, new ClientExec<MasterClientService.Client>() {
 +      @Override
 +      public void execute(MasterClientService.Client client) throws Exception {
 +        client.setSystemProperty(Tracer.traceInfo(), credentials.toThrift(instance), property, value);
 +      }
 +    });
 +  }
 +  
 +  @Override
 +  public void removeProperty(final String property) throws AccumuloException, AccumuloSecurityException {
 +    checkArgument(property != null, "property is null");
 +    MasterClient.execute(instance, new ClientExec<MasterClientService.Client>() {
 +      @Override
 +      public void execute(MasterClientService.Client client) throws Exception {
 +        client.removeSystemProperty(Tracer.traceInfo(), credentials.toThrift(instance), property);
 +      }
 +    });
 +  }
 +  
 +  @Override
 +  public Map<String,String> getSystemConfiguration() throws AccumuloException, AccumuloSecurityException {
 +    return ServerClient.execute(instance, new ClientExecReturn<Map<String,String>,ClientService.Client>() {
 +      @Override
 +      public Map<String,String> execute(ClientService.Client client) throws Exception {
 +        return client.getConfiguration(Tracer.traceInfo(), credentials.toThrift(instance), ConfigurationType.CURRENT);
 +      }
 +    });
 +  }
 +  
 +  @Override
 +  public Map<String,String> getSiteConfiguration() throws AccumuloException, AccumuloSecurityException {
 +    return ServerClient.execute(instance, new ClientExecReturn<Map<String,String>,ClientService.Client>() {
 +      @Override
 +      public Map<String,String> execute(ClientService.Client client) throws Exception {
 +        return client.getConfiguration(Tracer.traceInfo(), credentials.toThrift(instance), ConfigurationType.SITE);
 +      }
 +    });
 +  }
 +  
 +  @Override
 +  public List<String> getTabletServers() {
 +    ZooCache cache = ZooCache.getInstance(instance.getZooKeepers(), instance.getZooKeepersSessionTimeOut());
 +    String path = ZooUtil.getRoot(instance) + Constants.ZTSERVERS;
 +    List<String> results = new ArrayList<String>();
 +    for (String candidate : cache.getChildren(path)) {
 +      List<String> children = cache.getChildren(path + "/" + candidate);
 +      if (children != null && children.size() > 0) {
 +        List<String> copy = new ArrayList<String>(children);
 +        Collections.sort(copy);
 +        byte[] data = cache.get(path + "/" + candidate + "/" + copy.get(0));
 +        if (data != null && !"master".equals(new String(data, StandardCharsets.UTF_8))) {
 +          results.add(candidate);
 +        }
 +      }
 +    }
 +    return results;
 +  }
 +  
 +  @Override
 +  public List<ActiveScan> getActiveScans(String tserver) throws AccumuloException, AccumuloSecurityException {
 +    Client client = null;
 +    try {
 +      client = ThriftUtil.getTServerClient(tserver, ServerConfigurationUtil.getConfiguration(instance));
 +      
 +      List<ActiveScan> as = new ArrayList<ActiveScan>();
 +      for (org.apache.accumulo.core.tabletserver.thrift.ActiveScan activeScan : client.getActiveScans(Tracer.traceInfo(), credentials.toThrift(instance))) {
 +        try {
 +          as.add(new ActiveScan(instance, activeScan));
 +        } catch (TableNotFoundException e) {
 +          throw new AccumuloException(e);
 +        }
 +      }
 +      return as;
 +    } catch (TTransportException e) {
 +      throw new AccumuloException(e);
 +    } catch (ThriftSecurityException e) {
 +      throw new AccumuloSecurityException(e.user, e.code, e);
 +    } catch (TException e) {
 +      throw new AccumuloException(e);
 +    } finally {
 +      if (client != null)
 +        ThriftUtil.returnClient(client);
 +    }
 +  }
 +  
 +  @Override
 +  public boolean testClassLoad(final String className, final String asTypeName) throws AccumuloException, AccumuloSecurityException {
 +    return ServerClient.execute(instance, new ClientExecReturn<Boolean,ClientService.Client>() {
 +      @Override
 +      public Boolean execute(ClientService.Client client) throws Exception {
 +        return client.checkClass(Tracer.traceInfo(), credentials.toThrift(instance), className, asTypeName);
 +      }
 +    });
 +  }
 +  
 +  @Override
 +  public List<ActiveCompaction> getActiveCompactions(String tserver) throws AccumuloException, AccumuloSecurityException {
 +    Client client = null;
 +    try {
 +      client = ThriftUtil.getTServerClient(tserver, ServerConfigurationUtil.getConfiguration(instance));
 +      
 +      List<ActiveCompaction> as = new ArrayList<ActiveCompaction>();
 +      for (org.apache.accumulo.core.tabletserver.thrift.ActiveCompaction activeCompaction : client.getActiveCompactions(Tracer.traceInfo(),
 +          credentials.toThrift(instance))) {
 +        as.add(new ActiveCompaction(instance, activeCompaction));
 +      }
 +      return as;
 +    } catch (TTransportException e) {
 +      throw new AccumuloException(e);
 +    } catch (ThriftSecurityException e) {
 +      throw new AccumuloSecurityException(e.user, e.code, e);
 +    } catch (TException e) {
 +      throw new AccumuloException(e);
 +    } finally {
 +      if (client != null)
 +        ThriftUtil.returnClient(client);
 +    }
 +  }
 +  
 +  @Override
 +  public void ping(String tserver) throws AccumuloException {
 +    TTransport transport = null;
 +    try {
 +      transport = ThriftUtil.createTransport(AddressUtil.parseAddress(tserver, false), ServerConfigurationUtil.getConfiguration(instance));
 +      TabletClientService.Client client = ThriftUtil.createClient(new TabletClientService.Client.Factory(), transport);
 +      client.getTabletServerStatus(Tracer.traceInfo(), credentials.toThrift(instance));
 +    } catch (TTransportException e) {
 +      throw new AccumuloException(e);
 +    } catch (ThriftSecurityException e) {
 +      throw new AccumuloException(e);
 +    } catch (TException e) {
 +      throw new AccumuloException(e);
 +    } finally {
 +      if (transport != null) {
 +        transport.close();
 +      }
 +    }
    }
+   
+   /**
+    * @param instance
+    *          the connection information for this instance
+    * @param credentials
+    *          the Credential, containing principal and Authentication Token
+    */
+   public InstanceOperationsImpl(Instance instance, TCredentials credentials) {
+     this(instance, Credentials.fromThrift(credentials));
+   }
  }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/592c8b06/core/src/main/java/org/apache/accumulo/core/client/admin/SecurityOperationsImpl.java
----------------------------------------------------------------------
diff --cc core/src/main/java/org/apache/accumulo/core/client/admin/SecurityOperationsImpl.java
index 20c3768,875cc72..3d647a0
--- a/core/src/main/java/org/apache/accumulo/core/client/admin/SecurityOperationsImpl.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/admin/SecurityOperationsImpl.java
@@@ -16,338 -16,21 +16,276 @@@
   */
  package org.apache.accumulo.core.client.admin;
  
 +import static com.google.common.base.Preconditions.checkArgument;
 +import java.nio.ByteBuffer;
 +import java.util.Set;
 +
 +import org.apache.accumulo.core.client.AccumuloException;
 +import org.apache.accumulo.core.client.AccumuloSecurityException;
  import org.apache.accumulo.core.client.Instance;
- import org.apache.accumulo.core.client.impl.ClientExec;
- import org.apache.accumulo.core.client.impl.ClientExecReturn;
- import org.apache.accumulo.core.client.impl.ServerClient;
- import org.apache.accumulo.core.client.impl.thrift.ClientService;
- import org.apache.accumulo.core.client.impl.thrift.SecurityErrorCode;
- import org.apache.accumulo.core.client.impl.thrift.TableOperationExceptionType;
- import org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException;
- import org.apache.accumulo.core.client.impl.thrift.ThriftTableOperationException;
- import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
- import org.apache.accumulo.core.client.security.tokens.PasswordToken;
- import org.apache.accumulo.core.security.Authorizations;
  import org.apache.accumulo.core.security.Credentials;
 +import org.apache.accumulo.core.security.NamespacePermission;
 +import org.apache.accumulo.core.security.SystemPermission;
 +import org.apache.accumulo.core.security.TablePermission;
 +import org.apache.accumulo.core.util.ByteBufferUtil;
 +import org.apache.accumulo.trace.instrument.Tracer;
+ import org.apache.accumulo.core.security.thrift.TCredentials;
  
- public class SecurityOperationsImpl implements SecurityOperations {
- 
-   private Instance instance;
-   private Credentials credentials;
- 
-   private void execute(ClientExec<ClientService.Client> exec) throws AccumuloException, AccumuloSecurityException {
-     try {
-       ServerClient.executeRaw(instance, exec);
-     } catch (ThriftTableOperationException ttoe) {
-       // recast missing table
-       if (ttoe.getType() == TableOperationExceptionType.NOTFOUND)
-         throw new AccumuloSecurityException(null, SecurityErrorCode.TABLE_DOESNT_EXIST);
-       else if (ttoe.getType() == TableOperationExceptionType.NAMESPACE_NOTFOUND)
-         throw new AccumuloSecurityException(null, SecurityErrorCode.NAMESPACE_DOESNT_EXIST);
-       else
-         throw new AccumuloException(ttoe);
-     } catch (ThriftSecurityException e) {
-       throw new AccumuloSecurityException(e.user, e.code, e);
-     } catch (AccumuloException e) {
-       throw e;
-     } catch (Exception e) {
-       throw new AccumuloException(e);
-     }
-   }
- 
-   private <T> T execute(ClientExecReturn<T,ClientService.Client> exec) throws AccumuloException, AccumuloSecurityException {
-     try {
-       return ServerClient.executeRaw(instance, exec);
-     } catch (ThriftTableOperationException ttoe) {
-       // recast missing table
-       if (ttoe.getType() == TableOperationExceptionType.NOTFOUND)
-         throw new AccumuloSecurityException(null, SecurityErrorCode.TABLE_DOESNT_EXIST);
-       else if (ttoe.getType() == TableOperationExceptionType.NAMESPACE_NOTFOUND)
-         throw new AccumuloSecurityException(null, SecurityErrorCode.NAMESPACE_DOESNT_EXIST);
-       else
-         throw new AccumuloException(ttoe);
-     } catch (ThriftSecurityException e) {
-       throw new AccumuloSecurityException(e.user, e.code, e);
-     } catch (AccumuloException e) {
-       throw e;
-     } catch (Exception e) {
-       throw new AccumuloException(e);
-     }
-   }
+ /**
+  * @deprecated since 1.6.0; not intended for public api and you should not use it.
+  */
+ @Deprecated
+ public class SecurityOperationsImpl extends org.apache.accumulo.core.client.impl.SecurityOperationsImpl {
  
    public SecurityOperationsImpl(Instance instance, Credentials credentials) {
 -    super(instance, credentials);
 +    checkArgument(instance != null, "instance is null");
 +    checkArgument(credentials != null, "credentials is null");
 +    this.instance = instance;
 +    this.credentials = credentials;
 +  }
 +
 +  @Deprecated
 +  @Override
 +  public void createUser(String user, byte[] password, final Authorizations authorizations) throws AccumuloException, AccumuloSecurityException {
 +    createLocalUser(user, new PasswordToken(password));
 +    changeUserAuthorizations(user, authorizations);
 +  }
 +
 +  @Override
 +  public void createLocalUser(final String principal, final PasswordToken password) throws AccumuloException, AccumuloSecurityException {
 +    checkArgument(principal != null, "principal is null");
 +    checkArgument(password != null, "password is null");
 +    execute(new ClientExec<ClientService.Client>() {
 +      @Override
 +      public void execute(ClientService.Client client) throws Exception {
 +        client.createLocalUser(Tracer.traceInfo(), credentials.toThrift(instance), principal, ByteBuffer.wrap(password.getPassword()));
 +      }
 +    });
 +  }
 +
 +  @Deprecated
 +  @Override
 +  public void dropUser(final String user) throws AccumuloException, AccumuloSecurityException {
 +    dropLocalUser(user);
 +  }
 +
 +  @Override
 +  public void dropLocalUser(final String principal) throws AccumuloException, AccumuloSecurityException {
 +    checkArgument(principal != null, "principal is null");
 +    execute(new ClientExec<ClientService.Client>() {
 +      @Override
 +      public void execute(ClientService.Client client) throws Exception {
 +        client.dropLocalUser(Tracer.traceInfo(), credentials.toThrift(instance), principal);
 +      }
 +    });
 +  }
 +
 +  @Deprecated
 +  @Override
 +  public boolean authenticateUser(String user, byte[] password) throws AccumuloException, AccumuloSecurityException {
 +    return authenticateUser(user, new PasswordToken(password));
 +  }
 +
 +  @Override
 +  public boolean authenticateUser(final String principal, final AuthenticationToken token) throws AccumuloException, AccumuloSecurityException {
 +    checkArgument(principal != null, "principal is null");
 +    checkArgument(token != null, "token is null");
 +    final Credentials toAuth = new Credentials(principal, token);
 +    return execute(new ClientExecReturn<Boolean,ClientService.Client>() {
 +      @Override
 +      public Boolean execute(ClientService.Client client) throws Exception {
 +        return client.authenticateUser(Tracer.traceInfo(), credentials.toThrift(instance), toAuth.toThrift(instance));
 +      }
 +    });
 +  }
 +
 +  @Override
 +  @Deprecated
 +  public void changeUserPassword(String user, byte[] password) throws AccumuloException, AccumuloSecurityException {
 +    changeLocalUserPassword(user, new PasswordToken(password));
 +  }
 +
 +  @Override
 +  public void changeLocalUserPassword(final String principal, final PasswordToken token) throws AccumuloException, AccumuloSecurityException {
 +    checkArgument(principal != null, "principal is null");
 +    checkArgument(token != null, "token is null");
 +    final Credentials toChange = new Credentials(principal, token);
 +    execute(new ClientExec<ClientService.Client>() {
 +      @Override
 +      public void execute(ClientService.Client client) throws Exception {
 +        client.changeLocalUserPassword(Tracer.traceInfo(), credentials.toThrift(instance), principal, ByteBuffer.wrap(token.getPassword()));
 +      }
 +    });
 +    if (this.credentials.getPrincipal().equals(principal)) {
 +      this.credentials = toChange;
 +    }
 +  }
 +
 +  @Override
 +  public void changeUserAuthorizations(final String principal, final Authorizations authorizations) throws AccumuloException, AccumuloSecurityException {
 +    checkArgument(principal != null, "principal is null");
 +    checkArgument(authorizations != null, "authorizations is null");
 +    execute(new ClientExec<ClientService.Client>() {
 +      @Override
 +      public void execute(ClientService.Client client) throws Exception {
 +        client.changeAuthorizations(Tracer.traceInfo(), credentials.toThrift(instance), principal,
 +            ByteBufferUtil.toByteBuffers(authorizations.getAuthorizations()));
 +      }
 +    });
 +  }
 +
 +  @Override
 +  public Authorizations getUserAuthorizations(final String principal) throws AccumuloException, AccumuloSecurityException {
 +    checkArgument(principal != null, "principal is null");
 +    return execute(new ClientExecReturn<Authorizations,ClientService.Client>() {
 +      @Override
 +      public Authorizations execute(ClientService.Client client) throws Exception {
 +        return new Authorizations(client.getUserAuthorizations(Tracer.traceInfo(), credentials.toThrift(instance), principal));
 +      }
 +    });
 +  }
 +
 +  @Override
 +  public boolean hasSystemPermission(final String principal, final SystemPermission perm) throws AccumuloException, AccumuloSecurityException {
 +    checkArgument(principal != null, "principal is null");
 +    checkArgument(perm != null, "perm is null");
 +    return execute(new ClientExecReturn<Boolean,ClientService.Client>() {
 +      @Override
 +      public Boolean execute(ClientService.Client client) throws Exception {
 +        return client.hasSystemPermission(Tracer.traceInfo(), credentials.toThrift(instance), principal, perm.getId());
 +      }
 +    });
 +  }
 +
 +  @Override
 +  public boolean hasTablePermission(final String principal, final String table, final TablePermission perm) throws AccumuloException, AccumuloSecurityException {
 +    checkArgument(principal != null, "principal is null");
 +    checkArgument(table != null, "table is null");
 +    checkArgument(perm != null, "perm is null");
 +    try {
 +      return execute(new ClientExecReturn<Boolean,ClientService.Client>() {
 +        @Override
 +        public Boolean execute(ClientService.Client client) throws Exception {
 +          return client.hasTablePermission(Tracer.traceInfo(), credentials.toThrift(instance), principal, table, perm.getId());
 +        }
 +      });
 +    } catch (AccumuloSecurityException e) {
 +      if (e.getSecurityErrorCode() == org.apache.accumulo.core.client.security.SecurityErrorCode.NAMESPACE_DOESNT_EXIST)
 +        throw new AccumuloSecurityException(null, SecurityErrorCode.TABLE_DOESNT_EXIST, e);
 +      else
 +        throw e;
 +    }
 +  }
 +
 +  @Override
 +  public boolean hasNamespacePermission(final String principal, final String namespace, final NamespacePermission permission) throws AccumuloException,
 +      AccumuloSecurityException {
 +    checkArgument(principal != null, "principal is null");
 +    checkArgument(namespace != null, "namespace is null");
 +    checkArgument(permission != null, "permission is null");
 +    return execute(new ClientExecReturn<Boolean,ClientService.Client>() {
 +      @Override
 +      public Boolean execute(ClientService.Client client) throws Exception {
 +        return client.hasNamespacePermission(Tracer.traceInfo(), credentials.toThrift(instance), principal, namespace, permission.getId());
 +      }
 +    });
 +  }
 +
 +  @Override
 +  public void grantSystemPermission(final String principal, final SystemPermission permission) throws AccumuloException, AccumuloSecurityException {
 +    checkArgument(principal != null, "principal is null");
 +    checkArgument(permission != null, "permission is null");
 +    execute(new ClientExec<ClientService.Client>() {
 +      @Override
 +      public void execute(ClientService.Client client) throws Exception {
 +        client.grantSystemPermission(Tracer.traceInfo(), credentials.toThrift(instance), principal, permission.getId());
 +      }
 +    });
 +  }
 +
 +  @Override
 +  public void grantTablePermission(final String principal, final String table, final TablePermission permission) throws AccumuloException,
 +      AccumuloSecurityException {
 +    checkArgument(principal != null, "principal is null");
 +    checkArgument(table != null, "table is null");
 +    checkArgument(permission != null, "permission is null");
 +    try {
 +      execute(new ClientExec<ClientService.Client>() {
 +        @Override
 +        public void execute(ClientService.Client client) throws Exception {
 +          client.grantTablePermission(Tracer.traceInfo(), credentials.toThrift(instance), principal, table, permission.getId());
 +        }
 +      });
 +    } catch (AccumuloSecurityException e) {
 +      if (e.getSecurityErrorCode() == org.apache.accumulo.core.client.security.SecurityErrorCode.NAMESPACE_DOESNT_EXIST)
 +        throw new AccumuloSecurityException(null, SecurityErrorCode.TABLE_DOESNT_EXIST, e);
 +      else
 +        throw e;
 +    }
 +  }
 +
 +  @Override
 +  public void grantNamespacePermission(final String principal, final String namespace, final NamespacePermission permission) throws AccumuloException,
 +      AccumuloSecurityException {
 +    checkArgument(principal != null, "principal is null");
 +    checkArgument(namespace != null, "namespace is null");
 +    checkArgument(permission != null, "permission is null");
 +    execute(new ClientExec<ClientService.Client>() {
 +      @Override
 +      public void execute(ClientService.Client client) throws Exception {
 +        client.grantNamespacePermission(Tracer.traceInfo(), credentials.toThrift(instance), principal, namespace, permission.getId());
 +      }
 +    });
 +  }
 +
 +  @Override
 +  public void revokeSystemPermission(final String principal, final SystemPermission permission) throws AccumuloException, AccumuloSecurityException {
 +    checkArgument(principal != null, "principal is null");
 +    checkArgument(permission != null, "permission is null");
 +    execute(new ClientExec<ClientService.Client>() {
 +      @Override
 +      public void execute(ClientService.Client client) throws Exception {
 +        client.revokeSystemPermission(Tracer.traceInfo(), credentials.toThrift(instance), principal, permission.getId());
 +      }
 +    });
 +  }
 +
 +  @Override
 +  public void revokeTablePermission(final String principal, final String table, final TablePermission permission) throws AccumuloException,
 +      AccumuloSecurityException {
 +    checkArgument(principal != null, "principal is null");
 +    checkArgument(table != null, "table is null");
 +    checkArgument(permission != null, "permission is null");
 +    try {
 +      execute(new ClientExec<ClientService.Client>() {
 +        @Override
 +        public void execute(ClientService.Client client) throws Exception {
 +          client.revokeTablePermission(Tracer.traceInfo(), credentials.toThrift(instance), principal, table, permission.getId());
 +        }
 +      });
 +    } catch (AccumuloSecurityException e) {
 +      if (e.getSecurityErrorCode() == org.apache.accumulo.core.client.security.SecurityErrorCode.NAMESPACE_DOESNT_EXIST)
 +        throw new AccumuloSecurityException(null, SecurityErrorCode.TABLE_DOESNT_EXIST, e);
 +      else
 +        throw e;
 +    }
 +  }
 +
 +  @Override
 +  public void revokeNamespacePermission(final String principal, final String namespace, final NamespacePermission permission) throws AccumuloException,
 +      AccumuloSecurityException {
 +    checkArgument(principal != null, "principal is null");
 +    checkArgument(namespace != null, "namespace is null");
 +    checkArgument(permission != null, "permission is null");
 +    execute(new ClientExec<ClientService.Client>() {
 +      @Override
 +      public void execute(ClientService.Client client) throws Exception {
 +        client.revokeNamespacePermission(Tracer.traceInfo(), credentials.toThrift(instance), principal, namespace, permission.getId());
 +      }
 +    });
    }
  
-   @Deprecated
-   @Override
-   public Set<String> listUsers() throws AccumuloException, AccumuloSecurityException {
-     return listLocalUsers();
-   }
- 
-   @Override
-   public Set<String> listLocalUsers() throws AccumuloException, AccumuloSecurityException {
-     return execute(new ClientExecReturn<Set<String>,ClientService.Client>() {
-       @Override
-       public Set<String> execute(ClientService.Client client) throws Exception {
-         return client.listLocalUsers(Tracer.traceInfo(), credentials.toThrift(instance));
-       }
-     });
+   public SecurityOperationsImpl(Instance instance, TCredentials credentials) {
+     this(instance, Credentials.fromThrift(credentials));
    }
- 
  }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/592c8b06/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperationsHelper.java
----------------------------------------------------------------------
diff --cc core/src/main/java/org/apache/accumulo/core/client/admin/TableOperationsHelper.java
index f9bc6e7,1384a54..9d0d770
--- a/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperationsHelper.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperationsHelper.java
@@@ -16,189 -16,9 +16,193 @@@
   */
  package org.apache.accumulo.core.client.admin;
  
 +import static com.google.common.base.Preconditions.checkArgument;
 +import java.util.EnumSet;
 +import java.util.HashMap;
 +import java.util.Map;
 +import java.util.Map.Entry;
 +import java.util.TreeMap;
 +import java.util.TreeSet;
 +
 +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.TableNotFoundException;
 +import org.apache.accumulo.core.conf.Property;
 +import org.apache.accumulo.core.iterators.IteratorUtil.IteratorScope;
 +
- public abstract class TableOperationsHelper implements TableOperations {
+ /**
+  * @deprecated since 1.6.0; not intended for public api and you should not use it.
+  */
+ @Deprecated
+ public abstract class TableOperationsHelper extends org.apache.accumulo.core.client.impl.TableOperationsHelper {
 +
 +  @Override
 +  public void attachIterator(String tableName, IteratorSetting setting) throws AccumuloSecurityException, AccumuloException, TableNotFoundException {
 +    attachIterator(tableName, setting, EnumSet.allOf(IteratorScope.class));
 +  }
 +
 +  @Override
 +  public void attachIterator(String tableName, IteratorSetting setting, EnumSet<IteratorScope> scopes) throws AccumuloSecurityException, AccumuloException,
 +      TableNotFoundException {
 +    checkArgument(tableName != null, "tableName is null");
 +    checkArgument(setting != null, "setting is null");
 +    checkArgument(scopes != null, "scopes is null");
 +    checkIteratorConflicts(tableName, setting, scopes);
 +    for (IteratorScope scope : scopes) {
 +      String root = String.format("%s%s.%s", Property.TABLE_ITERATOR_PREFIX, scope.name().toLowerCase(), setting.getName());
 +      for (Entry<String,String> prop : setting.getOptions().entrySet()) {
 +        this.setProperty(tableName, root + ".opt." + prop.getKey(), prop.getValue());
 +      }
 +      this.setProperty(tableName, root, setting.getPriority() + "," + setting.getIteratorClass());
 +    }
 +  }
 +
 +  @Override
 +  public void removeIterator(String tableName, String name, EnumSet<IteratorScope> scopes) throws AccumuloSecurityException, AccumuloException,
 +      TableNotFoundException {
 +    Map<String,String> copy = new TreeMap<String,String>();
 +    for (Entry<String,String> property : this.getProperties(tableName)) {
 +      copy.put(property.getKey(), property.getValue());
 +    }
 +    for (IteratorScope scope : scopes) {
 +      String root = String.format("%s%s.%s", Property.TABLE_ITERATOR_PREFIX, scope.name().toLowerCase(), name);
 +      for (Entry<String,String> property : copy.entrySet()) {
 +        if (property.getKey().equals(root) || property.getKey().startsWith(root + ".opt."))
 +          this.removeProperty(tableName, property.getKey());
 +      }
 +    }
 +  }
 +
 +  @Override
 +  public IteratorSetting getIteratorSetting(String tableName, String name, IteratorScope scope) throws AccumuloSecurityException, AccumuloException,
 +      TableNotFoundException {
 +    checkArgument(tableName != null, "tableName is null");
 +    checkArgument(name != null, "name is null");
 +    checkArgument(scope != null, "scope is null");
 +    int priority = -1;
 +    String classname = null;
 +    Map<String,String> settings = new HashMap<String,String>();
 +
 +    String root = String.format("%s%s.%s", Property.TABLE_ITERATOR_PREFIX, scope.name().toLowerCase(), name);
 +    String opt = root + ".opt.";
 +    for (Entry<String,String> property : this.getProperties(tableName)) {
 +      if (property.getKey().equals(root)) {
 +        String parts[] = property.getValue().split(",");
 +        if (parts.length != 2) {
 +          throw new AccumuloException("Bad value for iterator setting: " + property.getValue());
 +        }
 +        priority = Integer.parseInt(parts[0]);
 +        classname = parts[1];
 +      } else if (property.getKey().startsWith(opt)) {
 +        settings.put(property.getKey().substring(opt.length()), property.getValue());
 +      }
 +    }
 +    if (priority <= 0 || classname == null) {
 +      return null;
 +    }
 +    return new IteratorSetting(priority, name, classname, settings);
 +  }
 +
 +  @Override
 +  public Map<String,EnumSet<IteratorScope>> listIterators(String tableName) throws AccumuloSecurityException, AccumuloException, TableNotFoundException {
 +    Map<String,EnumSet<IteratorScope>> result = new TreeMap<String,EnumSet<IteratorScope>>();
 +    for (Entry<String,String> property : this.getProperties(tableName)) {
 +      String name = property.getKey();
 +      String[] parts = name.split("\\.");
 +      if (parts.length == 4) {
 +        if (parts[0].equals("table") && parts[1].equals("iterator")) {
 +          IteratorScope scope = IteratorScope.valueOf(parts[2]);
 +          if (!result.containsKey(parts[3]))
 +            result.put(parts[3], EnumSet.noneOf(IteratorScope.class));
 +          result.get(parts[3]).add(scope);
 +        }
 +      }
 +    }
 +    return result;
 +  }
 +
 +  @Override
 +  public void checkIteratorConflicts(String tableName, IteratorSetting setting, EnumSet<IteratorScope> scopes) throws AccumuloException, TableNotFoundException {
 +    checkArgument(tableName != null, "tableName is null");
 +    checkArgument(setting != null, "setting is null");
 +    checkArgument(scopes != null, "scopes is null");
 +    for (IteratorScope scope : scopes) {
 +      String scopeStr = String.format("%s%s", Property.TABLE_ITERATOR_PREFIX, scope.name().toLowerCase());
 +      String nameStr = String.format("%s.%s", scopeStr, setting.getName());
 +      String optStr = String.format("%s.opt.", nameStr);
 +      Map<String,String> optionConflicts = new TreeMap<String,String>();
 +      for (Entry<String,String> property : this.getProperties(tableName)) {
 +        if (property.getKey().startsWith(scopeStr)) {
 +          if (property.getKey().equals(nameStr))
 +            throw new AccumuloException(new IllegalArgumentException("iterator name conflict for " + setting.getName() + ": " + property.getKey() + "="
 +                + property.getValue()));
 +          if (property.getKey().startsWith(optStr))
 +            optionConflicts.put(property.getKey(), property.getValue());
 +          if (property.getKey().contains(".opt."))
 +            continue;
 +          String parts[] = property.getValue().split(",");
 +          if (parts.length != 2)
 +            throw new AccumuloException("Bad value for existing iterator setting: " + property.getKey() + "=" + property.getValue());
 +          try {
 +            if (Integer.parseInt(parts[0]) == setting.getPriority())
 +              throw new AccumuloException(new IllegalArgumentException("iterator priority conflict: " + property.getKey() + "=" + property.getValue()));
 +          } catch (NumberFormatException e) {
 +            throw new AccumuloException("Bad value for existing iterator setting: " + property.getKey() + "=" + property.getValue());
 +          }
 +        }
 +      }
 +      if (optionConflicts.size() > 0)
 +        throw new AccumuloException(new IllegalArgumentException("iterator options conflict for " + setting.getName() + ": " + optionConflicts));
 +    }
 +  }
 +
 +  @Override
 +  public int addConstraint(String tableName, String constraintClassName) throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
 +    TreeSet<Integer> constraintNumbers = new TreeSet<Integer>();
 +    TreeMap<String,Integer> constraintClasses = new TreeMap<String,Integer>();
 +    int i;
 +    for (Entry<String,String> property : this.getProperties(tableName)) {
 +      if (property.getKey().startsWith(Property.TABLE_CONSTRAINT_PREFIX.toString())) {
 +        try {
 +          i = Integer.parseInt(property.getKey().substring(Property.TABLE_CONSTRAINT_PREFIX.toString().length()));
 +        } catch (NumberFormatException e) {
 +          throw new AccumuloException("Bad key for existing constraint: " + property.toString());
 +        }
 +        constraintNumbers.add(i);
 +        constraintClasses.put(property.getValue(), i);
 +      }
 +    }
 +    i = 1;
 +    while (constraintNumbers.contains(i))
 +      i++;
 +    if (constraintClasses.containsKey(constraintClassName))
 +      throw new AccumuloException("Constraint " + constraintClassName + " already exists for table " + tableName + " with number "
 +          + constraintClasses.get(constraintClassName));
 +    this.setProperty(tableName, Property.TABLE_CONSTRAINT_PREFIX.toString() + i, constraintClassName);
 +    return i;
 +  }
 +
 +  @Override
 +  public void removeConstraint(String tableName, int number) throws AccumuloException, AccumuloSecurityException {
 +    this.removeProperty(tableName, Property.TABLE_CONSTRAINT_PREFIX.toString() + number);
 +  }
 +
 +  @Override
 +  public Map<String,Integer> listConstraints(String tableName) throws AccumuloException, TableNotFoundException {
 +    Map<String,Integer> constraints = new TreeMap<String,Integer>();
 +    for (Entry<String,String> property : this.getProperties(tableName)) {
 +      if (property.getKey().startsWith(Property.TABLE_CONSTRAINT_PREFIX.toString())) {
 +        if (constraints.containsKey(property.getValue()))
 +          throw new AccumuloException("Same constraint configured twice: " + property.getKey() + "=" + Property.TABLE_CONSTRAINT_PREFIX
 +              + constraints.get(property.getValue()) + "=" + property.getKey());
 +        try {
 +          constraints.put(property.getValue(), Integer.parseInt(property.getKey().substring(Property.TABLE_CONSTRAINT_PREFIX.toString().length())));
 +        } catch (NumberFormatException e) {
 +          throw new AccumuloException("Bad key for existing constraint: " + property.toString());
 +        }
 +      }
 +    }
 +    return constraints;
 +  }
  }


[14/17] ACCUMULO-2726 cleanup for master's branch.

Posted by bu...@apache.org.
http://git-wip-us.apache.org/repos/asf/accumulo/blob/f277fea9/core/src/main/java/org/apache/accumulo/core/client/impl/TableOperationsImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/TableOperationsImpl.java b/core/src/main/java/org/apache/accumulo/core/client/impl/TableOperationsImpl.java
index cd49986..166abc8 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/impl/TableOperationsImpl.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/TableOperationsImpl.java
@@ -16,10 +16,13 @@
  */
 package org.apache.accumulo.core.client.impl;
 
+import static com.google.common.base.Preconditions.checkArgument;
+
 import java.io.BufferedReader;
 import java.io.IOException;
 import java.io.InputStreamReader;
 import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
@@ -61,8 +64,8 @@ import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.client.TableOfflineException;
 import org.apache.accumulo.core.client.admin.DiskUsage;
 import org.apache.accumulo.core.client.admin.FindMax;
-import org.apache.accumulo.core.client.admin.TableOperations;
 import org.apache.accumulo.core.client.admin.TimeType;
+import org.apache.accumulo.core.client.admin.TableOperations;
 import org.apache.accumulo.core.client.impl.TabletLocator.TabletLocation;
 import org.apache.accumulo.core.client.impl.thrift.ClientService;
 import org.apache.accumulo.core.client.impl.thrift.ClientService.Client;
@@ -92,7 +95,6 @@ import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.security.Credentials;
 import org.apache.accumulo.core.tabletserver.thrift.NotServingTabletException;
 import org.apache.accumulo.core.tabletserver.thrift.TabletClientService;
-import org.apache.accumulo.core.util.ArgumentChecker;
 import org.apache.accumulo.core.util.ByteBufferUtil;
 import org.apache.accumulo.core.util.CachedConfiguration;
 import org.apache.accumulo.core.util.LocalityGroupUtil;
@@ -100,7 +102,6 @@ import org.apache.accumulo.core.util.MapCounter;
 import org.apache.accumulo.core.util.NamingThreadFactory;
 import org.apache.accumulo.core.util.OpTimer;
 import org.apache.accumulo.core.util.Pair;
-import org.apache.accumulo.core.util.StringUtil;
 import org.apache.accumulo.core.util.TextUtil;
 import org.apache.accumulo.core.util.ThriftUtil;
 import org.apache.accumulo.core.util.UtilWaitThread;
@@ -116,6 +117,8 @@ import org.apache.thrift.TApplicationException;
 import org.apache.thrift.TException;
 import org.apache.thrift.transport.TTransportException;
 
+import com.google.common.base.Joiner;
+
 public class TableOperationsImpl extends TableOperationsHelper {
   private Instance instance;
   private Credentials credentials;
@@ -131,7 +134,8 @@ public class TableOperationsImpl extends TableOperationsHelper {
    *          the username/password for this connection
    */
   public TableOperationsImpl(Instance instance, Credentials credentials) {
-    ArgumentChecker.notNull(instance, credentials);
+    checkArgument(instance != null, "instance is null");
+    checkArgument(credentials != null, "credentials is null");
     this.instance = instance;
     this.credentials = credentials;
   }
@@ -158,7 +162,7 @@ public class TableOperationsImpl extends TableOperationsHelper {
    */
   @Override
   public boolean exists(String tableName) {
-    ArgumentChecker.notNull(tableName);
+    checkArgument(tableName != null, "tableName is null");
     if (tableName.equals(MetadataTable.NAME) || tableName.equals(RootTable.NAME))
       return true;
 
@@ -206,9 +210,10 @@ public class TableOperationsImpl extends TableOperationsHelper {
    */
   @Override
   public void create(String tableName, boolean limitVersion, TimeType timeType) throws AccumuloException, AccumuloSecurityException, TableExistsException {
-    ArgumentChecker.notNull(tableName, timeType);
+    checkArgument(tableName != null, "tableName is null");
+    checkArgument(timeType != null, "timeType is null");
 
-    List<ByteBuffer> args = Arrays.asList(ByteBuffer.wrap(tableName.getBytes(Constants.UTF8)), ByteBuffer.wrap(timeType.name().getBytes(Constants.UTF8)));
+    List<ByteBuffer> args = Arrays.asList(ByteBuffer.wrap(tableName.getBytes(StandardCharsets.UTF_8)), ByteBuffer.wrap(timeType.name().getBytes(StandardCharsets.UTF_8)));
 
     Map<String,String> opts;
     if (limitVersion)
@@ -522,9 +527,9 @@ public class TableOperationsImpl extends TableOperationsHelper {
   @Override
   public void merge(String tableName, Text start, Text end) throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
 
-    ArgumentChecker.notNull(tableName);
+    checkArgument(tableName != null, "tableName is null");
     ByteBuffer EMPTY = ByteBuffer.allocate(0);
-    List<ByteBuffer> args = Arrays.asList(ByteBuffer.wrap(tableName.getBytes(Constants.UTF8)), start == null ? EMPTY : TextUtil.getByteBuffer(start),
+    List<ByteBuffer> args = Arrays.asList(ByteBuffer.wrap(tableName.getBytes(StandardCharsets.UTF_8)), start == null ? EMPTY : TextUtil.getByteBuffer(start),
         end == null ? EMPTY : TextUtil.getByteBuffer(end));
     Map<String,String> opts = new HashMap<String,String>();
     try {
@@ -538,9 +543,9 @@ public class TableOperationsImpl extends TableOperationsHelper {
   @Override
   public void deleteRows(String tableName, Text start, Text end) throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
 
-    ArgumentChecker.notNull(tableName);
+    checkArgument(tableName != null, "tableName is null");
     ByteBuffer EMPTY = ByteBuffer.allocate(0);
-    List<ByteBuffer> args = Arrays.asList(ByteBuffer.wrap(tableName.getBytes(Constants.UTF8)), start == null ? EMPTY : TextUtil.getByteBuffer(start),
+    List<ByteBuffer> args = Arrays.asList(ByteBuffer.wrap(tableName.getBytes(StandardCharsets.UTF_8)), start == null ? EMPTY : TextUtil.getByteBuffer(start),
         end == null ? EMPTY : TextUtil.getByteBuffer(end));
     Map<String,String> opts = new HashMap<String,String>();
     try {
@@ -559,7 +564,7 @@ public class TableOperationsImpl extends TableOperationsHelper {
   @Override
   public Collection<Text> listSplits(String tableName) throws TableNotFoundException, AccumuloSecurityException {
 
-    ArgumentChecker.notNull(tableName);
+    checkArgument(tableName != null, "tableName is null");
 
     String tableId = Tables.getTableId(instance, tableName);
 
@@ -662,9 +667,9 @@ public class TableOperationsImpl extends TableOperationsHelper {
    */
   @Override
   public void delete(String tableName) throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
-    ArgumentChecker.notNull(tableName);
+    checkArgument(tableName != null, "tableName is null");
 
-    List<ByteBuffer> args = Arrays.asList(ByteBuffer.wrap(tableName.getBytes(Constants.UTF8)));
+    List<ByteBuffer> args = Arrays.asList(ByteBuffer.wrap(tableName.getBytes(StandardCharsets.UTF_8)));
     Map<String,String> opts = new HashMap<String,String>();
 
     try {
@@ -680,7 +685,8 @@ public class TableOperationsImpl extends TableOperationsHelper {
   public void clone(String srcTableName, String newTableName, boolean flush, Map<String,String> propertiesToSet, Set<String> propertiesToExclude)
       throws AccumuloSecurityException, TableNotFoundException, AccumuloException, TableExistsException {
 
-    ArgumentChecker.notNull(srcTableName, newTableName);
+    checkArgument(srcTableName != null, "srcTableName is null");
+    checkArgument(newTableName != null, "newTableName is null");
 
     String srcTableId = Tables.getTableId(instance, srcTableName);
 
@@ -693,7 +699,7 @@ public class TableOperationsImpl extends TableOperationsHelper {
     if (propertiesToSet == null)
       propertiesToSet = Collections.emptyMap();
 
-    List<ByteBuffer> args = Arrays.asList(ByteBuffer.wrap(srcTableId.getBytes(Constants.UTF8)), ByteBuffer.wrap(newTableName.getBytes(Constants.UTF8)));
+    List<ByteBuffer> args = Arrays.asList(ByteBuffer.wrap(srcTableId.getBytes(StandardCharsets.UTF_8)), ByteBuffer.wrap(newTableName.getBytes(StandardCharsets.UTF_8)));
     Map<String,String> opts = new HashMap<String,String>();
     for (Entry<String,String> entry : propertiesToSet.entrySet()) {
       if (entry.getKey().startsWith(CLONE_EXCLUDE_PREFIX))
@@ -728,7 +734,7 @@ public class TableOperationsImpl extends TableOperationsHelper {
   public void rename(String oldTableName, String newTableName) throws AccumuloSecurityException, TableNotFoundException, AccumuloException,
       TableExistsException {
 
-    List<ByteBuffer> args = Arrays.asList(ByteBuffer.wrap(oldTableName.getBytes(Constants.UTF8)), ByteBuffer.wrap(newTableName.getBytes(Constants.UTF8)));
+    List<ByteBuffer> args = Arrays.asList(ByteBuffer.wrap(oldTableName.getBytes(StandardCharsets.UTF_8)), ByteBuffer.wrap(newTableName.getBytes(StandardCharsets.UTF_8)));
     Map<String,String> opts = new HashMap<String,String>();
     doTableFateOperation(oldTableName, TableNotFoundException.class, FateOperation.TABLE_RENAME, args, opts);
   }
@@ -758,7 +764,7 @@ public class TableOperationsImpl extends TableOperationsHelper {
    */
   @Override
   public void flush(String tableName, Text start, Text end, boolean wait) throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
-    ArgumentChecker.notNull(tableName);
+    checkArgument(tableName != null, "tableName is null");
 
     String tableId = Tables.getTableId(instance, tableName);
     _flush(tableId, start, end, wait);
@@ -773,7 +779,7 @@ public class TableOperationsImpl extends TableOperationsHelper {
   @Override
   public void compact(String tableName, Text start, Text end, List<IteratorSetting> iterators, boolean flush, boolean wait) throws AccumuloSecurityException,
       TableNotFoundException, AccumuloException {
-    ArgumentChecker.notNull(tableName);
+    checkArgument(tableName != null, "tableName is null");
     ByteBuffer EMPTY = ByteBuffer.allocate(0);
 
     String tableId = Tables.getTableId(instance, tableName);
@@ -781,7 +787,7 @@ public class TableOperationsImpl extends TableOperationsHelper {
     if (flush)
       _flush(tableId, start, end, true);
 
-    List<ByteBuffer> args = Arrays.asList(ByteBuffer.wrap(tableId.getBytes(Constants.UTF8)), start == null ? EMPTY : TextUtil.getByteBuffer(start),
+    List<ByteBuffer> args = Arrays.asList(ByteBuffer.wrap(tableId.getBytes(StandardCharsets.UTF_8)), start == null ? EMPTY : TextUtil.getByteBuffer(start),
         end == null ? EMPTY : TextUtil.getByteBuffer(end), ByteBuffer.wrap(IteratorUtil.encodeIteratorSettings(iterators)));
 
     Map<String,String> opts = new HashMap<String,String>();
@@ -802,7 +808,7 @@ public class TableOperationsImpl extends TableOperationsHelper {
   public void cancelCompaction(String tableName) throws AccumuloSecurityException, TableNotFoundException, AccumuloException {
     String tableId = Tables.getTableId(instance, tableName);
 
-    List<ByteBuffer> args = Arrays.asList(ByteBuffer.wrap(tableId.getBytes(Constants.UTF8)));
+    List<ByteBuffer> args = Arrays.asList(ByteBuffer.wrap(tableId.getBytes(StandardCharsets.UTF_8)));
 
     Map<String,String> opts = new HashMap<String,String>();
     try {
@@ -886,7 +892,9 @@ public class TableOperationsImpl extends TableOperationsHelper {
    */
   @Override
   public void setProperty(final String tableName, final String property, final String value) throws AccumuloException, AccumuloSecurityException {
-    ArgumentChecker.notNull(tableName, property, value);
+    checkArgument(tableName != null, "tableName is null");
+    checkArgument(property != null, "property is null");
+    checkArgument(value != null, "value is null");
     try {
       MasterClient.executeTable(instance, new ClientExec<MasterClientService.Client>() {
         @Override
@@ -913,7 +921,8 @@ public class TableOperationsImpl extends TableOperationsHelper {
    */
   @Override
   public void removeProperty(final String tableName, final String property) throws AccumuloException, AccumuloSecurityException {
-    ArgumentChecker.notNull(tableName, property);
+    checkArgument(tableName != null, "tableName is null");
+    checkArgument(property != null, "property is null");
     try {
       MasterClient.executeTable(instance, new ClientExec<MasterClientService.Client>() {
         @Override
@@ -937,7 +946,7 @@ public class TableOperationsImpl extends TableOperationsHelper {
    */
   @Override
   public Iterable<Entry<String,String>> getProperties(final String tableName) throws AccumuloException, TableNotFoundException {
-    ArgumentChecker.notNull(tableName);
+    checkArgument(tableName != null, "tableName is null");
     try {
       return ServerClient.executeRaw(instance, new ClientExecReturn<Map<String,String>,ClientService.Client>() {
         @Override
@@ -996,7 +1005,7 @@ public class TableOperationsImpl extends TableOperationsHelper {
     }
 
     try {
-      setProperty(tableName, Property.TABLE_LOCALITY_GROUPS.getKey(), StringUtil.join(groups.keySet(), ","));
+      setProperty(tableName, Property.TABLE_LOCALITY_GROUPS.getKey(), Joiner.on(",").join(groups.keySet()));
     } catch (AccumuloException e) {
       if (e.getCause() instanceof TableNotFoundException)
         throw (TableNotFoundException) e.getCause();
@@ -1070,7 +1079,8 @@ public class TableOperationsImpl extends TableOperationsHelper {
   @Override
   public Set<Range> splitRangeByTablets(String tableName, Range range, int maxSplits) throws AccumuloException, AccumuloSecurityException,
       TableNotFoundException {
-    ArgumentChecker.notNull(tableName, range);
+    checkArgument(tableName != null, "tableName is null");
+    checkArgument(range != null, "range is null");
     if (maxSplits < 1)
       throw new IllegalArgumentException("maximum splits must be >= 1");
     if (maxSplits == 1)
@@ -1161,15 +1171,17 @@ public class TableOperationsImpl extends TableOperationsHelper {
   @Override
   public void importDirectory(String tableName, String dir, String failureDir, boolean setTime) throws IOException, AccumuloSecurityException,
       TableNotFoundException, AccumuloException {
-    ArgumentChecker.notNull(tableName, dir, failureDir);
+    checkArgument(tableName != null, "tableName is null");
+    checkArgument(dir != null, "dir is null");
+    checkArgument(failureDir != null, "failureDir is null");
     // check for table existance
     Tables.getTableId(instance, tableName);
 
     Path dirPath = checkPath(dir, "Bulk", "");
     Path failPath = checkPath(failureDir, "Bulk", "failure");
 
-    List<ByteBuffer> args = Arrays.asList(ByteBuffer.wrap(tableName.getBytes(Constants.UTF8)), ByteBuffer.wrap(dirPath.toString().getBytes(Constants.UTF8)),
-        ByteBuffer.wrap(failPath.toString().getBytes(Constants.UTF8)), ByteBuffer.wrap((setTime + "").getBytes(Constants.UTF8)));
+    List<ByteBuffer> args = Arrays.asList(ByteBuffer.wrap(tableName.getBytes(StandardCharsets.UTF_8)), ByteBuffer.wrap(dirPath.toString().getBytes(StandardCharsets.UTF_8)),
+        ByteBuffer.wrap(failPath.toString().getBytes(StandardCharsets.UTF_8)), ByteBuffer.wrap((setTime + "").getBytes(StandardCharsets.UTF_8)));
     Map<String,String> opts = new HashMap<String,String>();
 
     try {
@@ -1314,9 +1326,9 @@ public class TableOperationsImpl extends TableOperationsHelper {
   @Override
   public void offline(String tableName, boolean wait) throws AccumuloSecurityException, AccumuloException, TableNotFoundException {
 
-    ArgumentChecker.notNull(tableName);
+    checkArgument(tableName != null, "tableName is null");
     String tableId = Tables.getTableId(instance, tableName);
-    List<ByteBuffer> args = Arrays.asList(ByteBuffer.wrap(tableId.getBytes(Constants.UTF8)));
+    List<ByteBuffer> args = Arrays.asList(ByteBuffer.wrap(tableId.getBytes(StandardCharsets.UTF_8)));
     Map<String,String> opts = new HashMap<String,String>();
 
     try {
@@ -1346,9 +1358,9 @@ public class TableOperationsImpl extends TableOperationsHelper {
    */
   @Override
   public void online(String tableName, boolean wait) throws AccumuloSecurityException, AccumuloException, TableNotFoundException {
-    ArgumentChecker.notNull(tableName);
+    checkArgument(tableName != null, "tableName is null");
     String tableId = Tables.getTableId(instance, tableName);
-    List<ByteBuffer> args = Arrays.asList(ByteBuffer.wrap(tableId.getBytes(Constants.UTF8)));
+    List<ByteBuffer> args = Arrays.asList(ByteBuffer.wrap(tableId.getBytes(StandardCharsets.UTF_8)));
     Map<String,String> opts = new HashMap<String,String>();
 
     try {
@@ -1372,7 +1384,7 @@ public class TableOperationsImpl extends TableOperationsHelper {
    */
   @Override
   public void clearLocatorCache(String tableName) throws TableNotFoundException {
-    ArgumentChecker.notNull(tableName);
+    checkArgument(tableName != null, "tableName is null");
     TabletLocator tabLocator = TabletLocator.getLocator(instance, new Text(Tables.getTableId(instance, tableName)));
     tabLocator.invalidateCache();
   }
@@ -1390,7 +1402,8 @@ public class TableOperationsImpl extends TableOperationsHelper {
   @Override
   public Text getMaxRow(String tableName, Authorizations auths, Text startRow, boolean startInclusive, Text endRow, boolean endInclusive)
       throws TableNotFoundException, AccumuloException, AccumuloSecurityException {
-    ArgumentChecker.notNull(tableName, auths);
+    checkArgument(tableName != null, "tableName is null");
+    checkArgument(auths != null, "auths is null");
     Scanner scanner = instance.getConnector(credentials.getPrincipal(), credentials.getToken()).createScanner(tableName, auths);
     return FindMax.findMax(scanner, startRow, startInclusive, endRow, endInclusive);
   }
@@ -1447,7 +1460,7 @@ public class TableOperationsImpl extends TableOperationsHelper {
       ZipEntry zipEntry;
       while ((zipEntry = zis.getNextEntry()) != null) {
         if (zipEntry.getName().equals(Constants.EXPORT_TABLE_CONFIG_FILE)) {
-          BufferedReader in = new BufferedReader(new InputStreamReader(zis, Constants.UTF8));
+          BufferedReader in = new BufferedReader(new InputStreamReader(zis, StandardCharsets.UTF_8));
           try {
             String line;
             while ((line = in.readLine()) != null) {
@@ -1469,7 +1482,8 @@ public class TableOperationsImpl extends TableOperationsHelper {
 
   @Override
   public void importTable(String tableName, String importDir) throws TableExistsException, AccumuloException, AccumuloSecurityException {
-    ArgumentChecker.notNull(tableName, importDir);
+    checkArgument(tableName != null, "tableName is null");
+    checkArgument(importDir != null, "importDir is null");
 
     try {
       importDir = checkPath(importDir, "Table", "").toString();
@@ -1492,7 +1506,7 @@ public class TableOperationsImpl extends TableOperationsHelper {
       Logger.getLogger(this.getClass()).warn("Failed to check if imported table references external java classes : " + ioe.getMessage());
     }
 
-    List<ByteBuffer> args = Arrays.asList(ByteBuffer.wrap(tableName.getBytes(Constants.UTF8)), ByteBuffer.wrap(importDir.getBytes(Constants.UTF8)));
+    List<ByteBuffer> args = Arrays.asList(ByteBuffer.wrap(tableName.getBytes(StandardCharsets.UTF_8)), ByteBuffer.wrap(importDir.getBytes(StandardCharsets.UTF_8)));
 
     Map<String,String> opts = Collections.emptyMap();
 
@@ -1507,9 +1521,10 @@ public class TableOperationsImpl extends TableOperationsHelper {
 
   @Override
   public void exportTable(String tableName, String exportDir) throws TableNotFoundException, AccumuloException, AccumuloSecurityException {
-    ArgumentChecker.notNull(tableName, exportDir);
+    checkArgument(tableName != null, "tableName is null");
+    checkArgument(exportDir != null, "exportDir is null");
 
-    List<ByteBuffer> args = Arrays.asList(ByteBuffer.wrap(tableName.getBytes(Constants.UTF8)), ByteBuffer.wrap(exportDir.getBytes(Constants.UTF8)));
+    List<ByteBuffer> args = Arrays.asList(ByteBuffer.wrap(tableName.getBytes(StandardCharsets.UTF_8)), ByteBuffer.wrap(exportDir.getBytes(StandardCharsets.UTF_8)));
 
     Map<String,String> opts = Collections.emptyMap();
 
@@ -1524,7 +1539,9 @@ public class TableOperationsImpl extends TableOperationsHelper {
   @Override
   public boolean testClassLoad(final String tableName, final String className, final String asTypeName) throws TableNotFoundException, AccumuloException,
       AccumuloSecurityException {
-    ArgumentChecker.notNull(tableName, className, asTypeName);
+    checkArgument(tableName != null, "tableName is null");
+    checkArgument(className != null, "className is null");
+    checkArgument(asTypeName != null, "asTypeName is null");
 
     try {
       return ServerClient.executeRaw(instance, new ClientExecReturn<Boolean,ClientService.Client>() {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/f277fea9/core/src/main/java/org/apache/accumulo/core/client/mock/MockConnector.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mock/MockConnector.java b/core/src/main/java/org/apache/accumulo/core/client/mock/MockConnector.java
index 6cd6f2c..996198c 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mock/MockConnector.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mock/MockConnector.java
@@ -124,17 +124,17 @@ public class MockConnector extends Connector {
   
   @Override
   public TableOperations tableOperations() {
-    return new MockTableOperationsImpl(acu, username);
+    return new MockTableOperations(acu, username);
   }
   
   @Override
   public SecurityOperations securityOperations() {
-    return new MockSecurityOperationsImpl(acu);
+    return new MockSecurityOperations(acu);
   }
   
   @Override
   public InstanceOperations instanceOperations() {
-    return new MockInstanceOperationsImpl(acu);
+    return new MockInstanceOperations(acu);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/accumulo/blob/f277fea9/core/src/main/java/org/apache/accumulo/core/client/mock/MockInstanceOperations.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mock/MockInstanceOperations.java b/core/src/main/java/org/apache/accumulo/core/client/mock/MockInstanceOperations.java
index 58eff24..89bf39c 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mock/MockInstanceOperations.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mock/MockInstanceOperations.java
@@ -16,13 +16,72 @@
  */
 package org.apache.accumulo.core.client.mock;
 
-/**
- * @deprecated since 1.6.0; not intended for public api and you should not use it.
- */
-@Deprecated
-public class MockInstanceOperations extends MockInstanceOperationsImpl {
-  
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.accumulo.core.client.AccumuloException;
+import org.apache.accumulo.core.client.AccumuloSecurityException;
+import org.apache.accumulo.core.client.admin.ActiveCompaction;
+import org.apache.accumulo.core.client.admin.ActiveScan;
+import org.apache.accumulo.core.client.admin.InstanceOperations;
+import org.apache.accumulo.start.classloader.vfs.AccumuloVFSClassLoader;
+
+class MockInstanceOperations implements InstanceOperations {
+  MockAccumulo acu;
+
   public MockInstanceOperations(MockAccumulo acu) {
-    super(acu);
+    this.acu = acu;
+  }
+
+  @Override
+  public void setProperty(String property, String value) throws AccumuloException, AccumuloSecurityException {
+    acu.setProperty(property, value);
+  }
+
+  @Override
+  public void removeProperty(String property) throws AccumuloException, AccumuloSecurityException {
+    acu.removeProperty(property);
+  }
+
+  @Override
+  public Map<String,String> getSystemConfiguration() throws AccumuloException, AccumuloSecurityException {
+    return acu.systemProperties;
+  }
+
+  @Override
+  public Map<String,String> getSiteConfiguration() throws AccumuloException, AccumuloSecurityException {
+    return acu.systemProperties;
+  }
+
+  @Override
+  public List<String> getTabletServers() {
+    return new ArrayList<String>();
+  }
+
+  @Override
+  public List<ActiveScan> getActiveScans(String tserver) throws AccumuloException, AccumuloSecurityException {
+    return new ArrayList<ActiveScan>();
+  }
+
+  @Override
+  public boolean testClassLoad(String className, String asTypeName) throws AccumuloException, AccumuloSecurityException {
+    try {
+      AccumuloVFSClassLoader.loadClass(className, Class.forName(asTypeName));
+    } catch (ClassNotFoundException e) {
+      e.printStackTrace();
+      return false;
+    }
+    return true;
+  }
+
+  @Override
+  public List<ActiveCompaction> getActiveCompactions(String tserver) throws AccumuloException, AccumuloSecurityException {
+    return new ArrayList<ActiveCompaction>();
+  }
+
+  @Override
+  public void ping(String tserver) throws AccumuloException {
+
   }
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/f277fea9/core/src/main/java/org/apache/accumulo/core/client/mock/MockInstanceOperationsImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mock/MockInstanceOperationsImpl.java b/core/src/main/java/org/apache/accumulo/core/client/mock/MockInstanceOperationsImpl.java
deleted file mode 100644
index 1bec226..0000000
--- a/core/src/main/java/org/apache/accumulo/core/client/mock/MockInstanceOperationsImpl.java
+++ /dev/null
@@ -1,87 +0,0 @@
-/*
- * 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.core.client.mock;
-
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.accumulo.core.client.AccumuloException;
-import org.apache.accumulo.core.client.AccumuloSecurityException;
-import org.apache.accumulo.core.client.admin.ActiveCompaction;
-import org.apache.accumulo.core.client.admin.ActiveScan;
-import org.apache.accumulo.core.client.admin.InstanceOperations;
-import org.apache.accumulo.start.classloader.vfs.AccumuloVFSClassLoader;
-
-class MockInstanceOperationsImpl implements InstanceOperations {
-  MockAccumulo acu;
-
-  public MockInstanceOperationsImpl(MockAccumulo acu) {
-    this.acu = acu;
-  }
-
-  @Override
-  public void setProperty(String property, String value) throws AccumuloException, AccumuloSecurityException {
-    acu.setProperty(property, value);
-  }
-
-  @Override
-  public void removeProperty(String property) throws AccumuloException, AccumuloSecurityException {
-    acu.removeProperty(property);
-  }
-
-  @Override
-  public Map<String,String> getSystemConfiguration() throws AccumuloException, AccumuloSecurityException {
-    return acu.systemProperties;
-  }
-
-  @Override
-  public Map<String,String> getSiteConfiguration() throws AccumuloException, AccumuloSecurityException {
-    return acu.systemProperties;
-  }
-
-  @Override
-  public List<String> getTabletServers() {
-    return new ArrayList<String>();
-  }
-
-  @Override
-  public List<ActiveScan> getActiveScans(String tserver) throws AccumuloException, AccumuloSecurityException {
-    return new ArrayList<ActiveScan>();
-  }
-
-  @Override
-  public boolean testClassLoad(String className, String asTypeName) throws AccumuloException, AccumuloSecurityException {
-    try {
-      AccumuloVFSClassLoader.loadClass(className, Class.forName(asTypeName));
-    } catch (ClassNotFoundException e) {
-      e.printStackTrace();
-      return false;
-    }
-    return true;
-  }
-
-  @Override
-  public List<ActiveCompaction> getActiveCompactions(String tserver) throws AccumuloException, AccumuloSecurityException {
-    return new ArrayList<ActiveCompaction>();
-  }
-
-  @Override
-  public void ping(String tserver) throws AccumuloException {
-
-  }
-}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/f277fea9/core/src/main/java/org/apache/accumulo/core/client/mock/MockSecurityOperations.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mock/MockSecurityOperations.java b/core/src/main/java/org/apache/accumulo/core/client/mock/MockSecurityOperations.java
index 6271059..db88cfb 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mock/MockSecurityOperations.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mock/MockSecurityOperations.java
@@ -16,13 +16,210 @@
  */
 package org.apache.accumulo.core.client.mock;
 
-/**
- * @deprecated since 1.6.0; not intended for public api and you should not use it.
- */
-@Deprecated
-public class MockSecurityOperations extends MockSecurityOperationsImpl {
+import java.util.EnumSet;
+import java.util.Set;
+
+import org.apache.accumulo.core.client.AccumuloException;
+import org.apache.accumulo.core.client.AccumuloSecurityException;
+import org.apache.accumulo.core.client.admin.SecurityOperations;
+import org.apache.accumulo.core.client.impl.thrift.SecurityErrorCode;
+import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
+import org.apache.accumulo.core.client.security.tokens.PasswordToken;
+import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.core.security.NamespacePermission;
+import org.apache.accumulo.core.security.SystemPermission;
+import org.apache.accumulo.core.security.TablePermission;
+
+class MockSecurityOperations implements SecurityOperations {
+
+  final private MockAccumulo acu;
 
   MockSecurityOperations(MockAccumulo acu) {
-    super(acu);
+    this.acu = acu;
   }
+
+  @Deprecated
+  @Override
+  public void createUser(String user, byte[] password, Authorizations authorizations) throws AccumuloException, AccumuloSecurityException {
+    createLocalUser(user, new PasswordToken(password));
+    changeUserAuthorizations(user, authorizations);
+  }
+
+  @Override
+  public void createLocalUser(String principal, PasswordToken password) throws AccumuloException, AccumuloSecurityException {
+    this.acu.users.put(principal, new MockUser(principal, password, new Authorizations()));
+  }
+
+  @Deprecated
+  @Override
+  public void dropUser(String user) throws AccumuloException, AccumuloSecurityException {
+    dropLocalUser(user);
+  }
+
+  @Override
+  public void dropLocalUser(String principal) throws AccumuloException, AccumuloSecurityException {
+    this.acu.users.remove(principal);
+  }
+
+  @Deprecated
+  @Override
+  public boolean authenticateUser(String user, byte[] password) throws AccumuloException, AccumuloSecurityException {
+    return authenticateUser(user, new PasswordToken(password));
+  }
+
+  @Override
+  public boolean authenticateUser(String principal, AuthenticationToken token) throws AccumuloException, AccumuloSecurityException {
+    MockUser user = acu.users.get(principal);
+    if (user == null)
+      return false;
+    return user.token.equals(token);
+  }
+
+  @Deprecated
+  @Override
+  public void changeUserPassword(String user, byte[] password) throws AccumuloException, AccumuloSecurityException {
+    changeLocalUserPassword(user, new PasswordToken(password));
+  }
+
+  @Override
+  public void changeLocalUserPassword(String principal, PasswordToken token) throws AccumuloException, AccumuloSecurityException {
+    MockUser user = acu.users.get(principal);
+    if (user != null)
+      user.token = token.clone();
+    else
+      throw new AccumuloSecurityException(principal, SecurityErrorCode.USER_DOESNT_EXIST);
+  }
+
+  @Override
+  public void changeUserAuthorizations(String principal, Authorizations authorizations) throws AccumuloException, AccumuloSecurityException {
+    MockUser user = acu.users.get(principal);
+    if (user != null)
+      user.authorizations = authorizations;
+    else
+      throw new AccumuloSecurityException(principal, SecurityErrorCode.USER_DOESNT_EXIST);
+  }
+
+  @Override
+  public Authorizations getUserAuthorizations(String principal) throws AccumuloException, AccumuloSecurityException {
+    MockUser user = acu.users.get(principal);
+    if (user != null)
+      return user.authorizations;
+    else
+      throw new AccumuloSecurityException(principal, SecurityErrorCode.USER_DOESNT_EXIST);
+  }
+
+  @Override
+  public boolean hasSystemPermission(String principal, SystemPermission perm) throws AccumuloException, AccumuloSecurityException {
+    MockUser user = acu.users.get(principal);
+    if (user != null)
+      return user.permissions.contains(perm);
+    else
+      throw new AccumuloSecurityException(principal, SecurityErrorCode.USER_DOESNT_EXIST);
+  }
+
+  @Override
+  public boolean hasTablePermission(String principal, String tableName, TablePermission perm) throws AccumuloException, AccumuloSecurityException {
+    MockTable table = acu.tables.get(tableName);
+    if (table == null)
+      throw new AccumuloSecurityException(tableName, SecurityErrorCode.TABLE_DOESNT_EXIST);
+    EnumSet<TablePermission> perms = table.userPermissions.get(principal);
+    if (perms == null)
+      return false;
+    return perms.contains(perm);
+  }
+
+  @Override
+  public boolean hasNamespacePermission(String principal, String namespace, NamespacePermission permission) throws AccumuloException, AccumuloSecurityException {
+    MockNamespace mockNamespace = acu.namespaces.get(namespace);
+    if (mockNamespace == null)
+      throw new AccumuloSecurityException(namespace, SecurityErrorCode.NAMESPACE_DOESNT_EXIST);
+    EnumSet<NamespacePermission> perms = mockNamespace.userPermissions.get(principal);
+    if (perms == null)
+      return false;
+    return perms.contains(permission);
+  }
+
+  @Override
+  public void grantSystemPermission(String principal, SystemPermission permission) throws AccumuloException, AccumuloSecurityException {
+    MockUser user = acu.users.get(principal);
+    if (user != null)
+      user.permissions.add(permission);
+    else
+      throw new AccumuloSecurityException(principal, SecurityErrorCode.USER_DOESNT_EXIST);
+  }
+
+  @Override
+  public void grantTablePermission(String principal, String tableName, TablePermission permission) throws AccumuloException, AccumuloSecurityException {
+    if (acu.users.get(principal) == null)
+      throw new AccumuloSecurityException(principal, SecurityErrorCode.USER_DOESNT_EXIST);
+    MockTable table = acu.tables.get(tableName);
+    if (table == null)
+      throw new AccumuloSecurityException(tableName, SecurityErrorCode.TABLE_DOESNT_EXIST);
+    EnumSet<TablePermission> perms = table.userPermissions.get(principal);
+    if (perms == null)
+      table.userPermissions.put(principal, EnumSet.of(permission));
+    else
+      perms.add(permission);
+  }
+
+  @Override
+  public void grantNamespacePermission(String principal, String namespace, NamespacePermission permission) throws AccumuloException, AccumuloSecurityException {
+    if (acu.users.get(principal) == null)
+      throw new AccumuloSecurityException(principal, SecurityErrorCode.USER_DOESNT_EXIST);
+    MockNamespace mockNamespace = acu.namespaces.get(namespace);
+    if (mockNamespace == null)
+      throw new AccumuloSecurityException(namespace, SecurityErrorCode.NAMESPACE_DOESNT_EXIST);
+    EnumSet<NamespacePermission> perms = mockNamespace.userPermissions.get(principal);
+    if (perms == null)
+      mockNamespace.userPermissions.put(principal, EnumSet.of(permission));
+    else
+      perms.add(permission);
+  }
+
+  @Override
+  public void revokeSystemPermission(String principal, SystemPermission permission) throws AccumuloException, AccumuloSecurityException {
+    MockUser user = acu.users.get(principal);
+    if (user != null)
+      user.permissions.remove(permission);
+    else
+      throw new AccumuloSecurityException(principal, SecurityErrorCode.USER_DOESNT_EXIST);
+  }
+
+  @Override
+  public void revokeTablePermission(String principal, String tableName, TablePermission permission) throws AccumuloException, AccumuloSecurityException {
+    if (acu.users.get(principal) == null)
+      throw new AccumuloSecurityException(principal, SecurityErrorCode.USER_DOESNT_EXIST);
+    MockTable table = acu.tables.get(tableName);
+    if (table == null)
+      throw new AccumuloSecurityException(tableName, SecurityErrorCode.TABLE_DOESNT_EXIST);
+    EnumSet<TablePermission> perms = table.userPermissions.get(principal);
+    if (perms != null)
+      perms.remove(permission);
+
+  }
+
+  @Override
+  public void revokeNamespacePermission(String principal, String namespace, NamespacePermission permission) throws AccumuloException, AccumuloSecurityException {
+    if (acu.users.get(principal) == null)
+      throw new AccumuloSecurityException(principal, SecurityErrorCode.USER_DOESNT_EXIST);
+    MockNamespace mockNamespace = acu.namespaces.get(namespace);
+    if (mockNamespace == null)
+      throw new AccumuloSecurityException(namespace, SecurityErrorCode.NAMESPACE_DOESNT_EXIST);
+    EnumSet<NamespacePermission> perms = mockNamespace.userPermissions.get(principal);
+    if (perms != null)
+      perms.remove(permission);
+
+  }
+
+  @Deprecated
+  @Override
+  public Set<String> listUsers() throws AccumuloException, AccumuloSecurityException {
+    return listLocalUsers();
+  }
+
+  @Override
+  public Set<String> listLocalUsers() throws AccumuloException, AccumuloSecurityException {
+    return acu.users.keySet();
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/f277fea9/core/src/main/java/org/apache/accumulo/core/client/mock/MockSecurityOperationsImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mock/MockSecurityOperationsImpl.java b/core/src/main/java/org/apache/accumulo/core/client/mock/MockSecurityOperationsImpl.java
deleted file mode 100644
index f97069a..0000000
--- a/core/src/main/java/org/apache/accumulo/core/client/mock/MockSecurityOperationsImpl.java
+++ /dev/null
@@ -1,225 +0,0 @@
-/*
- * 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.core.client.mock;
-
-import java.util.EnumSet;
-import java.util.Set;
-
-import org.apache.accumulo.core.client.AccumuloException;
-import org.apache.accumulo.core.client.AccumuloSecurityException;
-import org.apache.accumulo.core.client.admin.SecurityOperations;
-import org.apache.accumulo.core.client.impl.thrift.SecurityErrorCode;
-import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
-import org.apache.accumulo.core.client.security.tokens.PasswordToken;
-import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.core.security.NamespacePermission;
-import org.apache.accumulo.core.security.SystemPermission;
-import org.apache.accumulo.core.security.TablePermission;
-
-class MockSecurityOperationsImpl implements SecurityOperations {
-
-  final private MockAccumulo acu;
-
-  MockSecurityOperationsImpl(MockAccumulo acu) {
-    this.acu = acu;
-  }
-
-  @Deprecated
-  @Override
-  public void createUser(String user, byte[] password, Authorizations authorizations) throws AccumuloException, AccumuloSecurityException {
-    createLocalUser(user, new PasswordToken(password));
-    changeUserAuthorizations(user, authorizations);
-  }
-
-  @Override
-  public void createLocalUser(String principal, PasswordToken password) throws AccumuloException, AccumuloSecurityException {
-    this.acu.users.put(principal, new MockUser(principal, password, new Authorizations()));
-  }
-
-  @Deprecated
-  @Override
-  public void dropUser(String user) throws AccumuloException, AccumuloSecurityException {
-    dropLocalUser(user);
-  }
-
-  @Override
-  public void dropLocalUser(String principal) throws AccumuloException, AccumuloSecurityException {
-    this.acu.users.remove(principal);
-  }
-
-  @Deprecated
-  @Override
-  public boolean authenticateUser(String user, byte[] password) throws AccumuloException, AccumuloSecurityException {
-    return authenticateUser(user, new PasswordToken(password));
-  }
-
-  @Override
-  public boolean authenticateUser(String principal, AuthenticationToken token) throws AccumuloException, AccumuloSecurityException {
-    MockUser user = acu.users.get(principal);
-    if (user == null)
-      return false;
-    return user.token.equals(token);
-  }
-
-  @Deprecated
-  @Override
-  public void changeUserPassword(String user, byte[] password) throws AccumuloException, AccumuloSecurityException {
-    changeLocalUserPassword(user, new PasswordToken(password));
-  }
-
-  @Override
-  public void changeLocalUserPassword(String principal, PasswordToken token) throws AccumuloException, AccumuloSecurityException {
-    MockUser user = acu.users.get(principal);
-    if (user != null)
-      user.token = token.clone();
-    else
-      throw new AccumuloSecurityException(principal, SecurityErrorCode.USER_DOESNT_EXIST);
-  }
-
-  @Override
-  public void changeUserAuthorizations(String principal, Authorizations authorizations) throws AccumuloException, AccumuloSecurityException {
-    MockUser user = acu.users.get(principal);
-    if (user != null)
-      user.authorizations = authorizations;
-    else
-      throw new AccumuloSecurityException(principal, SecurityErrorCode.USER_DOESNT_EXIST);
-  }
-
-  @Override
-  public Authorizations getUserAuthorizations(String principal) throws AccumuloException, AccumuloSecurityException {
-    MockUser user = acu.users.get(principal);
-    if (user != null)
-      return user.authorizations;
-    else
-      throw new AccumuloSecurityException(principal, SecurityErrorCode.USER_DOESNT_EXIST);
-  }
-
-  @Override
-  public boolean hasSystemPermission(String principal, SystemPermission perm) throws AccumuloException, AccumuloSecurityException {
-    MockUser user = acu.users.get(principal);
-    if (user != null)
-      return user.permissions.contains(perm);
-    else
-      throw new AccumuloSecurityException(principal, SecurityErrorCode.USER_DOESNT_EXIST);
-  }
-
-  @Override
-  public boolean hasTablePermission(String principal, String tableName, TablePermission perm) throws AccumuloException, AccumuloSecurityException {
-    MockTable table = acu.tables.get(tableName);
-    if (table == null)
-      throw new AccumuloSecurityException(tableName, SecurityErrorCode.TABLE_DOESNT_EXIST);
-    EnumSet<TablePermission> perms = table.userPermissions.get(principal);
-    if (perms == null)
-      return false;
-    return perms.contains(perm);
-  }
-
-  @Override
-  public boolean hasNamespacePermission(String principal, String namespace, NamespacePermission permission) throws AccumuloException, AccumuloSecurityException {
-    MockNamespace mockNamespace = acu.namespaces.get(namespace);
-    if (mockNamespace == null)
-      throw new AccumuloSecurityException(namespace, SecurityErrorCode.NAMESPACE_DOESNT_EXIST);
-    EnumSet<NamespacePermission> perms = mockNamespace.userPermissions.get(principal);
-    if (perms == null)
-      return false;
-    return perms.contains(permission);
-  }
-
-  @Override
-  public void grantSystemPermission(String principal, SystemPermission permission) throws AccumuloException, AccumuloSecurityException {
-    MockUser user = acu.users.get(principal);
-    if (user != null)
-      user.permissions.add(permission);
-    else
-      throw new AccumuloSecurityException(principal, SecurityErrorCode.USER_DOESNT_EXIST);
-  }
-
-  @Override
-  public void grantTablePermission(String principal, String tableName, TablePermission permission) throws AccumuloException, AccumuloSecurityException {
-    if (acu.users.get(principal) == null)
-      throw new AccumuloSecurityException(principal, SecurityErrorCode.USER_DOESNT_EXIST);
-    MockTable table = acu.tables.get(tableName);
-    if (table == null)
-      throw new AccumuloSecurityException(tableName, SecurityErrorCode.TABLE_DOESNT_EXIST);
-    EnumSet<TablePermission> perms = table.userPermissions.get(principal);
-    if (perms == null)
-      table.userPermissions.put(principal, EnumSet.of(permission));
-    else
-      perms.add(permission);
-  }
-
-  @Override
-  public void grantNamespacePermission(String principal, String namespace, NamespacePermission permission) throws AccumuloException, AccumuloSecurityException {
-    if (acu.users.get(principal) == null)
-      throw new AccumuloSecurityException(principal, SecurityErrorCode.USER_DOESNT_EXIST);
-    MockNamespace mockNamespace = acu.namespaces.get(namespace);
-    if (mockNamespace == null)
-      throw new AccumuloSecurityException(namespace, SecurityErrorCode.NAMESPACE_DOESNT_EXIST);
-    EnumSet<NamespacePermission> perms = mockNamespace.userPermissions.get(principal);
-    if (perms == null)
-      mockNamespace.userPermissions.put(principal, EnumSet.of(permission));
-    else
-      perms.add(permission);
-  }
-
-  @Override
-  public void revokeSystemPermission(String principal, SystemPermission permission) throws AccumuloException, AccumuloSecurityException {
-    MockUser user = acu.users.get(principal);
-    if (user != null)
-      user.permissions.remove(permission);
-    else
-      throw new AccumuloSecurityException(principal, SecurityErrorCode.USER_DOESNT_EXIST);
-  }
-
-  @Override
-  public void revokeTablePermission(String principal, String tableName, TablePermission permission) throws AccumuloException, AccumuloSecurityException {
-    if (acu.users.get(principal) == null)
-      throw new AccumuloSecurityException(principal, SecurityErrorCode.USER_DOESNT_EXIST);
-    MockTable table = acu.tables.get(tableName);
-    if (table == null)
-      throw new AccumuloSecurityException(tableName, SecurityErrorCode.TABLE_DOESNT_EXIST);
-    EnumSet<TablePermission> perms = table.userPermissions.get(principal);
-    if (perms != null)
-      perms.remove(permission);
-
-  }
-
-  @Override
-  public void revokeNamespacePermission(String principal, String namespace, NamespacePermission permission) throws AccumuloException, AccumuloSecurityException {
-    if (acu.users.get(principal) == null)
-      throw new AccumuloSecurityException(principal, SecurityErrorCode.USER_DOESNT_EXIST);
-    MockNamespace mockNamespace = acu.namespaces.get(namespace);
-    if (mockNamespace == null)
-      throw new AccumuloSecurityException(namespace, SecurityErrorCode.NAMESPACE_DOESNT_EXIST);
-    EnumSet<NamespacePermission> perms = mockNamespace.userPermissions.get(principal);
-    if (perms != null)
-      perms.remove(permission);
-
-  }
-
-  @Deprecated
-  @Override
-  public Set<String> listUsers() throws AccumuloException, AccumuloSecurityException {
-    return listLocalUsers();
-  }
-
-  @Override
-  public Set<String> listLocalUsers() throws AccumuloException, AccumuloSecurityException {
-    return acu.users.keySet();
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/f277fea9/core/src/main/java/org/apache/accumulo/core/client/mock/MockTableOperations.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mock/MockTableOperations.java b/core/src/main/java/org/apache/accumulo/core/client/mock/MockTableOperations.java
index ee8880c..f1e1914 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mock/MockTableOperations.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mock/MockTableOperations.java
@@ -16,13 +16,432 @@
  */
 package org.apache.accumulo.core.client.mock;
 
-/**
- * @deprecated since 1.6.0; not intended for public api and you should not use it.
- */
-@Deprecated
-public class MockTableOperations extends MockTableOperationsImpl {
+import java.io.DataInputStream;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Set;
+import java.util.SortedSet;
+import java.util.TreeSet;
+
+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.TableExistsException;
+import org.apache.accumulo.core.client.TableNotFoundException;
+import org.apache.accumulo.core.client.admin.DiskUsage;
+import org.apache.accumulo.core.client.admin.FindMax;
+import org.apache.accumulo.core.client.impl.TableOperationsHelper;
+import org.apache.accumulo.core.client.admin.TimeType;
+import org.apache.accumulo.core.client.impl.Tables;
+import org.apache.accumulo.core.conf.AccumuloConfiguration;
+import org.apache.accumulo.core.data.Key;
+import org.apache.accumulo.core.data.Mutation;
+import org.apache.accumulo.core.data.Range;
+import org.apache.accumulo.core.data.Value;
+import org.apache.accumulo.core.file.FileOperations;
+import org.apache.accumulo.core.file.FileSKVIterator;
+import org.apache.accumulo.core.metadata.MetadataTable;
+import org.apache.accumulo.core.metadata.RootTable;
+import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.core.security.ColumnVisibility;
+import org.apache.accumulo.start.classloader.vfs.AccumuloVFSClassLoader;
+import org.apache.commons.lang.NotImplementedException;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.Text;
+
+class MockTableOperations extends TableOperationsHelper {
+  private static final byte[] ZERO = {0};
+  private final MockAccumulo acu;
+  private final String username;
 
   MockTableOperations(MockAccumulo acu, String username) {
-    super(acu,username);
+    this.acu = acu;
+    this.username = username;
+  }
+
+  @Override
+  public SortedSet<String> list() {
+    return new TreeSet<String>(acu.tables.keySet());
+  }
+
+  @Override
+  public boolean exists(String tableName) {
+    return acu.tables.containsKey(tableName);
+  }
+
+  private boolean namespaceExists(String namespace) {
+    return acu.namespaces.containsKey(namespace);
+  }
+
+  @Override
+  public void create(String tableName) throws AccumuloException, AccumuloSecurityException, TableExistsException {
+    create(tableName, true, TimeType.MILLIS);
+  }
+
+  @Override
+  public void create(String tableName, boolean versioningIter) throws AccumuloException, AccumuloSecurityException, TableExistsException {
+    create(tableName, versioningIter, TimeType.MILLIS);
+  }
+
+  @Override
+  public void create(String tableName, boolean versioningIter, TimeType timeType) throws AccumuloException, AccumuloSecurityException, TableExistsException {
+    String namespace = Tables.qualify(tableName).getFirst();
+    if (!tableName.matches(Tables.VALID_NAME_REGEX)) {
+      throw new IllegalArgumentException();
+    }
+    if (exists(tableName))
+      throw new TableExistsException(tableName, tableName, "");
+
+    if (!namespaceExists(namespace)) {
+      throw new IllegalArgumentException("Namespace (" + namespace + ") does not exist, create it first");
+    }
+    acu.createTable(username, tableName, versioningIter, timeType);
+  }
+
+  @Override
+  public void addSplits(String tableName, SortedSet<Text> partitionKeys) throws TableNotFoundException, AccumuloException, AccumuloSecurityException {
+    if (!exists(tableName))
+      throw new TableNotFoundException(tableName, tableName, "");
+    acu.addSplits(tableName, partitionKeys);
+  }
+
+  @Deprecated
+  @Override
+  public Collection<Text> getSplits(String tableName) throws TableNotFoundException {
+    return listSplits(tableName);
+  }
+
+  @Deprecated
+  @Override
+  public Collection<Text> getSplits(String tableName, int maxSplits) throws TableNotFoundException {
+    return listSplits(tableName);
+  }
+
+  @Override
+  public Collection<Text> listSplits(String tableName) throws TableNotFoundException {
+    if (!exists(tableName))
+      throw new TableNotFoundException(tableName, tableName, "");
+    return acu.getSplits(tableName);
+  }
+
+  @Override
+  public Collection<Text> listSplits(String tableName, int maxSplits) throws TableNotFoundException {
+    return listSplits(tableName);
+  }
+
+  @Override
+  public void delete(String tableName) throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
+    if (!exists(tableName))
+      throw new TableNotFoundException(tableName, tableName, "");
+    acu.tables.remove(tableName);
+  }
+
+  @Override
+  public void rename(String oldTableName, String newTableName) throws AccumuloSecurityException, TableNotFoundException, AccumuloException,
+      TableExistsException {
+    if (!exists(oldTableName))
+      throw new TableNotFoundException(oldTableName, oldTableName, "");
+    if (exists(newTableName))
+      throw new TableExistsException(newTableName, newTableName, "");
+    MockTable t = acu.tables.remove(oldTableName);
+    String namespace = Tables.qualify(newTableName).getFirst();
+    MockNamespace n = acu.namespaces.get(namespace);
+    if (n == null) {
+      n = new MockNamespace();
+    }
+    t.setNamespaceName(namespace);
+    t.setNamespace(n);
+    acu.namespaces.put(namespace, n);
+    acu.tables.put(newTableName, t);
+  }
+
+  @Deprecated
+  @Override
+  public void flush(String tableName) throws AccumuloException, AccumuloSecurityException {}
+
+  @Override
+  public void setProperty(String tableName, String property, String value) throws AccumuloException, AccumuloSecurityException {
+    acu.tables.get(tableName).settings.put(property, value);
+  }
+
+  @Override
+  public void removeProperty(String tableName, String property) throws AccumuloException, AccumuloSecurityException {
+    acu.tables.get(tableName).settings.remove(property);
+  }
+
+  @Override
+  public Iterable<Entry<String,String>> getProperties(String tableName) throws TableNotFoundException {
+    String namespace = Tables.qualify(tableName).getFirst();
+    if (!exists(tableName)) {
+      if (!namespaceExists(namespace))
+        throw new TableNotFoundException(tableName, new NamespaceNotFoundException(null, namespace, null));
+      throw new TableNotFoundException(null, tableName, null);
+    }
+
+    Set<Entry<String,String>> props = new HashSet<Entry<String,String>>(acu.namespaces.get(namespace).settings.entrySet());
+
+    Set<Entry<String,String>> tableProps = acu.tables.get(tableName).settings.entrySet();
+    for (Entry<String,String> e : tableProps) {
+      if (props.contains(e)) {
+        props.remove(e);
+      }
+      props.add(e);
+    }
+    return props;
+  }
+
+  @Override
+  public void setLocalityGroups(String tableName, Map<String,Set<Text>> groups) throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
+    if (!exists(tableName))
+      throw new TableNotFoundException(tableName, tableName, "");
+    acu.tables.get(tableName).setLocalityGroups(groups);
+  }
+
+  @Override
+  public Map<String,Set<Text>> getLocalityGroups(String tableName) throws AccumuloException, TableNotFoundException {
+    if (!exists(tableName))
+      throw new TableNotFoundException(tableName, tableName, "");
+    return acu.tables.get(tableName).getLocalityGroups();
+  }
+
+  @Override
+  public Set<Range> splitRangeByTablets(String tableName, Range range, int maxSplits) throws AccumuloException, AccumuloSecurityException,
+      TableNotFoundException {
+    if (!exists(tableName))
+      throw new TableNotFoundException(tableName, tableName, "");
+    return Collections.singleton(range);
+  }
+
+  @Override
+  public void importDirectory(String tableName, String dir, String failureDir, boolean setTime) throws IOException, AccumuloException,
+      AccumuloSecurityException, TableNotFoundException {
+    long time = System.currentTimeMillis();
+    MockTable table = acu.tables.get(tableName);
+    if (table == null) {
+      throw new TableNotFoundException(null, tableName, "The table was not found");
+    }
+    Path importPath = new Path(dir);
+    Path failurePath = new Path(failureDir);
+
+    FileSystem fs = acu.getFileSystem();
+    /*
+     * check preconditions
+     */
+    // directories are directories
+    if (fs.isFile(importPath)) {
+      throw new IOException("Import path must be a directory.");
+    }
+    if (fs.isFile(failurePath)) {
+      throw new IOException("Failure path must be a directory.");
+    }
+    // failures are writable
+    Path createPath = failurePath.suffix("/.createFile");
+    FSDataOutputStream createStream = null;
+    try {
+      createStream = fs.create(createPath);
+    } catch (IOException e) {
+      throw new IOException("Error path is not writable.");
+    } finally {
+      if (createStream != null) {
+        createStream.close();
+      }
+    }
+    fs.delete(createPath, false);
+    // failures are empty
+    FileStatus[] failureChildStats = fs.listStatus(failurePath);
+    if (failureChildStats.length > 0) {
+      throw new IOException("Error path must be empty.");
+    }
+    /*
+     * Begin the import - iterate the files in the path
+     */
+    for (FileStatus importStatus : fs.listStatus(importPath)) {
+      try {
+        FileSKVIterator importIterator = FileOperations.getInstance().openReader(importStatus.getPath().toString(), true, fs, fs.getConf(),
+            AccumuloConfiguration.getDefaultConfiguration());
+        while (importIterator.hasTop()) {
+          Key key = importIterator.getTopKey();
+          Value value = importIterator.getTopValue();
+          if (setTime) {
+            key.setTimestamp(time);
+          }
+          Mutation mutation = new Mutation(key.getRow());
+          if (!key.isDeleted()) {
+            mutation.put(key.getColumnFamily(), key.getColumnQualifier(), new ColumnVisibility(key.getColumnVisibilityData().toArray()), key.getTimestamp(),
+                value);
+          } else {
+            mutation.putDelete(key.getColumnFamily(), key.getColumnQualifier(), new ColumnVisibility(key.getColumnVisibilityData().toArray()),
+                key.getTimestamp());
+          }
+          table.addMutation(mutation);
+          importIterator.next();
+        }
+      } catch (Exception e) {
+        FSDataOutputStream failureWriter = null;
+        DataInputStream failureReader = null;
+        try {
+          failureWriter = fs.create(failurePath.suffix("/" + importStatus.getPath().getName()));
+          failureReader = fs.open(importStatus.getPath());
+          int read = 0;
+          byte[] buffer = new byte[1024];
+          while (-1 != (read = failureReader.read(buffer))) {
+            failureWriter.write(buffer, 0, read);
+          }
+        } finally {
+          if (failureReader != null)
+            failureReader.close();
+          if (failureWriter != null)
+            failureWriter.close();
+        }
+      }
+      fs.delete(importStatus.getPath(), true);
+    }
+  }
+
+  @Override
+  public void offline(String tableName) throws AccumuloSecurityException, AccumuloException, TableNotFoundException {
+    offline(tableName, false);
+  }
+
+  @Override
+  public void offline(String tableName, boolean wait) throws AccumuloSecurityException, AccumuloException, TableNotFoundException {
+    if (!exists(tableName))
+      throw new TableNotFoundException(tableName, tableName, "");
+  }
+
+  @Override
+  public void online(String tableName) throws AccumuloSecurityException, AccumuloException, TableNotFoundException {
+    online(tableName, false);
+  }
+
+  @Override
+  public void online(String tableName, boolean wait) throws AccumuloSecurityException, AccumuloException, TableNotFoundException {
+    if (!exists(tableName))
+      throw new TableNotFoundException(tableName, tableName, "");
+  }
+
+  @Override
+  public void clearLocatorCache(String tableName) throws TableNotFoundException {
+    if (!exists(tableName))
+      throw new TableNotFoundException(tableName, tableName, "");
+  }
+
+  @Override
+  public Map<String,String> tableIdMap() {
+    Map<String,String> result = new HashMap<String,String>();
+    for (String table : acu.tables.keySet()) {
+      if (RootTable.NAME.equals(table))
+        result.put(table, RootTable.ID);
+      else if (MetadataTable.NAME.equals(table))
+        result.put(table, MetadataTable.ID);
+      else
+        result.put(table, table);
+    }
+    return result;
+  }
+
+  @Override
+  public List<DiskUsage> getDiskUsage(Set<String> tables) throws AccumuloException, AccumuloSecurityException {
+
+    List<DiskUsage> diskUsages = new ArrayList<DiskUsage>();
+    diskUsages.add(new DiskUsage(new TreeSet<String>(tables), 0l));
+
+    return diskUsages;
+  }
+
+  @Override
+  public void merge(String tableName, Text start, Text end) throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
+    if (!exists(tableName))
+      throw new TableNotFoundException(tableName, tableName, "");
+    acu.merge(tableName, start, end);
+  }
+
+  @Override
+  public void deleteRows(String tableName, Text start, Text end) throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
+    if (!exists(tableName))
+      throw new TableNotFoundException(tableName, tableName, "");
+    MockTable t = acu.tables.get(tableName);
+    Text startText = new Text(start);
+    Text endText = new Text(end);
+    startText.append(ZERO, 0, 1);
+    endText.append(ZERO, 0, 1);
+    Set<Key> keep = new TreeSet<Key>(t.table.subMap(new Key(startText), new Key(endText)).keySet());
+    t.table.keySet().removeAll(keep);
+  }
+
+  @Override
+  public void compact(String tableName, Text start, Text end, boolean flush, boolean wait) throws AccumuloSecurityException, TableNotFoundException,
+      AccumuloException {
+    if (!exists(tableName))
+      throw new TableNotFoundException(tableName, tableName, "");
+  }
+
+  @Override
+  public void compact(String tableName, Text start, Text end, List<IteratorSetting> iterators, boolean flush, boolean wait) throws AccumuloSecurityException,
+      TableNotFoundException, AccumuloException {
+    if (!exists(tableName))
+      throw new TableNotFoundException(tableName, tableName, "");
+  }
+
+  @Override
+  public void cancelCompaction(String tableName) throws AccumuloSecurityException, TableNotFoundException, AccumuloException {
+    if (!exists(tableName))
+      throw new TableNotFoundException(tableName, tableName, "");
+  }
+
+  @Override
+  public void clone(String srcTableName, String newTableName, boolean flush, Map<String,String> propertiesToSet, Set<String> propertiesToExclude)
+      throws AccumuloException, AccumuloSecurityException, TableNotFoundException, TableExistsException {
+    throw new NotImplementedException();
+  }
+
+  @Override
+  public void flush(String tableName, Text start, Text end, boolean wait) throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
+    if (!exists(tableName))
+      throw new TableNotFoundException(tableName, tableName, "");
+  }
+
+  @Override
+  public Text getMaxRow(String tableName, Authorizations auths, Text startRow, boolean startInclusive, Text endRow, boolean endInclusive)
+      throws TableNotFoundException, AccumuloException, AccumuloSecurityException {
+    MockTable table = acu.tables.get(tableName);
+    if (table == null)
+      throw new TableNotFoundException(tableName, tableName, "no such table");
+
+    return FindMax.findMax(new MockScanner(table, auths), startRow, startInclusive, endRow, endInclusive);
+  }
+
+  @Override
+  public void importTable(String tableName, String exportDir) throws TableExistsException, AccumuloException, AccumuloSecurityException {
+    throw new NotImplementedException();
+  }
+
+  @Override
+  public void exportTable(String tableName, String exportDir) throws TableNotFoundException, AccumuloException, AccumuloSecurityException {
+    throw new NotImplementedException();
+  }
+
+  @Override
+  public boolean testClassLoad(String tableName, String className, String asTypeName) throws AccumuloException, AccumuloSecurityException,
+      TableNotFoundException {
+
+    try {
+      AccumuloVFSClassLoader.loadClass(className, Class.forName(asTypeName));
+    } catch (ClassNotFoundException e) {
+      e.printStackTrace();
+      return false;
+    }
+    return true;
   }
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/f277fea9/core/src/main/java/org/apache/accumulo/core/client/mock/MockTableOperationsImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mock/MockTableOperationsImpl.java b/core/src/main/java/org/apache/accumulo/core/client/mock/MockTableOperationsImpl.java
deleted file mode 100644
index fea9568..0000000
--- a/core/src/main/java/org/apache/accumulo/core/client/mock/MockTableOperationsImpl.java
+++ /dev/null
@@ -1,447 +0,0 @@
-/*
- * 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.core.client.mock;
-
-import java.io.DataInputStream;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Set;
-import java.util.SortedSet;
-import java.util.TreeSet;
-
-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.TableExistsException;
-import org.apache.accumulo.core.client.TableNotFoundException;
-import org.apache.accumulo.core.client.admin.DiskUsage;
-import org.apache.accumulo.core.client.admin.FindMax;
-import org.apache.accumulo.core.client.impl.TableOperationsHelper;
-import org.apache.accumulo.core.client.admin.TimeType;
-import org.apache.accumulo.core.client.impl.Tables;
-import org.apache.accumulo.core.conf.AccumuloConfiguration;
-import org.apache.accumulo.core.data.Key;
-import org.apache.accumulo.core.data.Mutation;
-import org.apache.accumulo.core.data.Range;
-import org.apache.accumulo.core.data.Value;
-import org.apache.accumulo.core.file.FileOperations;
-import org.apache.accumulo.core.file.FileSKVIterator;
-import org.apache.accumulo.core.metadata.MetadataTable;
-import org.apache.accumulo.core.metadata.RootTable;
-import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.core.security.ColumnVisibility;
-import org.apache.accumulo.start.classloader.vfs.AccumuloVFSClassLoader;
-import org.apache.commons.lang.NotImplementedException;
-import org.apache.hadoop.fs.FSDataOutputStream;
-import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.Text;
-
-class MockTableOperationsImpl extends TableOperationsHelper {
-  private static final byte[] ZERO = {0};
-  private final MockAccumulo acu;
-  private final String username;
-
-  MockTableOperationsImpl(MockAccumulo acu, String username) {
-    this.acu = acu;
-    this.username = username;
-  }
-
-  @Override
-  public SortedSet<String> list() {
-    return new TreeSet<String>(acu.tables.keySet());
-  }
-
-  @Override
-  public boolean exists(String tableName) {
-    return acu.tables.containsKey(tableName);
-  }
-
-  private boolean namespaceExists(String namespace) {
-    return acu.namespaces.containsKey(namespace);
-  }
-
-  @Override
-  public void create(String tableName) throws AccumuloException, AccumuloSecurityException, TableExistsException {
-    create(tableName, true, TimeType.MILLIS);
-  }
-
-  @Override
-  public void create(String tableName, boolean versioningIter) throws AccumuloException, AccumuloSecurityException, TableExistsException {
-    create(tableName, versioningIter, TimeType.MILLIS);
-  }
-
-  @Override
-  public void create(String tableName, boolean versioningIter, TimeType timeType) throws AccumuloException, AccumuloSecurityException, TableExistsException {
-    String namespace = Tables.qualify(tableName).getFirst();
-    if (!tableName.matches(Tables.VALID_NAME_REGEX)) {
-      throw new IllegalArgumentException();
-    }
-    if (exists(tableName))
-      throw new TableExistsException(tableName, tableName, "");
-
-    if (!namespaceExists(namespace)) {
-      throw new IllegalArgumentException("Namespace (" + namespace + ") does not exist, create it first");
-    }
-    acu.createTable(username, tableName, versioningIter, timeType);
-  }
-
-  @Override
-  public void addSplits(String tableName, SortedSet<Text> partitionKeys) throws TableNotFoundException, AccumuloException, AccumuloSecurityException {
-    if (!exists(tableName))
-      throw new TableNotFoundException(tableName, tableName, "");
-    acu.addSplits(tableName, partitionKeys);
-  }
-
-  @Deprecated
-  @Override
-  public Collection<Text> getSplits(String tableName) throws TableNotFoundException {
-    return listSplits(tableName);
-  }
-
-  @Deprecated
-  @Override
-  public Collection<Text> getSplits(String tableName, int maxSplits) throws TableNotFoundException {
-    return listSplits(tableName);
-  }
-
-  @Override
-  public Collection<Text> listSplits(String tableName) throws TableNotFoundException {
-    if (!exists(tableName))
-      throw new TableNotFoundException(tableName, tableName, "");
-    return acu.getSplits(tableName);
-  }
-
-  @Override
-  public Collection<Text> listSplits(String tableName, int maxSplits) throws TableNotFoundException {
-    return listSplits(tableName);
-  }
-
-  @Override
-  public void delete(String tableName) throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
-    if (!exists(tableName))
-      throw new TableNotFoundException(tableName, tableName, "");
-    acu.tables.remove(tableName);
-  }
-
-  @Override
-  public void rename(String oldTableName, String newTableName) throws AccumuloSecurityException, TableNotFoundException, AccumuloException,
-      TableExistsException {
-    if (!exists(oldTableName))
-      throw new TableNotFoundException(oldTableName, oldTableName, "");
-    if (exists(newTableName))
-      throw new TableExistsException(newTableName, newTableName, "");
-    MockTable t = acu.tables.remove(oldTableName);
-    String namespace = Tables.qualify(newTableName).getFirst();
-    MockNamespace n = acu.namespaces.get(namespace);
-    if (n == null) {
-      n = new MockNamespace();
-    }
-    t.setNamespaceName(namespace);
-    t.setNamespace(n);
-    acu.namespaces.put(namespace, n);
-    acu.tables.put(newTableName, t);
-  }
-
-  @Deprecated
-  @Override
-  public void flush(String tableName) throws AccumuloException, AccumuloSecurityException {}
-
-  @Override
-  public void setProperty(String tableName, String property, String value) throws AccumuloException, AccumuloSecurityException {
-    acu.tables.get(tableName).settings.put(property, value);
-  }
-
-  @Override
-  public void removeProperty(String tableName, String property) throws AccumuloException, AccumuloSecurityException {
-    acu.tables.get(tableName).settings.remove(property);
-  }
-
-  @Override
-  public Iterable<Entry<String,String>> getProperties(String tableName) throws TableNotFoundException {
-    String namespace = Tables.qualify(tableName).getFirst();
-    if (!exists(tableName)) {
-      if (!namespaceExists(namespace))
-        throw new TableNotFoundException(tableName, new NamespaceNotFoundException(null, namespace, null));
-      throw new TableNotFoundException(null, tableName, null);
-    }
-
-    Set<Entry<String,String>> props = new HashSet<Entry<String,String>>(acu.namespaces.get(namespace).settings.entrySet());
-
-    Set<Entry<String,String>> tableProps = acu.tables.get(tableName).settings.entrySet();
-    for (Entry<String,String> e : tableProps) {
-      if (props.contains(e)) {
-        props.remove(e);
-      }
-      props.add(e);
-    }
-    return props;
-  }
-
-  @Override
-  public void setLocalityGroups(String tableName, Map<String,Set<Text>> groups) throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
-    if (!exists(tableName))
-      throw new TableNotFoundException(tableName, tableName, "");
-    acu.tables.get(tableName).setLocalityGroups(groups);
-  }
-
-  @Override
-  public Map<String,Set<Text>> getLocalityGroups(String tableName) throws AccumuloException, TableNotFoundException {
-    if (!exists(tableName))
-      throw new TableNotFoundException(tableName, tableName, "");
-    return acu.tables.get(tableName).getLocalityGroups();
-  }
-
-  @Override
-  public Set<Range> splitRangeByTablets(String tableName, Range range, int maxSplits) throws AccumuloException, AccumuloSecurityException,
-      TableNotFoundException {
-    if (!exists(tableName))
-      throw new TableNotFoundException(tableName, tableName, "");
-    return Collections.singleton(range);
-  }
-
-  @Override
-  public void importDirectory(String tableName, String dir, String failureDir, boolean setTime) throws IOException, AccumuloException,
-      AccumuloSecurityException, TableNotFoundException {
-    long time = System.currentTimeMillis();
-    MockTable table = acu.tables.get(tableName);
-    if (table == null) {
-      throw new TableNotFoundException(null, tableName, "The table was not found");
-    }
-    Path importPath = new Path(dir);
-    Path failurePath = new Path(failureDir);
-
-    FileSystem fs = acu.getFileSystem();
-    /*
-     * check preconditions
-     */
-    // directories are directories
-    if (fs.isFile(importPath)) {
-      throw new IOException("Import path must be a directory.");
-    }
-    if (fs.isFile(failurePath)) {
-      throw new IOException("Failure path must be a directory.");
-    }
-    // failures are writable
-    Path createPath = failurePath.suffix("/.createFile");
-    FSDataOutputStream createStream = null;
-    try {
-      createStream = fs.create(createPath);
-    } catch (IOException e) {
-      throw new IOException("Error path is not writable.");
-    } finally {
-      if (createStream != null) {
-        createStream.close();
-      }
-    }
-    fs.delete(createPath, false);
-    // failures are empty
-    FileStatus[] failureChildStats = fs.listStatus(failurePath);
-    if (failureChildStats.length > 0) {
-      throw new IOException("Error path must be empty.");
-    }
-    /*
-     * Begin the import - iterate the files in the path
-     */
-    for (FileStatus importStatus : fs.listStatus(importPath)) {
-      try {
-        FileSKVIterator importIterator = FileOperations.getInstance().openReader(importStatus.getPath().toString(), true, fs, fs.getConf(),
-            AccumuloConfiguration.getDefaultConfiguration());
-        while (importIterator.hasTop()) {
-          Key key = importIterator.getTopKey();
-          Value value = importIterator.getTopValue();
-          if (setTime) {
-            key.setTimestamp(time);
-          }
-          Mutation mutation = new Mutation(key.getRow());
-          if (!key.isDeleted()) {
-            mutation.put(key.getColumnFamily(), key.getColumnQualifier(), new ColumnVisibility(key.getColumnVisibilityData().toArray()), key.getTimestamp(),
-                value);
-          } else {
-            mutation.putDelete(key.getColumnFamily(), key.getColumnQualifier(), new ColumnVisibility(key.getColumnVisibilityData().toArray()),
-                key.getTimestamp());
-          }
-          table.addMutation(mutation);
-          importIterator.next();
-        }
-      } catch (Exception e) {
-        FSDataOutputStream failureWriter = null;
-        DataInputStream failureReader = null;
-        try {
-          failureWriter = fs.create(failurePath.suffix("/" + importStatus.getPath().getName()));
-          failureReader = fs.open(importStatus.getPath());
-          int read = 0;
-          byte[] buffer = new byte[1024];
-          while (-1 != (read = failureReader.read(buffer))) {
-            failureWriter.write(buffer, 0, read);
-          }
-        } finally {
-          if (failureReader != null)
-            failureReader.close();
-          if (failureWriter != null)
-            failureWriter.close();
-        }
-      }
-      fs.delete(importStatus.getPath(), true);
-    }
-  }
-
-  @Override
-  public void offline(String tableName) throws AccumuloSecurityException, AccumuloException, TableNotFoundException {
-    offline(tableName, false);
-  }
-
-  @Override
-  public void offline(String tableName, boolean wait) throws AccumuloSecurityException, AccumuloException, TableNotFoundException {
-    if (!exists(tableName))
-      throw new TableNotFoundException(tableName, tableName, "");
-  }
-
-  @Override
-  public void online(String tableName) throws AccumuloSecurityException, AccumuloException, TableNotFoundException {
-    online(tableName, false);
-  }
-
-  @Override
-  public void online(String tableName, boolean wait) throws AccumuloSecurityException, AccumuloException, TableNotFoundException {
-    if (!exists(tableName))
-      throw new TableNotFoundException(tableName, tableName, "");
-  }
-
-  @Override
-  public void clearLocatorCache(String tableName) throws TableNotFoundException {
-    if (!exists(tableName))
-      throw new TableNotFoundException(tableName, tableName, "");
-  }
-
-  @Override
-  public Map<String,String> tableIdMap() {
-    Map<String,String> result = new HashMap<String,String>();
-    for (String table : acu.tables.keySet()) {
-      if (RootTable.NAME.equals(table))
-        result.put(table, RootTable.ID);
-      else if (MetadataTable.NAME.equals(table))
-        result.put(table, MetadataTable.ID);
-      else
-        result.put(table, table);
-    }
-    return result;
-  }
-
-  @Override
-  public List<DiskUsage> getDiskUsage(Set<String> tables) throws AccumuloException, AccumuloSecurityException {
-
-    List<DiskUsage> diskUsages = new ArrayList<DiskUsage>();
-    diskUsages.add(new DiskUsage(new TreeSet<String>(tables), 0l));
-
-    return diskUsages;
-  }
-
-  @Override
-  public void merge(String tableName, Text start, Text end) throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
-    if (!exists(tableName))
-      throw new TableNotFoundException(tableName, tableName, "");
-    acu.merge(tableName, start, end);
-  }
-
-  @Override
-  public void deleteRows(String tableName, Text start, Text end) throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
-    if (!exists(tableName))
-      throw new TableNotFoundException(tableName, tableName, "");
-    MockTable t = acu.tables.get(tableName);
-    Text startText = new Text(start);
-    Text endText = new Text(end);
-    startText.append(ZERO, 0, 1);
-    endText.append(ZERO, 0, 1);
-    Set<Key> keep = new TreeSet<Key>(t.table.subMap(new Key(startText), new Key(endText)).keySet());
-    t.table.keySet().removeAll(keep);
-  }
-
-  @Override
-  public void compact(String tableName, Text start, Text end, boolean flush, boolean wait) throws AccumuloSecurityException, TableNotFoundException,
-      AccumuloException {
-    if (!exists(tableName))
-      throw new TableNotFoundException(tableName, tableName, "");
-  }
-
-  @Override
-  public void compact(String tableName, Text start, Text end, List<IteratorSetting> iterators, boolean flush, boolean wait) throws AccumuloSecurityException,
-      TableNotFoundException, AccumuloException {
-    if (!exists(tableName))
-      throw new TableNotFoundException(tableName, tableName, "");
-  }
-
-  @Override
-  public void cancelCompaction(String tableName) throws AccumuloSecurityException, TableNotFoundException, AccumuloException {
-    if (!exists(tableName))
-      throw new TableNotFoundException(tableName, tableName, "");
-  }
-
-  @Override
-  public void clone(String srcTableName, String newTableName, boolean flush, Map<String,String> propertiesToSet, Set<String> propertiesToExclude)
-      throws AccumuloException, AccumuloSecurityException, TableNotFoundException, TableExistsException {
-    throw new NotImplementedException();
-  }
-
-  @Override
-  public void flush(String tableName, Text start, Text end, boolean wait) throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
-    if (!exists(tableName))
-      throw new TableNotFoundException(tableName, tableName, "");
-  }
-
-  @Override
-  public Text getMaxRow(String tableName, Authorizations auths, Text startRow, boolean startInclusive, Text endRow, boolean endInclusive)
-      throws TableNotFoundException, AccumuloException, AccumuloSecurityException {
-    MockTable table = acu.tables.get(tableName);
-    if (table == null)
-      throw new TableNotFoundException(tableName, tableName, "no such table");
-
-    return FindMax.findMax(new MockScanner(table, auths), startRow, startInclusive, endRow, endInclusive);
-  }
-
-  @Override
-  public void importTable(String tableName, String exportDir) throws TableExistsException, AccumuloException, AccumuloSecurityException {
-    throw new NotImplementedException();
-  }
-
-  @Override
-  public void exportTable(String tableName, String exportDir) throws TableNotFoundException, AccumuloException, AccumuloSecurityException {
-    throw new NotImplementedException();
-  }
-
-  @Override
-  public boolean testClassLoad(String tableName, String className, String asTypeName) throws AccumuloException, AccumuloSecurityException,
-      TableNotFoundException {
-
-    try {
-      AccumuloVFSClassLoader.loadClass(className, Class.forName(asTypeName));
-    } catch (ClassNotFoundException e) {
-      e.printStackTrace();
-      return false;
-    }
-    return true;
-  }
-}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/f277fea9/core/src/main/java/org/apache/accumulo/core/client/mock/MockTabletLocator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mock/MockTabletLocator.java b/core/src/main/java/org/apache/accumulo/core/client/mock/MockTabletLocator.java
deleted file mode 100644
index ac6a6e9..0000000
--- a/core/src/main/java/org/apache/accumulo/core/client/mock/MockTabletLocator.java
+++ /dev/null
@@ -1,53 +0,0 @@
-/*
- * 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.core.client.mock;
-
-import java.util.List;
-import java.util.Map;
-
-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.data.KeyExtent;
-import org.apache.accumulo.core.data.Mutation;
-import org.apache.accumulo.core.data.Range;
-import org.apache.accumulo.core.security.Credentials;
-import org.apache.accumulo.core.security.thrift.TCredentials;
-import org.apache.hadoop.io.Text;
-
-/**
- * @deprecated since 1.6.0; not intended for public api and you should not use it.
- */
-@Deprecated
-public class MockTabletLocator extends org.apache.accumulo.core.client.mock.impl.MockTabletLocator {
-  public MockTabletLocator() {}
-
-  public TabletLocation locateTablet(Text row, boolean skipRow, boolean retry, TCredentials credentials) throws AccumuloException, AccumuloSecurityException,
-      TableNotFoundException {
-    return locateTablet(Credentials.fromThrift(credentials), row, skipRow, retry);
-  }
-  
-  public <T extends Mutation> void binMutations(List<T> mutations, Map<String,TabletServerMutations<T>> binnedMutations, List<T> failures, TCredentials credentials)
-      throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
-    binMutations(Credentials.fromThrift(credentials), mutations, binnedMutations, failures);
-  }
-
-  public List<Range> binRanges(List<Range> ranges, Map<String,Map<KeyExtent,List<Range>>> binnedRanges, TCredentials credentials) throws AccumuloException,
-      AccumuloSecurityException, TableNotFoundException {
-    return binRanges(Credentials.fromThrift(credentials), ranges, binnedRanges);
-  }
-}


[17/17] git commit: Merge branch '1.6.0-SNAPSHOT'

Posted by bu...@apache.org.
Merge branch '1.6.0-SNAPSHOT'


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

Branch: refs/heads/master
Commit: eae953e6afd36885f502d368b946a35d380cb14d
Parents: f277fea 448e757
Author: Sean Busbey <bu...@cloudera.com>
Authored: Fri Apr 25 18:11:43 2014 -0500
Committer: Sean Busbey <bu...@cloudera.com>
Committed: Fri Apr 25 18:11:43 2014 -0500

----------------------------------------------------------------------
 CHANGES | 73 ++++++++++++++++++++----------------------------------------
 1 file changed, 24 insertions(+), 49 deletions(-)
----------------------------------------------------------------------



[10/17] Merge branch '1.6.0-SNAPSHOT'

Posted by bu...@apache.org.
http://git-wip-us.apache.org/repos/asf/accumulo/blob/592c8b06/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperationsImpl.java
----------------------------------------------------------------------
diff --cc core/src/main/java/org/apache/accumulo/core/client/admin/TableOperationsImpl.java
index 4923e65,05b5b01..95ba3d8
--- a/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperationsImpl.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperationsImpl.java
@@@ -16,121 -16,15 +16,83 @@@
   */
  package org.apache.accumulo.core.client.admin;
  
 +import static com.google.common.base.Preconditions.checkArgument;
 +
 +import java.io.BufferedReader;
 +import java.io.IOException;
 +import java.io.InputStreamReader;
 +import java.nio.ByteBuffer;
 +import java.nio.charset.StandardCharsets;
 +import java.util.ArrayList;
 +import java.util.Arrays;
 +import java.util.Collection;
 +import java.util.Collections;
 +import java.util.EnumSet;
 +import java.util.HashMap;
 +import java.util.HashSet;
 +import java.util.Iterator;
 +import java.util.LinkedList;
 +import java.util.List;
 +import java.util.Map;
 +import java.util.Map.Entry;
 +import java.util.Random;
 +import java.util.Set;
 +import java.util.SortedSet;
 +import java.util.TreeMap;
 +import java.util.TreeSet;
 +import java.util.concurrent.CountDownLatch;
 +import java.util.concurrent.ExecutorService;
 +import java.util.concurrent.Executors;
 +import java.util.concurrent.TimeUnit;
 +import java.util.concurrent.atomic.AtomicReference;
 +import java.util.zip.ZipEntry;
 +import java.util.zip.ZipInputStream;
 +
 +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.Instance;
- import org.apache.accumulo.core.client.IsolatedScanner;
- import org.apache.accumulo.core.client.IteratorSetting;
- import org.apache.accumulo.core.client.NamespaceExistsException;
- import org.apache.accumulo.core.client.NamespaceNotFoundException;
- import org.apache.accumulo.core.client.RowIterator;
- import org.apache.accumulo.core.client.Scanner;
- import org.apache.accumulo.core.client.TableDeletedException;
- import org.apache.accumulo.core.client.TableExistsException;
- import org.apache.accumulo.core.client.TableNotFoundException;
- import org.apache.accumulo.core.client.TableOfflineException;
- import org.apache.accumulo.core.client.impl.AccumuloServerException;
- import org.apache.accumulo.core.client.impl.ClientExec;
- import org.apache.accumulo.core.client.impl.ClientExecReturn;
- import org.apache.accumulo.core.client.impl.MasterClient;
- import org.apache.accumulo.core.client.impl.ServerClient;
- import org.apache.accumulo.core.client.impl.ServerConfigurationUtil;
- import org.apache.accumulo.core.client.impl.Tables;
- import org.apache.accumulo.core.client.impl.TabletLocator;
- import org.apache.accumulo.core.client.impl.TabletLocator.TabletLocation;
- import org.apache.accumulo.core.client.impl.thrift.ClientService;
- import org.apache.accumulo.core.client.impl.thrift.ClientService.Client;
- import org.apache.accumulo.core.client.impl.thrift.TDiskUsage;
- import org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException;
- import org.apache.accumulo.core.client.impl.thrift.ThriftTableOperationException;
- import org.apache.accumulo.core.conf.AccumuloConfiguration;
- import org.apache.accumulo.core.conf.ConfigurationCopy;
- import org.apache.accumulo.core.conf.Property;
- import org.apache.accumulo.core.constraints.Constraint;
- import org.apache.accumulo.core.data.ByteSequence;
- import org.apache.accumulo.core.data.Key;
- import org.apache.accumulo.core.data.KeyExtent;
- import org.apache.accumulo.core.data.Range;
- import org.apache.accumulo.core.data.Value;
- import org.apache.accumulo.core.iterators.IteratorUtil;
- import org.apache.accumulo.core.iterators.IteratorUtil.IteratorScope;
- import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
- import org.apache.accumulo.core.master.state.tables.TableState;
- import org.apache.accumulo.core.master.thrift.FateOperation;
- import org.apache.accumulo.core.master.thrift.MasterClientService;
- import org.apache.accumulo.core.metadata.MetadataServicer;
- import org.apache.accumulo.core.metadata.MetadataTable;
- import org.apache.accumulo.core.metadata.RootTable;
- import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection;
- import org.apache.accumulo.core.security.Authorizations;
  import org.apache.accumulo.core.security.Credentials;
 +import org.apache.accumulo.core.tabletserver.thrift.NotServingTabletException;
 +import org.apache.accumulo.core.tabletserver.thrift.TabletClientService;
 +import org.apache.accumulo.core.util.ByteBufferUtil;
 +import org.apache.accumulo.core.util.CachedConfiguration;
 +import org.apache.accumulo.core.util.LocalityGroupUtil;
 +import org.apache.accumulo.core.util.MapCounter;
 +import org.apache.accumulo.core.util.NamingThreadFactory;
 +import org.apache.accumulo.core.util.OpTimer;
 +import org.apache.accumulo.core.util.Pair;
 +import org.apache.accumulo.core.util.TextUtil;
 +import org.apache.accumulo.core.util.ThriftUtil;
 +import org.apache.accumulo.core.util.UtilWaitThread;
 +import org.apache.accumulo.core.volume.VolumeConfiguration;
 +import org.apache.accumulo.trace.instrument.Tracer;
 +import org.apache.hadoop.fs.FileStatus;
 +import org.apache.hadoop.fs.FileSystem;
 +import org.apache.hadoop.fs.Path;
 +import org.apache.hadoop.io.Text;
 +import org.apache.log4j.Level;
 +import org.apache.log4j.Logger;
 +import org.apache.thrift.TApplicationException;
 +import org.apache.thrift.TException;
 +import org.apache.thrift.transport.TTransportException;
+ import org.apache.accumulo.core.security.thrift.TCredentials;
  
 +import com.google.common.base.Joiner;
 +
 +public class TableOperationsImpl extends TableOperationsHelper {
 +  private Instance instance;
 +  private Credentials credentials;
 +
 +  public static final String CLONE_EXCLUDE_PREFIX = "!";
 +
 +  private static final Logger log = Logger.getLogger(TableOperations.class);
 +
+ /**
+  * @deprecated since 1.6.0; not intended for public api and you should not use it.
+  */
+ @Deprecated
+ public class TableOperationsImpl extends org.apache.accumulo.core.client.impl.TableOperationsImpl {
    /**
     * @param instance
     *          the connection information for this instance
@@@ -138,1473 -32,16 +100,1482 @@@
     *          the username/password for this connection
     */
    public TableOperationsImpl(Instance instance, Credentials credentials) {
 -    super(instance, credentials);
 +    checkArgument(instance != null, "instance is null");
 +    checkArgument(credentials != null, "credentials is null");
 +    this.instance = instance;
 +    this.credentials = credentials;
 +  }
 +
 +  /**
 +   * Retrieve a list of tables in Accumulo.
 +   * 
 +   * @return List of tables in accumulo
 +   */
 +  @Override
 +  public SortedSet<String> list() {
 +    OpTimer opTimer = new OpTimer(log, Level.TRACE).start("Fetching list of tables...");
 +    TreeSet<String> tableNames = new TreeSet<String>(Tables.getNameToIdMap(instance).keySet());
 +    opTimer.stop("Fetched " + tableNames.size() + " table names in %DURATION%");
 +    return tableNames;
 +  }
 +
 +  /**
 +   * A method to check if a table exists in Accumulo.
 +   * 
 +   * @param tableName
 +   *          the name of the table
 +   * @return true if the table exists
 +   */
 +  @Override
 +  public boolean exists(String tableName) {
 +    checkArgument(tableName != null, "tableName is null");
 +    if (tableName.equals(MetadataTable.NAME) || tableName.equals(RootTable.NAME))
 +      return true;
 +
 +    OpTimer opTimer = new OpTimer(log, Level.TRACE).start("Checking if table " + tableName + " exists...");
 +    boolean exists = Tables.getNameToIdMap(instance).containsKey(tableName);
 +    opTimer.stop("Checked existance of " + exists + " in %DURATION%");
 +    return exists;
 +  }
 +
 +  /**
 +   * Create a table with no special configuration
 +   * 
 +   * @param tableName
 +   *          the name of the table
 +   * @throws AccumuloException
 +   *           if a general error occurs
 +   * @throws AccumuloSecurityException
 +   *           if the user does not have permission
 +   * @throws TableExistsException
 +   *           if the table already exists
 +   */
 +  @Override
 +  public void create(String tableName) throws AccumuloException, AccumuloSecurityException, TableExistsException {
 +    create(tableName, true, TimeType.MILLIS);
 +  }
 +
 +  /**
 +   * @param tableName
 +   *          the name of the table
 +   * @param limitVersion
 +   *          Enables/disables the versioning iterator, which will limit the number of Key versions kept.
 +   */
 +  @Override
 +  public void create(String tableName, boolean limitVersion) throws AccumuloException, AccumuloSecurityException, TableExistsException {
 +    create(tableName, limitVersion, TimeType.MILLIS);
 +  }
 +
 +  /**
 +   * @param tableName
 +   *          the name of the table
 +   * @param timeType
 +   *          specifies logical or real-time based time recording for entries in the table
 +   * @param limitVersion
 +   *          Enables/disables the versioning iterator, which will limit the number of Key versions kept.
 +   */
 +  @Override
 +  public void create(String tableName, boolean limitVersion, TimeType timeType) throws AccumuloException, AccumuloSecurityException, TableExistsException {
 +    checkArgument(tableName != null, "tableName is null");
 +    checkArgument(timeType != null, "timeType is null");
 +
 +    List<ByteBuffer> args = Arrays.asList(ByteBuffer.wrap(tableName.getBytes(StandardCharsets.UTF_8)), ByteBuffer.wrap(timeType.name().getBytes(StandardCharsets.UTF_8)));
 +
 +    Map<String,String> opts;
 +    if (limitVersion)
 +      opts = IteratorUtil.generateInitialTableProperties(limitVersion);
 +    else
 +      opts = Collections.emptyMap();
 +
 +    try {
 +      doTableFateOperation(tableName, AccumuloException.class, FateOperation.TABLE_CREATE, args, opts);
 +    } catch (TableNotFoundException e) {
 +      // should not happen
 +      throw new AssertionError(e);
 +    }
 +  }
 +
 +  private long beginFateOperation() throws ThriftSecurityException, TException {
 +    while (true) {
 +      MasterClientService.Iface client = null;
 +      try {
 +        client = MasterClient.getConnectionWithRetry(instance);
 +        return client.beginFateOperation(Tracer.traceInfo(), credentials.toThrift(instance));
 +      } catch (TTransportException tte) {
 +        log.debug("Failed to call beginFateOperation(), retrying ... ", tte);
 +        UtilWaitThread.sleep(100);
 +      } finally {
 +        MasterClient.close(client);
 +      }
 +    }
 +  }
 +
 +  // This method is for retrying in the case of network failures; anything else it passes to the caller to deal with
 +  private void executeFateOperation(long opid, FateOperation op, List<ByteBuffer> args, Map<String,String> opts, boolean autoCleanUp)
 +      throws ThriftSecurityException, TException, ThriftTableOperationException {
 +    while (true) {
 +      MasterClientService.Iface client = null;
 +      try {
 +        client = MasterClient.getConnectionWithRetry(instance);
 +        client.executeFateOperation(Tracer.traceInfo(), credentials.toThrift(instance), opid, op, args, opts, autoCleanUp);
 +        break;
 +      } catch (TTransportException tte) {
 +        log.debug("Failed to call executeFateOperation(), retrying ... ", tte);
 +        UtilWaitThread.sleep(100);
 +      } finally {
 +        MasterClient.close(client);
 +      }
 +    }
 +  }
 +
 +  private String waitForFateOperation(long opid) throws ThriftSecurityException, TException, ThriftTableOperationException {
 +    while (true) {
 +      MasterClientService.Iface client = null;
 +      try {
 +        client = MasterClient.getConnectionWithRetry(instance);
 +        return client.waitForFateOperation(Tracer.traceInfo(), credentials.toThrift(instance), opid);
 +      } catch (TTransportException tte) {
 +        log.debug("Failed to call waitForFateOperation(), retrying ... ", tte);
 +        UtilWaitThread.sleep(100);
 +      } finally {
 +        MasterClient.close(client);
 +      }
 +    }
 +  }
 +
 +  private void finishFateOperation(long opid) throws ThriftSecurityException, TException {
 +    while (true) {
 +      MasterClientService.Iface client = null;
 +      try {
 +        client = MasterClient.getConnectionWithRetry(instance);
 +        client.finishFateOperation(Tracer.traceInfo(), credentials.toThrift(instance), opid);
 +        break;
 +      } catch (TTransportException tte) {
 +        log.debug("Failed to call finishFateOperation(), retrying ... ", tte);
 +        UtilWaitThread.sleep(100);
 +      } finally {
 +        MasterClient.close(client);
 +      }
 +    }
 +  }
 +
 +  String doFateOperation(FateOperation op, List<ByteBuffer> args, Map<String,String> opts) throws AccumuloSecurityException, TableExistsException,
 +      TableNotFoundException, AccumuloException, NamespaceExistsException, NamespaceNotFoundException {
 +    return doFateOperation(op, args, opts, true);
 +  }
 +
 +  String doFateOperation(FateOperation op, List<ByteBuffer> args, Map<String,String> opts, boolean wait) throws AccumuloSecurityException,
 +      TableExistsException, TableNotFoundException, AccumuloException, NamespaceExistsException, NamespaceNotFoundException {
 +    Long opid = null;
 +
 +    try {
 +      opid = beginFateOperation();
 +      executeFateOperation(opid, op, args, opts, !wait);
 +      if (!wait) {
 +        opid = null;
 +        return null;
 +      }
 +      String ret = waitForFateOperation(opid);
 +      return ret;
 +    } catch (ThriftSecurityException e) {
 +      String tableName = ByteBufferUtil.toString(args.get(0));
 +      switch (e.getCode()) {
 +        case TABLE_DOESNT_EXIST:
 +          throw new TableNotFoundException(null, tableName, "Target table does not exist");
 +        case NAMESPACE_DOESNT_EXIST:
 +          throw new NamespaceNotFoundException(null, tableName, "Target namespace does not exist");
 +        default:
 +          String tableInfo = Tables.getPrintableTableInfoFromName(instance, tableName);
 +          throw new AccumuloSecurityException(e.user, e.code, tableInfo, e);
 +      }
 +    } catch (ThriftTableOperationException e) {
 +      switch (e.getType()) {
 +        case EXISTS:
 +          throw new TableExistsException(e);
 +        case NOTFOUND:
 +          throw new TableNotFoundException(e);
 +        case NAMESPACE_EXISTS:
 +          throw new NamespaceExistsException(e);
 +        case NAMESPACE_NOTFOUND:
 +          throw new NamespaceNotFoundException(e);
 +        case OFFLINE:
 +          throw new TableOfflineException(instance, null);
 +        default:
 +          throw new AccumuloException(e.description, e);
 +      }
 +    } catch (Exception e) {
 +      throw new AccumuloException(e.getMessage(), e);
 +    } finally {
 +      Tables.clearCache(instance);
 +      // always finish table op, even when exception
 +      if (opid != null)
 +        try {
 +          finishFateOperation(opid);
 +        } catch (Exception e) {
 +          log.warn(e.getMessage(), e);
 +        }
 +    }
 +  }
 +
 +  private static class SplitEnv {
 +    private String tableName;
 +    private String tableId;
 +    private ExecutorService executor;
 +    private CountDownLatch latch;
 +    private AtomicReference<Exception> exception;
 +
 +    SplitEnv(String tableName, String tableId, ExecutorService executor, CountDownLatch latch, AtomicReference<Exception> exception) {
 +      this.tableName = tableName;
 +      this.tableId = tableId;
 +      this.executor = executor;
 +      this.latch = latch;
 +      this.exception = exception;
 +    }
 +  }
 +
 +  private class SplitTask implements Runnable {
 +
 +    private List<Text> splits;
 +    private SplitEnv env;
 +
 +    SplitTask(SplitEnv env, List<Text> splits) {
 +      this.env = env;
 +      this.splits = splits;
 +    }
 +
 +    @Override
 +    public void run() {
 +      try {
 +        if (env.exception.get() != null)
 +          return;
 +
 +        if (splits.size() <= 2) {
 +          addSplits(env.tableName, new TreeSet<Text>(splits), env.tableId);
 +          for (int i = 0; i < splits.size(); i++)
 +            env.latch.countDown();
 +          return;
 +        }
 +
 +        int mid = splits.size() / 2;
 +
 +        // split the middle split point to ensure that child task split different tablets and can therefore
 +        // run in parallel
 +        addSplits(env.tableName, new TreeSet<Text>(splits.subList(mid, mid + 1)), env.tableId);
 +        env.latch.countDown();
 +
 +        env.executor.submit(new SplitTask(env, splits.subList(0, mid)));
 +        env.executor.submit(new SplitTask(env, splits.subList(mid + 1, splits.size())));
 +
 +      } catch (Exception e) {
 +        env.exception.compareAndSet(null, e);
 +      }
 +    }
 +
 +  }
 +
 +  /**
 +   * @param tableName
 +   *          the name of the table
 +   * @param partitionKeys
 +   *          a sorted set of row key values to pre-split the table on
 +   * @throws AccumuloException
 +   *           if a general error occurs
 +   * @throws AccumuloSecurityException
 +   *           if the user does not have permission
 +   * @throws TableNotFoundException
 +   *           if the table does not exist
 +   */
 +  @Override
 +  public void addSplits(String tableName, SortedSet<Text> partitionKeys) throws TableNotFoundException, AccumuloException, AccumuloSecurityException {
 +    String tableId = Tables.getTableId(instance, tableName);
 +
 +    List<Text> splits = new ArrayList<Text>(partitionKeys);
 +    // should be sorted because we copied from a sorted set, but that makes assumptions about
 +    // how the copy was done so resort to be sure.
 +    Collections.sort(splits);
 +
 +    CountDownLatch latch = new CountDownLatch(splits.size());
 +    AtomicReference<Exception> exception = new AtomicReference<Exception>(null);
 +
 +    ExecutorService executor = Executors.newFixedThreadPool(16, new NamingThreadFactory("addSplits"));
 +    try {
 +      executor.submit(new SplitTask(new SplitEnv(tableName, tableId, executor, latch, exception), splits));
 +
 +      while (!latch.await(100, TimeUnit.MILLISECONDS)) {
 +        if (exception.get() != null) {
 +          executor.shutdownNow();
 +          Exception excep = exception.get();
 +          if (excep instanceof TableNotFoundException)
 +            throw (TableNotFoundException) excep;
 +          else if (excep instanceof AccumuloException)
 +            throw (AccumuloException) excep;
 +          else if (excep instanceof AccumuloSecurityException)
 +            throw (AccumuloSecurityException) excep;
 +          else if (excep instanceof RuntimeException)
 +            throw (RuntimeException) excep;
 +          else
 +            throw new RuntimeException(excep);
 +        }
 +      }
 +    } catch (InterruptedException e) {
 +      throw new RuntimeException(e);
 +    } finally {
 +      executor.shutdown();
 +    }
 +  }
 +
 +  private void addSplits(String tableName, SortedSet<Text> partitionKeys, String tableId) throws AccumuloException, AccumuloSecurityException,
 +      TableNotFoundException, AccumuloServerException {
 +    TabletLocator tabLocator = TabletLocator.getLocator(instance, new Text(tableId));
 +
 +    for (Text split : partitionKeys) {
 +      boolean successful = false;
 +      int attempt = 0;
 +
 +      while (!successful) {
 +
 +        if (attempt > 0)
 +          UtilWaitThread.sleep(100);
 +
 +        attempt++;
 +
 +        TabletLocation tl = tabLocator.locateTablet(credentials, split, false, false);
 +
 +        if (tl == null) {
 +          if (!Tables.exists(instance, tableId))
 +            throw new TableNotFoundException(tableId, tableName, null);
 +          else if (Tables.getTableState(instance, tableId) == TableState.OFFLINE)
 +            throw new TableOfflineException(instance, tableId);
 +          continue;
 +        }
 +
 +        try {
 +          TabletClientService.Client client = ThriftUtil.getTServerClient(tl.tablet_location, ServerConfigurationUtil.getConfiguration(instance));
 +          try {
 +            OpTimer opTimer = null;
 +            if (log.isTraceEnabled())
 +              opTimer = new OpTimer(log, Level.TRACE).start("Splitting tablet " + tl.tablet_extent + " on " + tl.tablet_location + " at " + split);
 +
 +            client.splitTablet(Tracer.traceInfo(), credentials.toThrift(instance), tl.tablet_extent.toThrift(), TextUtil.getByteBuffer(split));
 +
 +            // just split it, might as well invalidate it in the cache
 +            tabLocator.invalidateCache(tl.tablet_extent);
 +
 +            if (opTimer != null)
 +              opTimer.stop("Split tablet in %DURATION%");
 +          } finally {
 +            ThriftUtil.returnClient(client);
 +          }
 +
 +        } catch (TApplicationException tae) {
 +          throw new AccumuloServerException(tl.tablet_location, tae);
 +        } catch (TTransportException e) {
 +          tabLocator.invalidateCache(tl.tablet_location);
 +          continue;
 +        } catch (ThriftSecurityException e) {
 +          Tables.clearCache(instance);
 +          if (!Tables.exists(instance, tableId))
 +            throw new TableNotFoundException(tableId, tableName, null);
 +          throw new AccumuloSecurityException(e.user, e.code, e);
 +        } catch (NotServingTabletException e) {
 +          tabLocator.invalidateCache(tl.tablet_extent);
 +          continue;
 +        } catch (TException e) {
 +          tabLocator.invalidateCache(tl.tablet_location);
 +          continue;
 +        }
 +
 +        successful = true;
 +      }
 +    }
 +  }
 +
 +  @Override
 +  public void merge(String tableName, Text start, Text end) throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
 +
 +    checkArgument(tableName != null, "tableName is null");
 +    ByteBuffer EMPTY = ByteBuffer.allocate(0);
 +    List<ByteBuffer> args = Arrays.asList(ByteBuffer.wrap(tableName.getBytes(StandardCharsets.UTF_8)), start == null ? EMPTY : TextUtil.getByteBuffer(start),
 +        end == null ? EMPTY : TextUtil.getByteBuffer(end));
 +    Map<String,String> opts = new HashMap<String,String>();
 +    try {
 +      doTableFateOperation(tableName, TableNotFoundException.class, FateOperation.TABLE_MERGE, args, opts);
 +    } catch (TableExistsException e) {
 +      // should not happen
 +      throw new AssertionError(e);
 +    }
 +  }
 +
 +  @Override
 +  public void deleteRows(String tableName, Text start, Text end) throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
 +
 +    checkArgument(tableName != null, "tableName is null");
 +    ByteBuffer EMPTY = ByteBuffer.allocate(0);
 +    List<ByteBuffer> args = Arrays.asList(ByteBuffer.wrap(tableName.getBytes(StandardCharsets.UTF_8)), start == null ? EMPTY : TextUtil.getByteBuffer(start),
 +        end == null ? EMPTY : TextUtil.getByteBuffer(end));
 +    Map<String,String> opts = new HashMap<String,String>();
 +    try {
 +      doTableFateOperation(tableName, TableNotFoundException.class, FateOperation.TABLE_DELETE_RANGE, args, opts);
 +    } catch (TableExistsException e) {
 +      // should not happen
 +      throw new AssertionError(e);
 +    }
 +  }
 +
 +  /**
 +   * @param tableName
 +   *          the name of the table
 +   * @return the split points (end-row names) for the table's current split profile
 +   */
 +  @Override
 +  public Collection<Text> listSplits(String tableName) throws TableNotFoundException, AccumuloSecurityException {
 +
 +    checkArgument(tableName != null, "tableName is null");
 +
 +    String tableId = Tables.getTableId(instance, tableName);
 +
 +    TreeMap<KeyExtent,String> tabletLocations = new TreeMap<KeyExtent,String>();
 +
 +    while (true) {
 +      try {
 +        tabletLocations.clear();
 +        // the following method throws AccumuloException for some conditions that should be retried
 +        MetadataServicer.forTableId(instance, credentials, tableId).getTabletLocations(tabletLocations);
 +        break;
 +      } catch (AccumuloSecurityException ase) {
 +        throw ase;
 +      } catch (Exception e) {
 +        if (!Tables.exists(instance, tableId)) {
 +          throw new TableNotFoundException(tableId, tableName, null);
 +        }
 +
 +        if (e instanceof RuntimeException && e.getCause() instanceof AccumuloSecurityException) {
 +          throw (AccumuloSecurityException) e.getCause();
 +        }
 +
 +        log.info(e.getMessage() + " ... retrying ...");
 +        UtilWaitThread.sleep(3000);
 +      }
 +    }
 +
 +    ArrayList<Text> endRows = new ArrayList<Text>(tabletLocations.size());
 +
 +    for (KeyExtent ke : tabletLocations.keySet())
 +      if (ke.getEndRow() != null)
 +        endRows.add(ke.getEndRow());
 +
 +    return endRows;
 +  }
 +
 +  @Deprecated
 +  @Override
 +  public Collection<Text> getSplits(String tableName) throws TableNotFoundException {
 +    try {
 +      return listSplits(tableName);
 +    } catch (AccumuloSecurityException e) {
 +      throw new RuntimeException(e);
 +    }
 +  }
 +
 +  /**
 +   * @param tableName
 +   *          the name of the table
 +   * @param maxSplits
 +   *          specifies the maximum number of splits to return
 +   * @return the split points (end-row names) for the table's current split profile, grouped into fewer splits so as not to exceed maxSplits
 +   */
 +  @Override
 +  public Collection<Text> listSplits(String tableName, int maxSplits) throws TableNotFoundException, AccumuloSecurityException {
 +    Collection<Text> endRows = listSplits(tableName);
 +
 +    if (endRows.size() <= maxSplits)
 +      return endRows;
 +
 +    double r = (maxSplits + 1) / (double) (endRows.size());
 +    double pos = 0;
 +
 +    ArrayList<Text> subset = new ArrayList<Text>(maxSplits);
 +
 +    int j = 0;
 +    for (int i = 0; i < endRows.size() && j < maxSplits; i++) {
 +      pos += r;
 +      while (pos > 1) {
 +        subset.add(((ArrayList<Text>) endRows).get(i));
 +        j++;
 +        pos -= 1;
 +      }
 +    }
 +
 +    return subset;
 +  }
 +
 +  @Deprecated
 +  @Override
 +  public Collection<Text> getSplits(String tableName, int maxSplits) throws TableNotFoundException {
 +    try {
 +      return listSplits(tableName, maxSplits);
 +    } catch (AccumuloSecurityException e) {
 +      throw new RuntimeException(e);
 +    }
 +  }
 +
 +  /**
 +   * Delete a table
 +   * 
 +   * @param tableName
 +   *          the name of the table
 +   * @throws AccumuloException
 +   *           if a general error occurs
 +   * @throws AccumuloSecurityException
 +   *           if the user does not have permission
 +   * @throws TableNotFoundException
 +   *           if the table does not exist
 +   */
 +  @Override
 +  public void delete(String tableName) throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
 +    checkArgument(tableName != null, "tableName is null");
 +
 +    List<ByteBuffer> args = Arrays.asList(ByteBuffer.wrap(tableName.getBytes(StandardCharsets.UTF_8)));
 +    Map<String,String> opts = new HashMap<String,String>();
 +
 +    try {
 +      doTableFateOperation(tableName, TableNotFoundException.class, FateOperation.TABLE_DELETE, args, opts);
 +    } catch (TableExistsException e) {
 +      // should not happen
 +      throw new AssertionError(e);
 +    }
 +
 +  }
 +
 +  @Override
 +  public void clone(String srcTableName, String newTableName, boolean flush, Map<String,String> propertiesToSet, Set<String> propertiesToExclude)
 +      throws AccumuloSecurityException, TableNotFoundException, AccumuloException, TableExistsException {
 +
 +    checkArgument(srcTableName != null, "srcTableName is null");
 +    checkArgument(newTableName != null, "newTableName is null");
 +
 +    String srcTableId = Tables.getTableId(instance, srcTableName);
 +
 +    if (flush)
 +      _flush(srcTableId, null, null, true);
 +
 +    if (propertiesToExclude == null)
 +      propertiesToExclude = Collections.emptySet();
 +
 +    if (propertiesToSet == null)
 +      propertiesToSet = Collections.emptyMap();
 +
 +    List<ByteBuffer> args = Arrays.asList(ByteBuffer.wrap(srcTableId.getBytes(StandardCharsets.UTF_8)), ByteBuffer.wrap(newTableName.getBytes(StandardCharsets.UTF_8)));
 +    Map<String,String> opts = new HashMap<String,String>();
 +    for (Entry<String,String> entry : propertiesToSet.entrySet()) {
 +      if (entry.getKey().startsWith(CLONE_EXCLUDE_PREFIX))
 +        throw new IllegalArgumentException("Property can not start with " + CLONE_EXCLUDE_PREFIX);
 +      opts.put(entry.getKey(), entry.getValue());
 +    }
 +
 +    for (String prop : propertiesToExclude) {
 +      opts.put(CLONE_EXCLUDE_PREFIX + prop, "");
 +    }
 +
 +    doTableFateOperation(newTableName, AccumuloException.class, FateOperation.TABLE_CLONE, args, opts);
 +  }
 +
 +  /**
 +   * Rename a table
 +   * 
 +   * @param oldTableName
 +   *          the old table name
 +   * @param newTableName
 +   *          the new table name
 +   * @throws AccumuloException
 +   *           if a general error occurs
 +   * @throws AccumuloSecurityException
 +   *           if the user does not have permission
 +   * @throws TableNotFoundException
 +   *           if the old table name does not exist
 +   * @throws TableExistsException
 +   *           if the new table name already exists
 +   */
 +  @Override
 +  public void rename(String oldTableName, String newTableName) throws AccumuloSecurityException, TableNotFoundException, AccumuloException,
 +      TableExistsException {
 +
 +    List<ByteBuffer> args = Arrays.asList(ByteBuffer.wrap(oldTableName.getBytes(StandardCharsets.UTF_8)), ByteBuffer.wrap(newTableName.getBytes(StandardCharsets.UTF_8)));
 +    Map<String,String> opts = new HashMap<String,String>();
 +    doTableFateOperation(oldTableName, TableNotFoundException.class, FateOperation.TABLE_RENAME, args, opts);
 +  }
 +
 +  /**
 +   * @deprecated since 1.4 {@link #flush(String, Text, Text, boolean)}
 +   */
 +  @Override
 +  @Deprecated
 +  public void flush(String tableName) throws AccumuloException, AccumuloSecurityException {
 +    try {
 +      flush(tableName, null, null, false);
 +    } catch (TableNotFoundException e) {
 +      throw new AccumuloException(e.getMessage(), e);
 +    }
 +  }
 +
 +  /**
 +   * Flush a table
 +   * 
 +   * @param tableName
 +   *          the name of the table
 +   * @throws AccumuloException
 +   *           if a general error occurs
 +   * @throws AccumuloSecurityException
 +   *           if the user does not have permission
 +   */
 +  @Override
 +  public void flush(String tableName, Text start, Text end, boolean wait) throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
 +    checkArgument(tableName != null, "tableName is null");
 +
 +    String tableId = Tables.getTableId(instance, tableName);
 +    _flush(tableId, start, end, wait);
 +  }
 +
 +  @Override
 +  public void compact(String tableName, Text start, Text end, boolean flush, boolean wait) throws AccumuloSecurityException, TableNotFoundException,
 +      AccumuloException {
 +    compact(tableName, start, end, new ArrayList<IteratorSetting>(), flush, wait);
 +  }
 +
 +  @Override
 +  public void compact(String tableName, Text start, Text end, List<IteratorSetting> iterators, boolean flush, boolean wait) throws AccumuloSecurityException,
 +      TableNotFoundException, AccumuloException {
 +    checkArgument(tableName != null, "tableName is null");
 +    ByteBuffer EMPTY = ByteBuffer.allocate(0);
 +
 +    String tableId = Tables.getTableId(instance, tableName);
 +
 +    if (flush)
 +      _flush(tableId, start, end, true);
 +
 +    List<ByteBuffer> args = Arrays.asList(ByteBuffer.wrap(tableId.getBytes(StandardCharsets.UTF_8)), start == null ? EMPTY : TextUtil.getByteBuffer(start),
 +        end == null ? EMPTY : TextUtil.getByteBuffer(end), ByteBuffer.wrap(IteratorUtil.encodeIteratorSettings(iterators)));
 +
 +    Map<String,String> opts = new HashMap<String,String>();
 +    try {
 +      doFateOperation(FateOperation.TABLE_COMPACT, args, opts, wait);
 +    } catch (TableExistsException e) {
 +      // should not happen
 +      throw new AssertionError(e);
 +    } catch (NamespaceExistsException e) {
 +      // should not happen
 +      throw new AssertionError(e);
 +    } catch (NamespaceNotFoundException e) {
 +      throw new TableNotFoundException(null, tableName, "Namespace not found", e);
 +    }
 +  }
 +
 +  @Override
 +  public void cancelCompaction(String tableName) throws AccumuloSecurityException, TableNotFoundException, AccumuloException {
 +    String tableId = Tables.getTableId(instance, tableName);
 +
 +    List<ByteBuffer> args = Arrays.asList(ByteBuffer.wrap(tableId.getBytes(StandardCharsets.UTF_8)));
 +
 +    Map<String,String> opts = new HashMap<String,String>();
 +    try {
 +      doTableFateOperation(tableName, TableNotFoundException.class, FateOperation.TABLE_CANCEL_COMPACT, args, opts);
 +    } catch (TableExistsException e) {
 +      // should not happen
 +      throw new AssertionError(e);
 +    }
 +
 +  }
 +
 +  private void _flush(String tableId, Text start, Text end, boolean wait) throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
 +
 +    try {
 +      long flushID;
 +
 +      // used to pass the table name. but the tableid associated with a table name could change between calls.
 +      // so pass the tableid to both calls
 +
 +      while (true) {
 +        MasterClientService.Iface client = null;
 +        try {
 +          client = MasterClient.getConnectionWithRetry(instance);
 +          flushID = client.initiateFlush(Tracer.traceInfo(), credentials.toThrift(instance), tableId);
 +          break;
 +        } catch (TTransportException tte) {
 +          log.debug("Failed to call initiateFlush, retrying ... ", tte);
 +          UtilWaitThread.sleep(100);
 +        } finally {
 +          MasterClient.close(client);
 +        }
 +      }
 +
 +      while (true) {
 +        MasterClientService.Iface client = null;
 +        try {
 +          client = MasterClient.getConnectionWithRetry(instance);
 +          client.waitForFlush(Tracer.traceInfo(), credentials.toThrift(instance), tableId, TextUtil.getByteBuffer(start), TextUtil.getByteBuffer(end), flushID,
 +              wait ? Long.MAX_VALUE : 1);
 +          break;
 +        } catch (TTransportException tte) {
 +          log.debug("Failed to call initiateFlush, retrying ... ", tte);
 +          UtilWaitThread.sleep(100);
 +        } finally {
 +          MasterClient.close(client);
 +        }
 +      }
 +    } catch (ThriftSecurityException e) {
 +      switch (e.getCode()) {
 +        case TABLE_DOESNT_EXIST:
 +          throw new TableNotFoundException(tableId, null, e.getMessage(), e);
 +        default:
 +          log.debug("flush security exception on table id " + tableId);
 +          throw new AccumuloSecurityException(e.user, e.code, e);
 +      }
 +    } catch (ThriftTableOperationException e) {
 +      switch (e.getType()) {
 +        case NOTFOUND:
 +          throw new TableNotFoundException(e);
 +        default:
 +          throw new AccumuloException(e.description, e);
 +      }
 +    } catch (Exception e) {
 +      throw new AccumuloException(e);
 +    }
 +  }
 +
 +  /**
 +   * Sets a property on a table
 +   * 
 +   * @param tableName
 +   *          the name of the table
 +   * @param property
 +   *          the name of a per-table property
 +   * @param value
 +   *          the value to set a per-table property to
 +   * @throws AccumuloException
 +   *           if a general error occurs
 +   * @throws AccumuloSecurityException
 +   *           if the user does not have permission
 +   */
 +  @Override
 +  public void setProperty(final String tableName, final String property, final String value) throws AccumuloException, AccumuloSecurityException {
 +    checkArgument(tableName != null, "tableName is null");
 +    checkArgument(property != null, "property is null");
 +    checkArgument(value != null, "value is null");
 +    try {
 +      MasterClient.executeTable(instance, new ClientExec<MasterClientService.Client>() {
 +        @Override
 +        public void execute(MasterClientService.Client client) throws Exception {
 +          client.setTableProperty(Tracer.traceInfo(), credentials.toThrift(instance), tableName, property, value);
 +        }
 +      });
 +    } catch (TableNotFoundException e) {
 +      throw new AccumuloException(e);
 +    }
 +  }
 +
 +  /**
 +   * Removes a property from a table
 +   * 
 +   * @param tableName
 +   *          the name of the table
 +   * @param property
 +   *          the name of a per-table property
 +   * @throws AccumuloException
 +   *           if a general error occurs
 +   * @throws AccumuloSecurityException
 +   *           if the user does not have permission
 +   */
 +  @Override
 +  public void removeProperty(final String tableName, final String property) throws AccumuloException, AccumuloSecurityException {
 +    checkArgument(tableName != null, "tableName is null");
 +    checkArgument(property != null, "property is null");
 +    try {
 +      MasterClient.executeTable(instance, new ClientExec<MasterClientService.Client>() {
 +        @Override
 +        public void execute(MasterClientService.Client client) throws Exception {
 +          client.removeTableProperty(Tracer.traceInfo(), credentials.toThrift(instance), tableName, property);
 +        }
 +      });
 +    } catch (TableNotFoundException e) {
 +      throw new AccumuloException(e);
 +    }
 +  }
 +
 +  /**
 +   * Gets properties of a table
 +   * 
 +   * @param tableName
 +   *          the name of the table
 +   * @return all properties visible by this table (system and per-table properties)
 +   * @throws TableNotFoundException
 +   *           if the table does not exist
 +   */
 +  @Override
 +  public Iterable<Entry<String,String>> getProperties(final String tableName) throws AccumuloException, TableNotFoundException {
 +    checkArgument(tableName != null, "tableName is null");
 +    try {
 +      return ServerClient.executeRaw(instance, new ClientExecReturn<Map<String,String>,ClientService.Client>() {
 +        @Override
 +        public Map<String,String> execute(ClientService.Client client) throws Exception {
 +          return client.getTableConfiguration(Tracer.traceInfo(), credentials.toThrift(instance), tableName);
 +        }
 +      }).entrySet();
 +    } catch (ThriftTableOperationException e) {
 +      switch (e.getType()) {
 +        case NOTFOUND:
 +          throw new TableNotFoundException(e);
 +        case NAMESPACE_NOTFOUND:
 +          throw new TableNotFoundException(tableName, new NamespaceNotFoundException(e));
 +        default:
 +          throw new AccumuloException(e.description, e);
 +      }
 +    } catch (AccumuloException e) {
 +      throw e;
 +    } catch (Exception e) {
 +      throw new AccumuloException(e);
 +    }
 +
 +  }
 +
 +  /**
 +   * Sets a tables locality groups. A tables locality groups can be changed at any time.
 +   * 
 +   * @param tableName
 +   *          the name of the table
 +   * @param groups
 +   *          mapping of locality group names to column families in the locality group
 +   * @throws AccumuloException
 +   *           if a general error occurs
 +   * @throws AccumuloSecurityException
 +   *           if the user does not have permission
 +   * @throws TableNotFoundException
 +   *           if the table does not exist
 +   */
 +  @Override
 +  public void setLocalityGroups(String tableName, Map<String,Set<Text>> groups) throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
 +    // ensure locality groups do not overlap
 +    HashSet<Text> all = new HashSet<Text>();
 +    for (Entry<String,Set<Text>> entry : groups.entrySet()) {
 +
 +      if (!Collections.disjoint(all, entry.getValue())) {
 +        throw new IllegalArgumentException("Group " + entry.getKey() + " overlaps with another group");
 +      }
 +
 +      all.addAll(entry.getValue());
 +    }
 +
 +    for (Entry<String,Set<Text>> entry : groups.entrySet()) {
 +      Set<Text> colFams = entry.getValue();
 +      String value = LocalityGroupUtil.encodeColumnFamilies(colFams);
 +      setProperty(tableName, Property.TABLE_LOCALITY_GROUP_PREFIX + entry.getKey(), value);
 +    }
 +
 +    try {
 +      setProperty(tableName, Property.TABLE_LOCALITY_GROUPS.getKey(), Joiner.on(",").join(groups.keySet()));
 +    } catch (AccumuloException e) {
 +      if (e.getCause() instanceof TableNotFoundException)
 +        throw (TableNotFoundException) e.getCause();
 +      throw e;
 +    }
 +
 +    // remove anything extraneous
 +    String prefix = Property.TABLE_LOCALITY_GROUP_PREFIX.getKey();
 +    for (Entry<String,String> entry : getProperties(tableName)) {
 +      String property = entry.getKey();
 +      if (property.startsWith(prefix)) {
 +        // this property configures a locality group, find out which
 +        // one:
 +        String[] parts = property.split("\\.");
 +        String group = parts[parts.length - 1];
 +
 +        if (!groups.containsKey(group)) {
 +          removeProperty(tableName, property);
 +        }
 +      }
 +    }
 +  }
 +
 +  /**
 +   * 
 +   * Gets the locality groups currently set for a table.
 +   * 
 +   * @param tableName
 +   *          the name of the table
 +   * @return mapping of locality group names to column families in the locality group
 +   * @throws AccumuloException
 +   *           if a general error occurs
 +   * @throws TableNotFoundException
 +   *           if the table does not exist
 +   */
 +  @Override
 +  public Map<String,Set<Text>> getLocalityGroups(String tableName) throws AccumuloException, TableNotFoundException {
 +    AccumuloConfiguration conf = new ConfigurationCopy(this.getProperties(tableName));
 +    Map<String,Set<ByteSequence>> groups = LocalityGroupUtil.getLocalityGroups(conf);
 +
 +    Map<String,Set<Text>> groups2 = new HashMap<String,Set<Text>>();
 +    for (Entry<String,Set<ByteSequence>> entry : groups.entrySet()) {
 +
 +      HashSet<Text> colFams = new HashSet<Text>();
 +
 +      for (ByteSequence bs : entry.getValue()) {
 +        colFams.add(new Text(bs.toArray()));
 +      }
 +
 +      groups2.put(entry.getKey(), colFams);
 +    }
 +
 +    return groups2;
 +  }
 +
 +  /**
 +   * @param tableName
 +   *          the name of the table
 +   * @param range
 +   *          a range to split
 +   * @param maxSplits
 +   *          the maximum number of splits
 +   * @return the range, split into smaller ranges that fall on boundaries of the table's split points as evenly as possible
 +   * @throws AccumuloException
 +   *           if a general error occurs
 +   * @throws AccumuloSecurityException
 +   *           if the user does not have permission
 +   * @throws TableNotFoundException
 +   *           if the table does not exist
 +   */
 +  @Override
 +  public Set<Range> splitRangeByTablets(String tableName, Range range, int maxSplits) throws AccumuloException, AccumuloSecurityException,
 +      TableNotFoundException {
 +    checkArgument(tableName != null, "tableName is null");
 +    checkArgument(range != null, "range is null");
 +    if (maxSplits < 1)
 +      throw new IllegalArgumentException("maximum splits must be >= 1");
 +    if (maxSplits == 1)
 +      return Collections.singleton(range);
 +
 +    Random random = new Random();
 +    Map<String,Map<KeyExtent,List<Range>>> binnedRanges = new HashMap<String,Map<KeyExtent,List<Range>>>();
 +    String tableId = Tables.getTableId(instance, tableName);
 +    TabletLocator tl = TabletLocator.getLocator(instance, new Text(tableId));
 +    // its possible that the cache could contain complete, but old information about a tables tablets... so clear it
 +    tl.invalidateCache();
 +    while (!tl.binRanges(credentials, Collections.singletonList(range), binnedRanges).isEmpty()) {
 +      if (!Tables.exists(instance, tableId))
 +        throw new TableDeletedException(tableId);
 +      if (Tables.getTableState(instance, tableId) == TableState.OFFLINE)
 +        throw new TableOfflineException(instance, tableId);
 +
 +      log.warn("Unable to locate bins for specified range. Retrying.");
 +      // sleep randomly between 100 and 200ms
 +      UtilWaitThread.sleep(100 + random.nextInt(100));
 +      binnedRanges.clear();
 +      tl.invalidateCache();
 +    }
 +
 +    // group key extents to get <= maxSplits
 +    LinkedList<KeyExtent> unmergedExtents = new LinkedList<KeyExtent>();
 +    List<KeyExtent> mergedExtents = new ArrayList<KeyExtent>();
 +
 +    for (Map<KeyExtent,List<Range>> map : binnedRanges.values())
 +      unmergedExtents.addAll(map.keySet());
 +
 +    // the sort method is efficient for linked list
 +    Collections.sort(unmergedExtents);
 +
 +    while (unmergedExtents.size() + mergedExtents.size() > maxSplits) {
 +      if (unmergedExtents.size() >= 2) {
 +        KeyExtent first = unmergedExtents.removeFirst();
 +        KeyExtent second = unmergedExtents.removeFirst();
 +        first.setEndRow(second.getEndRow());
 +        mergedExtents.add(first);
 +      } else {
 +        mergedExtents.addAll(unmergedExtents);
 +        unmergedExtents.clear();
 +        unmergedExtents.addAll(mergedExtents);
 +        mergedExtents.clear();
 +      }
 +
 +    }
 +
 +    mergedExtents.addAll(unmergedExtents);
 +
 +    Set<Range> ranges = new HashSet<Range>();
 +    for (KeyExtent k : mergedExtents)
 +      ranges.add(k.toDataRange().clip(range));
 +
 +    return ranges;
 +  }
 +
 +  // TODO Remove deprecation warning surppression when Hadoop1 support is dropped
 +  @SuppressWarnings("deprecation")
 +  private Path checkPath(String dir, String kind, String type) throws IOException, AccumuloException {
 +    Path ret;
 +    FileSystem fs = VolumeConfiguration.getVolume(dir, CachedConfiguration.getInstance(), ServerConfigurationUtil.getConfiguration(instance)).getFileSystem();
 +
 +    if (dir.contains(":")) {
 +      ret = new Path(dir);
 +    } else {
 +      ret = fs.makeQualified(new Path(dir));
 +    }
 +
 +    if (!fs.exists(ret))
 +      throw new AccumuloException(kind + " import " + type + " directory " + dir + " does not exist!");
 +
 +    if (!fs.getFileStatus(ret).isDir()) {
 +      throw new AccumuloException(kind + " import " + type + " directory " + dir + " is not a directory!");
 +    }
 +
 +    if (type.equals("failure")) {
 +      FileStatus[] listStatus = fs.listStatus(ret);
 +      if (listStatus != null && listStatus.length != 0) {
 +        throw new AccumuloException("Bulk import failure directory " + ret + " is not empty");
 +      }
 +    }
 +
 +    return ret;
 +  }
 +
 +  @Override
 +  public void importDirectory(String tableName, String dir, String failureDir, boolean setTime) throws IOException, AccumuloSecurityException,
 +      TableNotFoundException, AccumuloException {
 +    checkArgument(tableName != null, "tableName is null");
 +    checkArgument(dir != null, "dir is null");
 +    checkArgument(failureDir != null, "failureDir is null");
 +    // check for table existance
 +    Tables.getTableId(instance, tableName);
 +
 +    Path dirPath = checkPath(dir, "Bulk", "");
 +    Path failPath = checkPath(failureDir, "Bulk", "failure");
 +
 +    List<ByteBuffer> args = Arrays.asList(ByteBuffer.wrap(tableName.getBytes(StandardCharsets.UTF_8)), ByteBuffer.wrap(dirPath.toString().getBytes(StandardCharsets.UTF_8)),
 +        ByteBuffer.wrap(failPath.toString().getBytes(StandardCharsets.UTF_8)), ByteBuffer.wrap((setTime + "").getBytes(StandardCharsets.UTF_8)));
 +    Map<String,String> opts = new HashMap<String,String>();
 +
 +    try {
 +      doTableFateOperation(tableName, TableNotFoundException.class, FateOperation.TABLE_BULK_IMPORT, args, opts);
 +    } catch (TableExistsException e) {
 +      // should not happen
 +      throw new AssertionError(e);
 +    }
 +  }
 +
 +  private void waitForTableStateTransition(String tableId, TableState expectedState) throws AccumuloException, TableNotFoundException,
 +      AccumuloSecurityException {
 +
 +    Text startRow = null;
 +    Text lastRow = null;
 +
 +    while (true) {
 +
 +      if (Tables.getTableState(instance, tableId) != expectedState) {
 +        Tables.clearCache(instance);
 +        if (Tables.getTableState(instance, tableId) != expectedState) {
 +          if (!Tables.exists(instance, tableId))
 +            throw new TableDeletedException(tableId);
 +          throw new AccumuloException("Unexpected table state " + tableId + " " + Tables.getTableState(instance, tableId) + " != " + expectedState);
 +        }
 +      }
 +
 +      Range range = new KeyExtent(new Text(tableId), null, null).toMetadataRange();
 +      if (startRow == null || lastRow == null)
 +        range = new KeyExtent(new Text(tableId), null, null).toMetadataRange();
 +      else
 +        range = new Range(startRow, lastRow);
 +
 +      String metaTable = MetadataTable.NAME;
 +      if (tableId.equals(MetadataTable.ID))
 +        metaTable = RootTable.NAME;
 +      Scanner scanner = instance.getConnector(credentials.getPrincipal(), credentials.getToken()).createScanner(metaTable, Authorizations.EMPTY);
 +      scanner = new IsolatedScanner(scanner);
 +      TabletsSection.TabletColumnFamily.PREV_ROW_COLUMN.fetch(scanner);
 +      scanner.fetchColumnFamily(TabletsSection.CurrentLocationColumnFamily.NAME);
 +      scanner.setRange(range);
 +
 +      RowIterator rowIter = new RowIterator(scanner);
 +
 +      KeyExtent lastExtent = null;
 +
 +      int total = 0;
 +      int waitFor = 0;
 +      int holes = 0;
 +      Text continueRow = null;
 +      MapCounter<String> serverCounts = new MapCounter<String>();
 +
 +      while (rowIter.hasNext()) {
 +        Iterator<Entry<Key,Value>> row = rowIter.next();
 +
 +        total++;
 +
 +        KeyExtent extent = null;
 +        String future = null;
 +        String current = null;
 +
 +        while (row.hasNext()) {
 +          Entry<Key,Value> entry = row.next();
 +          Key key = entry.getKey();
 +
 +          if (key.getColumnFamily().equals(TabletsSection.FutureLocationColumnFamily.NAME))
 +            future = entry.getValue().toString();
 +
 +          if (key.getColumnFamily().equals(TabletsSection.CurrentLocationColumnFamily.NAME))
 +            current = entry.getValue().toString();
 +
 +          if (TabletsSection.TabletColumnFamily.PREV_ROW_COLUMN.hasColumns(key))
 +            extent = new KeyExtent(key.getRow(), entry.getValue());
 +        }
 +
 +        if ((expectedState == TableState.ONLINE && current == null) || (expectedState == TableState.OFFLINE && (future != null || current != null))) {
 +          if (continueRow == null)
 +            continueRow = extent.getMetadataEntry();
 +          waitFor++;
 +          lastRow = extent.getMetadataEntry();
 +
 +          if (current != null)
 +            serverCounts.increment(current, 1);
 +          if (future != null)
 +            serverCounts.increment(future, 1);
 +        }
 +
 +        if (!extent.getTableId().toString().equals(tableId)) {
 +          throw new AccumuloException("Saw unexpected table Id " + tableId + " " + extent);
 +        }
 +
 +        if (lastExtent != null && !extent.isPreviousExtent(lastExtent)) {
 +          holes++;
 +        }
 +
 +        lastExtent = extent;
 +      }
 +
 +      if (continueRow != null) {
 +        startRow = continueRow;
 +      }
 +
 +      if (holes > 0 || total == 0) {
 +        startRow = null;
 +        lastRow = null;
 +      }
 +
 +      if (waitFor > 0 || holes > 0 || total == 0) {
 +        long waitTime;
 +        long maxPerServer = 0;
 +        if (serverCounts.size() > 0) {
 +          maxPerServer = Collections.max(serverCounts.values());
 +          waitTime = maxPerServer * 10;
 +        } else
 +          waitTime = waitFor * 10;
 +        waitTime = Math.max(100, waitTime);
 +        waitTime = Math.min(5000, waitTime);
 +        log.trace("Waiting for " + waitFor + "(" + maxPerServer + ") tablets, startRow = " + startRow + " lastRow = " + lastRow + ", holes=" + holes
 +            + " sleeping:" + waitTime + "ms");
 +        UtilWaitThread.sleep(waitTime);
 +      } else {
 +        break;
 +      }
 +
 +    }
 +  }
 +
 +  @Override
 +  public void offline(String tableName) throws AccumuloSecurityException, AccumuloException, TableNotFoundException {
 +    offline(tableName, false);
 +  }
 +
 +  /**
 +   * 
 +   * @param tableName
 +   *          the table to take offline
 +   * @throws AccumuloException
 +   *           when there is a general accumulo error
 +   * @throws AccumuloSecurityException
 +   *           when the user does not have the proper permissions
 +   */
 +  @Override
 +  public void offline(String tableName, boolean wait) throws AccumuloSecurityException, AccumuloException, TableNotFoundException {
 +
 +    checkArgument(tableName != null, "tableName is null");
 +    String tableId = Tables.getTableId(instance, tableName);
 +    List<ByteBuffer> args = Arrays.asList(ByteBuffer.wrap(tableId.getBytes(StandardCharsets.UTF_8)));
 +    Map<String,String> opts = new HashMap<String,String>();
 +
 +    try {
 +      doTableFateOperation(tableName, TableNotFoundException.class, FateOperation.TABLE_OFFLINE, args, opts);
 +    } catch (TableExistsException e) {
 +      // should not happen
 +      throw new AssertionError(e);
 +    }
 +
 +    if (wait)
 +      waitForTableStateTransition(tableId, TableState.OFFLINE);
 +  }
 +
 +  @Override
 +  public void online(String tableName) throws AccumuloSecurityException, AccumuloException, TableNotFoundException {
 +    online(tableName, false);
 +  }
 +
 +  /**
 +   * 
 +   * @param tableName
 +   *          the table to take online
 +   * @throws AccumuloException
 +   *           when there is a general accumulo error
 +   * @throws AccumuloSecurityException
 +   *           when the user does not have the proper permissions
 +   */
 +  @Override
 +  public void online(String tableName, boolean wait) throws AccumuloSecurityException, AccumuloException, TableNotFoundException {
 +    checkArgument(tableName != null, "tableName is null");
 +    String tableId = Tables.getTableId(instance, tableName);
 +    List<ByteBuffer> args = Arrays.asList(ByteBuffer.wrap(tableId.getBytes(StandardCharsets.UTF_8)));
 +    Map<String,String> opts = new HashMap<String,String>();
 +
 +    try {
 +      doTableFateOperation(tableName, TableNotFoundException.class, FateOperation.TABLE_ONLINE, args, opts);
 +    } catch (TableExistsException e) {
 +      // should not happen
 +      throw new AssertionError(e);
 +    }
 +
 +    if (wait)
 +      waitForTableStateTransition(tableId, TableState.ONLINE);
 +  }
 +
 +  /**
 +   * Clears the tablet locator cache for a specified table
 +   * 
 +   * @param tableName
 +   *          the name of the table
 +   * @throws TableNotFoundException
 +   *           if table does not exist
 +   */
 +  @Override
 +  public void clearLocatorCache(String tableName) throws TableNotFoundException {
 +    checkArgument(tableName != null, "tableName is null");
 +    TabletLocator tabLocator = TabletLocator.getLocator(instance, new Text(Tables.getTableId(instance, tableName)));
 +    tabLocator.invalidateCache();
 +  }
 +
 +  /**
 +   * Get a mapping of table name to internal table id.
 +   * 
 +   * @return the map from table name to internal table id
 +   */
 +  @Override
 +  public Map<String,String> tableIdMap() {
 +    return Tables.getNameToIdMap(instance);
 +  }
 +
 +  @Override
 +  public Text getMaxRow(String tableName, Authorizations auths, Text startRow, boolean startInclusive, Text endRow, boolean endInclusive)
 +      throws TableNotFoundException, AccumuloException, AccumuloSecurityException {
 +    checkArgument(tableName != null, "tableName is null");
 +    checkArgument(auths != null, "auths is null");
 +    Scanner scanner = instance.getConnector(credentials.getPrincipal(), credentials.getToken()).createScanner(tableName, auths);
 +    return FindMax.findMax(scanner, startRow, startInclusive, endRow, endInclusive);
 +  }
 +
 +  @Override
 +  public List<DiskUsage> getDiskUsage(Set<String> tableNames) throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
 +
 +    List<TDiskUsage> diskUsages = null;
 +    while (diskUsages == null) {
 +      Pair<String,Client> pair = null;
 +      try {
 +        // this operation may us a lot of memory... its likely that connections to tabletservers hosting metadata tablets will be cached, so do not use cached
 +        // connections
 +        pair = ServerClient.getConnection(instance, false);
 +        diskUsages = pair.getSecond().getDiskUsage(tableNames, credentials.toThrift(instance));
 +      } catch (ThriftTableOperationException e) {
 +        switch (e.getType()) {
 +          case NOTFOUND:
 +            throw new TableNotFoundException(e);
 +          case NAMESPACE_NOTFOUND:
 +            throw new TableNotFoundException(e.getTableName(), new NamespaceNotFoundException(e));
 +          default:
 +            throw new AccumuloException(e.description, e);
 +        }
 +      } catch (ThriftSecurityException e) {
 +        throw new AccumuloSecurityException(e.getUser(), e.getCode());
 +      } catch (TTransportException e) {
 +        // some sort of communication error occurred, retry
 +        log.debug("disk usage request failed " + pair.getFirst() + ", retrying ... ", e);
 +        UtilWaitThread.sleep(100);
 +      } catch (TException e) {
 +        // may be a TApplicationException which indicates error on the server side
 +        throw new AccumuloException(e);
 +      } finally {
 +        // must always return thrift connection
 +        if (pair != null)
 +          ServerClient.close(pair.getSecond());
 +      }
 +    }
 +
 +    List<DiskUsage> finalUsages = new ArrayList<DiskUsage>();
 +    for (TDiskUsage diskUsage : diskUsages) {
 +      finalUsages.add(new DiskUsage(new TreeSet<String>(diskUsage.getTables()), diskUsage.getUsage()));
 +    }
 +
 +    return finalUsages;
 +  }
 +
 +  public static Map<String,String> getExportedProps(FileSystem fs, Path path) throws IOException {
 +    HashMap<String,String> props = new HashMap<String,String>();
 +
 +    ZipInputStream zis = new ZipInputStream(fs.open(path));
 +    try {
 +      ZipEntry zipEntry;
 +      while ((zipEntry = zis.getNextEntry()) != null) {
 +        if (zipEntry.getName().equals(Constants.EXPORT_TABLE_CONFIG_FILE)) {
 +          BufferedReader in = new BufferedReader(new InputStreamReader(zis, StandardCharsets.UTF_8));
 +          try {
 +            String line;
 +            while ((line = in.readLine()) != null) {
 +              String sa[] = line.split("=", 2);
 +              props.put(sa[0], sa[1]);
 +            }
 +          } finally {
 +            in.close();
 +          }
 +
 +          break;
 +        }
 +      }
 +    } finally {
 +      zis.close();
 +    }
 +    return props;
 +  }
 +
 +  @Override
 +  public void importTable(String tableName, String importDir) throws TableExistsException, AccumuloException, AccumuloSecurityException {
 +    checkArgument(tableName != null, "tableName is null");
 +    checkArgument(importDir != null, "importDir is null");
 +
 +    try {
 +      importDir = checkPath(importDir, "Table", "").toString();
 +    } catch (IOException e) {
 +      throw new AccumuloException(e);
 +    }
 +
 +    try {
 +      FileSystem fs = new Path(importDir).getFileSystem(CachedConfiguration.getInstance());
 +      Map<String,String> props = getExportedProps(fs, new Path(importDir, Constants.EXPORT_FILE));
 +
 +      for (Entry<String,String> entry : props.entrySet()) {
 +        if (Property.isClassProperty(entry.getKey()) && !entry.getValue().contains(Constants.CORE_PACKAGE_NAME)) {
 +          Logger.getLogger(this.getClass()).info(
 +              "Imported table sets '" + entry.getKey() + "' to '" + entry.getValue() + "'.  Ensure this class is on Accumulo classpath.");
 +        }
 +      }
 +
 +    } catch (IOException ioe) {
 +      Logger.getLogger(this.getClass()).warn("Failed to check if imported table references external java classes : " + ioe.getMessage());
 +    }
 +
 +    List<ByteBuffer> args = Arrays.asList(ByteBuffer.wrap(tableName.getBytes(StandardCharsets.UTF_8)), ByteBuffer.wrap(importDir.getBytes(StandardCharsets.UTF_8)));
 +
 +    Map<String,String> opts = Collections.emptyMap();
 +
 +    try {
 +      doTableFateOperation(tableName, AccumuloException.class, FateOperation.TABLE_IMPORT, args, opts);
 +    } catch (TableNotFoundException e) {
 +      // should not happen
 +      throw new AssertionError(e);
 +    }
 +
 +  }
 +
 +  @Override
 +  public void exportTable(String tableName, String exportDir) throws TableNotFoundException, AccumuloException, AccumuloSecurityException {
 +    checkArgument(tableName != null, "tableName is null");
 +    checkArgument(exportDir != null, "exportDir is null");
 +
 +    List<ByteBuffer> args = Arrays.asList(ByteBuffer.wrap(tableName.getBytes(StandardCharsets.UTF_8)), ByteBuffer.wrap(exportDir.getBytes(StandardCharsets.UTF_8)));
 +
 +    Map<String,String> opts = Collections.emptyMap();
 +
 +    try {
 +      doTableFateOperation(tableName, TableNotFoundException.class, FateOperation.TABLE_EXPORT, args, opts);
 +    } catch (TableExistsException e) {
 +      // should not happen
 +      throw new AssertionError(e);
 +    }
 +  }
 +
 +  @Override
 +  public boolean testClassLoad(final String tableName, final String className, final String asTypeName) throws TableNotFoundException, AccumuloException,
 +      AccumuloSecurityException {
 +    checkArgument(tableName != null, "tableName is null");
 +    checkArgument(className != null, "className is null");
 +    checkArgument(asTypeName != null, "asTypeName is null");
 +
 +    try {
 +      return ServerClient.executeRaw(instance, new ClientExecReturn<Boolean,ClientService.Client>() {
 +        @Override
 +        public Boolean execute(ClientService.Client client) throws Exception {
 +          return client.checkTableClass(Tracer.traceInfo(), credentials.toThrift(instance), tableName, className, asTypeName);
 +        }
 +      });
 +    } catch (ThriftTableOperationException e) {
 +      switch (e.getType()) {
 +        case NOTFOUND:
 +          throw new TableNotFoundException(e);
 +        case NAMESPACE_NOTFOUND:
 +          throw new TableNotFoundException(tableName, new NamespaceNotFoundException(e));
 +        default:
 +          throw new AccumuloException(e.description, e);
 +      }
 +    } catch (ThriftSecurityException e) {
 +      throw new AccumuloSecurityException(e.user, e.code, e);
 +    } catch (AccumuloException e) {
 +      throw e;
 +    } catch (Exception e) {
 +      throw new AccumuloException(e);
 +    }
 +  }
 +
 +  @Override
 +  public void attachIterator(String tableName, IteratorSetting setting, EnumSet<IteratorScope> scopes) throws AccumuloSecurityException, AccumuloException,
 +      TableNotFoundException {
 +    testClassLoad(tableName, setting.getIteratorClass(), SortedKeyValueIterator.class.getName());
 +    super.attachIterator(tableName, setting, scopes);
 +  }
 +
 +  @Override
 +  public int addConstraint(String tableName, String constraintClassName) throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
 +    testClassLoad(tableName, constraintClassName, Constraint.class.getName());
 +    return super.addConstraint(tableName, constraintClassName);
 +  }
 +
 +  private void doTableFateOperation(String tableName, Class<? extends Exception> namespaceNotFoundExceptionClass, FateOperation op, List<ByteBuffer> args,
 +      Map<String,String> opts) throws AccumuloSecurityException, AccumuloException, TableExistsException, TableNotFoundException {
 +    try {
 +      doFateOperation(op, args, opts);
 +    } catch (NamespaceExistsException e) {
 +      // should not happen
 +      throw new AssertionError(e);
 +    } catch (NamespaceNotFoundException e) {
 +      if (namespaceNotFoundExceptionClass == null) {
 +        // should not happen
 +        throw new AssertionError(e);
 +      } else if (AccumuloException.class.isAssignableFrom(namespaceNotFoundExceptionClass)) {
 +        throw new AccumuloException("Cannot create table in non-existent namespace", e);
 +      } else if (TableNotFoundException.class.isAssignableFrom(namespaceNotFoundExceptionClass)) {
 +        throw new TableNotFoundException(null, tableName, "Namespace not found", e);
 +      } else {
 +        // should not happen
 +        throw new AssertionError(e);
 +      }
 +    }
    }
  
+   /**
+    * @param instance
+    *          the connection information for this instance
+    * @param credentials
+    *          the username/password for this connection
+    */
+   public TableOperationsImpl(Instance instance, TCredentials credentials) {
+     this(instance, Credentials.fromThrift(credentials));
+   }
  }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/592c8b06/core/src/main/java/org/apache/accumulo/core/client/impl/ConnectorImpl.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/accumulo/blob/592c8b06/core/src/main/java/org/apache/accumulo/core/client/impl/NamespaceOperationsImpl.java
----------------------------------------------------------------------
diff --cc core/src/main/java/org/apache/accumulo/core/client/impl/NamespaceOperationsImpl.java
index 0000000,2af187f..7087ac5
mode 000000,100644..100644
--- a/core/src/main/java/org/apache/accumulo/core/client/impl/NamespaceOperationsImpl.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/NamespaceOperationsImpl.java
@@@ -1,0 -1,244 +1,250 @@@
+ /*
+  * 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.core.client.impl;
+ 
++import static com.google.common.base.Preconditions.checkArgument;
+ import java.nio.ByteBuffer;
+ import java.util.Arrays;
+ import java.util.Collections;
+ import java.util.EnumSet;
+ import java.util.HashMap;
+ import java.util.List;
+ import java.util.Map;
+ import java.util.Map.Entry;
+ import java.util.SortedSet;
+ import java.util.TreeSet;
+ 
+ import org.apache.accumulo.core.client.AccumuloException;
+ import org.apache.accumulo.core.client.AccumuloSecurityException;
+ import org.apache.accumulo.core.client.Instance;
+ import org.apache.accumulo.core.client.IteratorSetting;
+ import org.apache.accumulo.core.client.NamespaceExistsException;
+ import org.apache.accumulo.core.client.NamespaceNotEmptyException;
+ import org.apache.accumulo.core.client.NamespaceNotFoundException;
+ import org.apache.accumulo.core.client.admin.TableOperations;
+ import org.apache.accumulo.core.client.TableExistsException;
+ import org.apache.accumulo.core.client.TableNotFoundException;
+ import org.apache.accumulo.core.client.impl.thrift.ClientService;
+ import org.apache.accumulo.core.client.impl.thrift.SecurityErrorCode;
+ import org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException;
+ import org.apache.accumulo.core.client.impl.thrift.ThriftTableOperationException;
+ import org.apache.accumulo.core.constraints.Constraint;
+ import org.apache.accumulo.core.iterators.IteratorUtil.IteratorScope;
+ import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
+ import org.apache.accumulo.core.master.thrift.FateOperation;
+ import org.apache.accumulo.core.master.thrift.MasterClientService;
+ import org.apache.accumulo.core.security.Credentials;
 -import org.apache.accumulo.core.util.ArgumentChecker;
+ import org.apache.accumulo.core.util.OpTimer;
+ import org.apache.accumulo.trace.instrument.Tracer;
+ import org.apache.log4j.Level;
+ import org.apache.log4j.Logger;
+ 
+ public class NamespaceOperationsImpl extends NamespaceOperationsHelper {
+   private Instance instance;
+   private Credentials credentials;
+   private TableOperationsImpl tableOps;
+ 
+   private static final Logger log = Logger.getLogger(TableOperations.class);
+ 
+   public NamespaceOperationsImpl(Instance instance, Credentials credentials, TableOperationsImpl tableOps) {
 -    ArgumentChecker.notNull(instance, credentials);
++    checkArgument(instance != null, "instance is null");
++    checkArgument(credentials != null, "credentials is null");
+     this.instance = instance;
+     this.credentials = credentials;
+     this.tableOps = tableOps;
+   }
+ 
+   @Override
+   public SortedSet<String> list() {
+     OpTimer opTimer = new OpTimer(log, Level.TRACE).start("Fetching list of namespaces...");
+     TreeSet<String> namespaces = new TreeSet<String>(Namespaces.getNameToIdMap(instance).keySet());
+     opTimer.stop("Fetched " + namespaces.size() + " namespaces in %DURATION%");
+     return namespaces;
+   }
+ 
+   @Override
+   public boolean exists(String namespace) {
 -    ArgumentChecker.notNull(namespace);
++    checkArgument(namespace != null, "namespace is null");
+ 
+     OpTimer opTimer = new OpTimer(log, Level.TRACE).start("Checking if namespace " + namespace + " exists...");
+     boolean exists = Namespaces.getNameToIdMap(instance).containsKey(namespace);
+     opTimer.stop("Checked existance of " + exists + " in %DURATION%");
+     return exists;
+   }
+ 
+   @Override
+   public void create(String namespace) throws AccumuloException, AccumuloSecurityException, NamespaceExistsException {
 -    ArgumentChecker.notNull(namespace);
++    checkArgument(namespace != null, "namespace is null");
+ 
+     try {
+       doNamespaceFateOperation(FateOperation.NAMESPACE_CREATE, Arrays.asList(ByteBuffer.wrap(namespace.getBytes())), Collections.<String,String> emptyMap());
+     } catch (NamespaceNotFoundException e) {
+       // should not happen
+       throw new AssertionError(e);
+     }
+   }
+ 
+   @Override
+   public void delete(String namespace) throws AccumuloException, AccumuloSecurityException, NamespaceNotFoundException, NamespaceNotEmptyException {
 -    ArgumentChecker.notNull(namespace);
++    checkArgument(namespace != null, "namespace is null");
+     String namespaceId = Namespaces.getNamespaceId(instance, namespace);
+ 
+     if (namespaceId.equals(Namespaces.ACCUMULO_NAMESPACE_ID) || namespaceId.equals(Namespaces.DEFAULT_NAMESPACE_ID)) {
+       log.debug(credentials.getPrincipal() + " attempted to delete the " + namespaceId + " namespace");
+       throw new AccumuloSecurityException(credentials.getPrincipal(), SecurityErrorCode.UNSUPPORTED_OPERATION);
+     }
+ 
+     if (Namespaces.getTableIds(instance, namespaceId).size() > 0) {
+       throw new NamespaceNotEmptyException(namespaceId, namespace, null);
+     }
+ 
+     List<ByteBuffer> args = Arrays.asList(ByteBuffer.wrap(namespace.getBytes()));
+     Map<String,String> opts = new HashMap<String,String>();
+ 
+     try {
+       doNamespaceFateOperation(FateOperation.NAMESPACE_DELETE, args, opts);
+     } catch (NamespaceExistsException e) {
+       // should not happen
+       throw new AssertionError(e);
+     }
+ 
+   }
+ 
+   @Override
+   public void rename(String oldNamespaceName, String newNamespaceName) throws AccumuloSecurityException, NamespaceNotFoundException, AccumuloException,
+       NamespaceExistsException {
+ 
+     List<ByteBuffer> args = Arrays.asList(ByteBuffer.wrap(oldNamespaceName.getBytes()), ByteBuffer.wrap(newNamespaceName.getBytes()));
+     Map<String,String> opts = new HashMap<String,String>();
+     doNamespaceFateOperation(FateOperation.NAMESPACE_RENAME, args, opts);
+   }
+ 
+   @Override
+   public void setProperty(final String namespace, final String property, final String value) throws AccumuloException, AccumuloSecurityException,
+       NamespaceNotFoundException {
 -    ArgumentChecker.notNull(namespace, property, value);
++    checkArgument(namespace != null, "namespace is null");
++    checkArgument(property != null, "property is null");
++    checkArgument(value != null, "value is null");
+ 
+     MasterClient.executeNamespace(instance, new ClientExec<MasterClientService.Client>() {
+       @Override
+       public void execute(MasterClientService.Client client) throws Exception {
+         client.setNamespaceProperty(Tracer.traceInfo(), credentials.toThrift(instance), namespace, property, value);
+       }
+     });
+   }
+ 
+   @Override
+   public void removeProperty(final String namespace, final String property) throws AccumuloException, AccumuloSecurityException, NamespaceNotFoundException {
 -    ArgumentChecker.notNull(namespace, property);
++    checkArgument(namespace != null, "namespace is null");
++    checkArgument(property != null, "property is null");
+ 
+     MasterClient.executeNamespace(instance, new ClientExec<MasterClientService.Client>() {
+       @Override
+       public void execute(MasterClientService.Client client) throws Exception {
+         client.removeNamespaceProperty(Tracer.traceInfo(), credentials.toThrift(instance), namespace, property);
+       }
+     });
+   }
+ 
+   @Override
+   public Iterable<Entry<String,String>> getProperties(final String namespace) throws AccumuloException, NamespaceNotFoundException {
 -    ArgumentChecker.notNull(namespace);
++    checkArgument(namespace != null, "namespace is null");
+     try {
+       return ServerClient.executeRaw(instance, new ClientExecReturn<Map<String,String>,ClientService.Client>() {
+         @Override
+         public Map<String,String> execute(ClientService.Client client) throws Exception {
+           return client.getNamespaceConfiguration(Tracer.traceInfo(), credentials.toThrift(instance), namespace);
+         }
+       }).entrySet();
+     } catch (ThriftTableOperationException e) {
+       switch (e.getType()) {
+         case NAMESPACE_NOTFOUND:
+           throw new NamespaceNotFoundException(e);
+         case OTHER:
+         default:
+           throw new AccumuloException(e.description, e);
+       }
+     } catch (AccumuloException e) {
+       throw e;
+     } catch (Exception e) {
+       throw new AccumuloException(e);
+     }
+ 
+   }
+ 
+   @Override
+   public Map<String,String> namespaceIdMap() {
+     return Namespaces.getNameToIdMap(instance);
+   }
+ 
+   @Override
+   public boolean testClassLoad(final String namespace, final String className, final String asTypeName) throws NamespaceNotFoundException, AccumuloException,
+       AccumuloSecurityException {
 -    ArgumentChecker.notNull(namespace, className, asTypeName);
++    checkArgument(namespace != null, "namespace is null");
++    checkArgument(className != null, "className is null");
++    checkArgument(asTypeName != null, "asTypeName is null");
+ 
+     try {
+       return ServerClient.executeRaw(instance, new ClientExecReturn<Boolean,ClientService.Client>() {
+         @Override
+         public Boolean execute(ClientService.Client client) throws Exception {
+           return client.checkNamespaceClass(Tracer.traceInfo(), credentials.toThrift(instance), namespace, className, asTypeName);
+         }
+       });
+     } catch (ThriftTableOperationException e) {
+       switch (e.getType()) {
+         case NAMESPACE_NOTFOUND:
+           throw new NamespaceNotFoundException(e);
+         default:
+           throw new AccumuloException(e.description, e);
+       }
+     } catch (ThriftSecurityException e) {
+       throw new AccumuloSecurityException(e.user, e.code, e);
+     } catch (AccumuloException e) {
+       throw e;
+     } catch (Exception e) {
+       throw new AccumuloException(e);
+     }
+   }
+ 
+   @Override
+   public void attachIterator(String namespace, IteratorSetting setting, EnumSet<IteratorScope> scopes) throws AccumuloSecurityException, AccumuloException,
+       NamespaceNotFoundException {
+     testClassLoad(namespace, setting.getIteratorClass(), SortedKeyValueIterator.class.getName());
+     super.attachIterator(namespace, setting, scopes);
+   }
+ 
+   @Override
+   public int addConstraint(String namespace, String constraintClassName) throws AccumuloException, AccumuloSecurityException, NamespaceNotFoundException {
+     testClassLoad(namespace, constraintClassName, Constraint.class.getName());
+     return super.addConstraint(namespace, constraintClassName);
+   }
+ 
+   private String doNamespaceFateOperation(FateOperation op, List<ByteBuffer> args, Map<String,String> opts) throws AccumuloSecurityException,
+       AccumuloException, NamespaceExistsException, NamespaceNotFoundException {
+     try {
+       return tableOps.doFateOperation(op, args, opts);
+     } catch (TableExistsException e) {
+       // should not happen
+       throw new AssertionError(e);
+     } catch (TableNotFoundException e) {
+       // should not happen
+       throw new AssertionError(e);
+     }
+   }
+ }


[08/17] git commit: ACCUMULO-2726 missed a deprecation for part of Mock.

Posted by bu...@apache.org.
ACCUMULO-2726 missed a deprecation for part of Mock.


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

Branch: refs/heads/master
Commit: 9bdad265990eb57ca7e101c0a3a446856798de55
Parents: 956f19b
Author: Sean Busbey <bu...@cloudera.com>
Authored: Fri Apr 25 14:12:51 2014 -0500
Committer: Sean Busbey <bu...@cloudera.com>
Committed: Fri Apr 25 14:15:01 2014 -0500

----------------------------------------------------------------------
 .../apache/accumulo/core/client/mock/MockSecurityOperations.java | 4 ++++
 1 file changed, 4 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/9bdad265/core/src/main/java/org/apache/accumulo/core/client/mock/MockSecurityOperations.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mock/MockSecurityOperations.java b/core/src/main/java/org/apache/accumulo/core/client/mock/MockSecurityOperations.java
index 107c711..6271059 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mock/MockSecurityOperations.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mock/MockSecurityOperations.java
@@ -16,6 +16,10 @@
  */
 package org.apache.accumulo.core.client.mock;
 
+/**
+ * @deprecated since 1.6.0; not intended for public api and you should not use it.
+ */
+@Deprecated
 public class MockSecurityOperations extends MockSecurityOperationsImpl {
 
   MockSecurityOperations(MockAccumulo acu) {


[13/17] ACCUMULO-2726 cleanup for master's branch.

Posted by bu...@apache.org.
http://git-wip-us.apache.org/repos/asf/accumulo/blob/f277fea9/core/src/test/java/org/apache/accumulo/core/client/admin/TableOperationsHelperTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/accumulo/core/client/admin/TableOperationsHelperTest.java b/core/src/test/java/org/apache/accumulo/core/client/admin/TableOperationsHelperTest.java
deleted file mode 100644
index 1d91574..0000000
--- a/core/src/test/java/org/apache/accumulo/core/client/admin/TableOperationsHelperTest.java
+++ /dev/null
@@ -1,223 +0,0 @@
-/*
- * 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.core.client.admin;
-
-import java.io.IOException;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Set;
-import java.util.SortedSet;
-import java.util.TreeMap;
-
-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.TableExistsException;
-import org.apache.accumulo.core.client.TableNotFoundException;
-import org.apache.accumulo.core.data.Range;
-import org.apache.accumulo.core.security.Authorizations;
-import org.apache.hadoop.io.Text;
-
-/**
- * This class is left in place specifically to test for regressions against the published version of TableOperationsHelper.
- */
-public class TableOperationsHelperTest extends org.apache.accumulo.core.client.impl.TableOperationsHelperTest {
-  
-  @SuppressWarnings("deprecation")
-  static class Tester extends TableOperationsHelper {
-    Map<String,Map<String,String>> settings = new HashMap<String,Map<String,String>>();
-    
-    @Override
-    public SortedSet<String> list() {
-      return null;
-    }
-    
-    @Override
-    public boolean exists(String tableName) {
-      return true;
-    }
-    
-    @Override
-    public void create(String tableName) throws AccumuloException, AccumuloSecurityException, TableExistsException {}
-    
-    @Override
-    public void create(String tableName, boolean limitVersion) throws AccumuloException, AccumuloSecurityException, TableExistsException {
-      create(tableName, limitVersion, TimeType.MILLIS);
-    }
-    
-    @Override
-    public void create(String tableName, boolean versioningIter, TimeType timeType) throws AccumuloException, AccumuloSecurityException, TableExistsException {}
-    
-    @Override
-    public void addSplits(String tableName, SortedSet<Text> partitionKeys) throws TableNotFoundException, AccumuloException, AccumuloSecurityException {}
-    
-    @Deprecated
-    @Override
-    public Collection<Text> getSplits(String tableName) throws TableNotFoundException {
-      return null;
-    }
-    
-    @Deprecated
-    @Override
-    public Collection<Text> getSplits(String tableName, int maxSplits) throws TableNotFoundException {
-      return null;
-    }
-    
-    @Override
-    public Collection<Text> listSplits(String tableName) throws TableNotFoundException {
-      return null;
-    }
-    
-    @Override
-    public Collection<Text> listSplits(String tableName, int maxSplits) throws TableNotFoundException {
-      return null;
-    }
-    
-    @Override
-    public Text getMaxRow(String tableName, Authorizations auths, Text startRow, boolean startInclusive, Text endRow, boolean endInclusive)
-        throws TableNotFoundException, AccumuloException, AccumuloSecurityException {
-      return null;
-    }
-    
-    @Override
-    public void merge(String tableName, Text start, Text end) throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
-      
-    }
-    
-    @Override
-    public void deleteRows(String tableName, Text start, Text end) throws AccumuloException, AccumuloSecurityException, TableNotFoundException {}
-    
-    @Override
-    public void compact(String tableName, Text start, Text end, boolean flush, boolean wait) throws AccumuloSecurityException, TableNotFoundException,
-        AccumuloException {}
-    
-    @Override
-    public void compact(String tableName, Text start, Text end, List<IteratorSetting> iterators, boolean flush, boolean wait) throws AccumuloSecurityException,
-        TableNotFoundException, AccumuloException {}
-    
-    @Override
-    public void delete(String tableName) throws AccumuloException, AccumuloSecurityException, TableNotFoundException {}
-    
-    @Override
-    public void clone(String srcTableName, String newTableName, boolean flush, Map<String,String> propertiesToSet, Set<String> propertiesToExclude)
-        throws AccumuloException, AccumuloSecurityException, TableNotFoundException, TableExistsException {}
-    
-    @Override
-    public void rename(String oldTableName, String newTableName) throws AccumuloSecurityException, TableNotFoundException, AccumuloException,
-        TableExistsException {}
-    
-    @Deprecated
-    @Override
-    public void flush(String tableName) throws AccumuloException, AccumuloSecurityException {}
-    
-    @Override
-    public void flush(String tableName, Text start, Text end, boolean wait) throws AccumuloException, AccumuloSecurityException, TableNotFoundException {}
-    
-    @Override
-    public void setProperty(String tableName, String property, String value) throws AccumuloException, AccumuloSecurityException {
-      if (!settings.containsKey(tableName))
-        settings.put(tableName, new TreeMap<String,String>());
-      settings.get(tableName).put(property, value);
-    }
-    
-    @Override
-    public void removeProperty(String tableName, String property) throws AccumuloException, AccumuloSecurityException {
-      if (!settings.containsKey(tableName))
-        return;
-      settings.get(tableName).remove(property);
-    }
-    
-    @Override
-    public Iterable<Entry<String,String>> getProperties(String tableName) throws AccumuloException, TableNotFoundException {
-      Map<String,String> empty = Collections.emptyMap();
-      if (!settings.containsKey(tableName))
-        return empty.entrySet();
-      return settings.get(tableName).entrySet();
-    }
-    
-    @Override
-    public void setLocalityGroups(String tableName, Map<String,Set<Text>> groups) throws AccumuloException, AccumuloSecurityException, TableNotFoundException {}
-    
-    @Override
-    public Map<String,Set<Text>> getLocalityGroups(String tableName) throws AccumuloException, TableNotFoundException {
-      return null;
-    }
-    
-    @Override
-    public Set<Range> splitRangeByTablets(String tableName, Range range, int maxSplits) throws AccumuloException, AccumuloSecurityException,
-        TableNotFoundException {
-      return null;
-    }
-    
-    @Override
-    public void importDirectory(String tableName, String dir, String failureDir, boolean setTime) throws TableNotFoundException, IOException,
-        AccumuloException, AccumuloSecurityException {}
-    
-    @Override
-    public void offline(String tableName) throws AccumuloSecurityException, AccumuloException, TableNotFoundException {
-      
-    }
-    
-    @Override
-    public void online(String tableName) throws AccumuloSecurityException, AccumuloException, TableNotFoundException {}
-    
-    @Override
-    public void offline(String tableName, boolean wait) throws AccumuloSecurityException, AccumuloException, TableNotFoundException {
-      
-    }
-    
-    @Override
-    public void online(String tableName, boolean wait) throws AccumuloSecurityException, AccumuloException, TableNotFoundException {}
-    
-    @Override
-    public void clearLocatorCache(String tableName) throws TableNotFoundException {}
-    
-    @Override
-    public Map<String,String> tableIdMap() {
-      return null;
-    }
-    
-    @Override
-    public List<DiskUsage> getDiskUsage(Set<String> tables) throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
-      return null;
-    }
-    
-    @Override
-    public void importTable(String tableName, String exportDir) throws TableExistsException, AccumuloException, AccumuloSecurityException {}
-    
-    @Override
-    public void exportTable(String tableName, String exportDir) throws TableNotFoundException, AccumuloException, AccumuloSecurityException {}
-    
-    @Override
-    public void cancelCompaction(String tableName) throws AccumuloSecurityException, TableNotFoundException, AccumuloException {}
-    
-    @Override
-    public boolean testClassLoad(String tableName, String className, String asTypeName) throws AccumuloException, AccumuloSecurityException,
-        TableNotFoundException {
-      return false;
-    }
-  }
-
-  @Override
-  protected org.apache.accumulo.core.client.impl.TableOperationsHelper getHelper() {
-    return new Tester();
-  }
-}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/f277fea9/shell/src/main/java/org/apache/accumulo/shell/mock/MockShell.java
----------------------------------------------------------------------
diff --git a/shell/src/main/java/org/apache/accumulo/shell/mock/MockShell.java b/shell/src/main/java/org/apache/accumulo/shell/mock/MockShell.java
index 38bb501..cf028eb 100644
--- a/shell/src/main/java/org/apache/accumulo/shell/mock/MockShell.java
+++ b/shell/src/main/java/org/apache/accumulo/shell/mock/MockShell.java
@@ -21,11 +21,7 @@ import java.io.IOException;
 import java.io.InputStream;
 import java.io.OutputStream;
 import java.nio.charset.StandardCharsets;
-import java.io.Writer;
 
-import org.apache.commons.io.output.WriterOutputStream;
-
-import org.apache.commons.cli.CommandLine;
 import jline.console.ConsoleReader;
 
 import org.apache.accumulo.core.client.mock.MockInstance;
@@ -41,29 +37,12 @@ public class MockShell extends Shell {
   protected InputStream in;
   protected OutputStream out;
 
-  /**
-   * Will only be set if you use either the Writer constructor or the setWriter(Writer) method
-   * @deprecated since 1.6.0; use out
-   */
-  @Deprecated
-  protected Writer writer = null;
-
   public MockShell(InputStream in, OutputStream out) throws IOException {
     super();
     this.in = in;
     this.out = out;
-    // we presume they don't use the writer field unless they use the other constructor.
   }
 
-  /**
-   * @deprecated since 1.6.0; use OutputStream version
-   */
-  @Deprecated
-  public MockShell(InputStream in, Writer out) throws IOException {
-    this(in, new WriterOutputStream(out, Constants.UTF8.name()));
-    this.writer = out;
-  }
-  
   public boolean config(String... args) {
     configError = super.config(args);
     
@@ -90,15 +69,6 @@ public class MockShell extends Shell {
     instance = new MockInstance();
   }
 
-  /**
-   * @deprecated since 1.6.0; use ShellOptionsJC version
-   */
-  @Deprecated
-  protected void setInstance(CommandLine cl) {
-    // same result as in previous version
-    setInstance((ShellOptionsJC)null);
-  }
-  
   public int start() throws IOException {
     if (configError)
       return 1;
@@ -155,15 +125,6 @@ public class MockShell extends Shell {
   }
 
   /**
-   * @deprecated since 1.6.0; use the OutputStream version
-   */
-  @Deprecated
-  public void setConsoleWriter(Writer out) {
-    setConsoleWriter(new WriterOutputStream(out, Constants.UTF8.name()));
-    this.writer = out;
-  }
-  
-  /**
    * Convenience method to create the byte-array to hand to the console
    * 
    * @param commands


[04/17] ACCUMULO-2726 Adds back missing methods for binary backwards compat with non-Deprecated methods from 1.5.0. Marks things that we're for sure removing with @Deprecated; mostly things that shouldn't have been in public to begin with.

Posted by bu...@apache.org.
http://git-wip-us.apache.org/repos/asf/accumulo/blob/f4454a06/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperationsImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperationsImpl.java b/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperationsImpl.java
index 3d69cc1..05b5b01 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperationsImpl.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperationsImpl.java
@@ -16,118 +16,15 @@
  */
 package org.apache.accumulo.core.client.admin;
 
-import java.io.BufferedReader;
-import java.io.IOException;
-import java.io.InputStreamReader;
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.EnumSet;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Random;
-import java.util.Set;
-import java.util.SortedSet;
-import java.util.TreeMap;
-import java.util.TreeSet;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicReference;
-import java.util.zip.ZipEntry;
-import java.util.zip.ZipInputStream;
-
-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.Instance;
-import org.apache.accumulo.core.client.IsolatedScanner;
-import org.apache.accumulo.core.client.IteratorSetting;
-import org.apache.accumulo.core.client.NamespaceExistsException;
-import org.apache.accumulo.core.client.NamespaceNotFoundException;
-import org.apache.accumulo.core.client.RowIterator;
-import org.apache.accumulo.core.client.Scanner;
-import org.apache.accumulo.core.client.TableDeletedException;
-import org.apache.accumulo.core.client.TableExistsException;
-import org.apache.accumulo.core.client.TableNotFoundException;
-import org.apache.accumulo.core.client.TableOfflineException;
-import org.apache.accumulo.core.client.impl.AccumuloServerException;
-import org.apache.accumulo.core.client.impl.ClientExec;
-import org.apache.accumulo.core.client.impl.ClientExecReturn;
-import org.apache.accumulo.core.client.impl.MasterClient;
-import org.apache.accumulo.core.client.impl.ServerClient;
-import org.apache.accumulo.core.client.impl.ServerConfigurationUtil;
-import org.apache.accumulo.core.client.impl.Tables;
-import org.apache.accumulo.core.client.impl.TabletLocator;
-import org.apache.accumulo.core.client.impl.TabletLocator.TabletLocation;
-import org.apache.accumulo.core.client.impl.thrift.ClientService;
-import org.apache.accumulo.core.client.impl.thrift.ClientService.Client;
-import org.apache.accumulo.core.client.impl.thrift.TDiskUsage;
-import org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException;
-import org.apache.accumulo.core.client.impl.thrift.ThriftTableOperationException;
-import org.apache.accumulo.core.conf.AccumuloConfiguration;
-import org.apache.accumulo.core.conf.ConfigurationCopy;
-import org.apache.accumulo.core.conf.Property;
-import org.apache.accumulo.core.constraints.Constraint;
-import org.apache.accumulo.core.data.ByteSequence;
-import org.apache.accumulo.core.data.Key;
-import org.apache.accumulo.core.data.KeyExtent;
-import org.apache.accumulo.core.data.Range;
-import org.apache.accumulo.core.data.Value;
-import org.apache.accumulo.core.iterators.IteratorUtil;
-import org.apache.accumulo.core.iterators.IteratorUtil.IteratorScope;
-import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
-import org.apache.accumulo.core.master.state.tables.TableState;
-import org.apache.accumulo.core.master.thrift.FateOperation;
-import org.apache.accumulo.core.master.thrift.MasterClientService;
-import org.apache.accumulo.core.metadata.MetadataServicer;
-import org.apache.accumulo.core.metadata.MetadataTable;
-import org.apache.accumulo.core.metadata.RootTable;
-import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection;
-import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.security.Credentials;
-import org.apache.accumulo.core.tabletserver.thrift.NotServingTabletException;
-import org.apache.accumulo.core.tabletserver.thrift.TabletClientService;
-import org.apache.accumulo.core.util.ArgumentChecker;
-import org.apache.accumulo.core.util.ByteBufferUtil;
-import org.apache.accumulo.core.util.CachedConfiguration;
-import org.apache.accumulo.core.util.LocalityGroupUtil;
-import org.apache.accumulo.core.util.MapCounter;
-import org.apache.accumulo.core.util.NamingThreadFactory;
-import org.apache.accumulo.core.util.OpTimer;
-import org.apache.accumulo.core.util.Pair;
-import org.apache.accumulo.core.util.StringUtil;
-import org.apache.accumulo.core.util.TextUtil;
-import org.apache.accumulo.core.util.ThriftUtil;
-import org.apache.accumulo.core.util.UtilWaitThread;
-import org.apache.accumulo.core.volume.VolumeConfiguration;
-import org.apache.accumulo.trace.instrument.Tracer;
-import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.Text;
-import org.apache.log4j.Level;
-import org.apache.log4j.Logger;
-import org.apache.thrift.TApplicationException;
-import org.apache.thrift.TException;
-import org.apache.thrift.transport.TTransportException;
-
-public class TableOperationsImpl extends TableOperationsHelper {
-  private Instance instance;
-  private Credentials credentials;
-
-  public static final String CLONE_EXCLUDE_PREFIX = "!";
-
-  private static final Logger log = Logger.getLogger(TableOperations.class);
+import org.apache.accumulo.core.security.thrift.TCredentials;
 
+/**
+ * @deprecated since 1.6.0; not intended for public api and you should not use it.
+ */
+@Deprecated
+public class TableOperationsImpl extends org.apache.accumulo.core.client.impl.TableOperationsImpl {
   /**
    * @param instance
    *          the connection information for this instance
@@ -135,1459 +32,16 @@ public class TableOperationsImpl extends TableOperationsHelper {
    *          the username/password for this connection
    */
   public TableOperationsImpl(Instance instance, Credentials credentials) {
-    ArgumentChecker.notNull(instance, credentials);
-    this.instance = instance;
-    this.credentials = credentials;
-  }
-
-  /**
-   * Retrieve a list of tables in Accumulo.
-   * 
-   * @return List of tables in accumulo
-   */
-  @Override
-  public SortedSet<String> list() {
-    OpTimer opTimer = new OpTimer(log, Level.TRACE).start("Fetching list of tables...");
-    TreeSet<String> tableNames = new TreeSet<String>(Tables.getNameToIdMap(instance).keySet());
-    opTimer.stop("Fetched " + tableNames.size() + " table names in %DURATION%");
-    return tableNames;
-  }
-
-  /**
-   * A method to check if a table exists in Accumulo.
-   * 
-   * @param tableName
-   *          the name of the table
-   * @return true if the table exists
-   */
-  @Override
-  public boolean exists(String tableName) {
-    ArgumentChecker.notNull(tableName);
-    if (tableName.equals(MetadataTable.NAME) || tableName.equals(RootTable.NAME))
-      return true;
-
-    OpTimer opTimer = new OpTimer(log, Level.TRACE).start("Checking if table " + tableName + " exists...");
-    boolean exists = Tables.getNameToIdMap(instance).containsKey(tableName);
-    opTimer.stop("Checked existance of " + exists + " in %DURATION%");
-    return exists;
-  }
-
-  /**
-   * Create a table with no special configuration
-   * 
-   * @param tableName
-   *          the name of the table
-   * @throws AccumuloException
-   *           if a general error occurs
-   * @throws AccumuloSecurityException
-   *           if the user does not have permission
-   * @throws TableExistsException
-   *           if the table already exists
-   */
-  @Override
-  public void create(String tableName) throws AccumuloException, AccumuloSecurityException, TableExistsException {
-    create(tableName, true, TimeType.MILLIS);
+    super(instance, credentials);
   }
 
   /**
-   * @param tableName
-   *          the name of the table
-   * @param limitVersion
-   *          Enables/disables the versioning iterator, which will limit the number of Key versions kept.
-   */
-  @Override
-  public void create(String tableName, boolean limitVersion) throws AccumuloException, AccumuloSecurityException, TableExistsException {
-    create(tableName, limitVersion, TimeType.MILLIS);
-  }
-
-  /**
-   * @param tableName
-   *          the name of the table
-   * @param timeType
-   *          specifies logical or real-time based time recording for entries in the table
-   * @param limitVersion
-   *          Enables/disables the versioning iterator, which will limit the number of Key versions kept.
-   */
-  @Override
-  public void create(String tableName, boolean limitVersion, TimeType timeType) throws AccumuloException, AccumuloSecurityException, TableExistsException {
-    ArgumentChecker.notNull(tableName, timeType);
-
-    List<ByteBuffer> args = Arrays.asList(ByteBuffer.wrap(tableName.getBytes(Constants.UTF8)), ByteBuffer.wrap(timeType.name().getBytes(Constants.UTF8)));
-
-    Map<String,String> opts;
-    if (limitVersion)
-      opts = IteratorUtil.generateInitialTableProperties(limitVersion);
-    else
-      opts = Collections.emptyMap();
-
-    try {
-      doTableFateOperation(tableName, AccumuloException.class, FateOperation.TABLE_CREATE, args, opts);
-    } catch (TableNotFoundException e) {
-      // should not happen
-      throw new AssertionError(e);
-    }
-  }
-
-  private long beginFateOperation() throws ThriftSecurityException, TException {
-    while (true) {
-      MasterClientService.Iface client = null;
-      try {
-        client = MasterClient.getConnectionWithRetry(instance);
-        return client.beginFateOperation(Tracer.traceInfo(), credentials.toThrift(instance));
-      } catch (TTransportException tte) {
-        log.debug("Failed to call beginFateOperation(), retrying ... ", tte);
-        UtilWaitThread.sleep(100);
-      } finally {
-        MasterClient.close(client);
-      }
-    }
-  }
-
-  // This method is for retrying in the case of network failures; anything else it passes to the caller to deal with
-  private void executeFateOperation(long opid, FateOperation op, List<ByteBuffer> args, Map<String,String> opts, boolean autoCleanUp)
-      throws ThriftSecurityException, TException, ThriftTableOperationException {
-    while (true) {
-      MasterClientService.Iface client = null;
-      try {
-        client = MasterClient.getConnectionWithRetry(instance);
-        client.executeFateOperation(Tracer.traceInfo(), credentials.toThrift(instance), opid, op, args, opts, autoCleanUp);
-        break;
-      } catch (TTransportException tte) {
-        log.debug("Failed to call executeFateOperation(), retrying ... ", tte);
-        UtilWaitThread.sleep(100);
-      } finally {
-        MasterClient.close(client);
-      }
-    }
-  }
-
-  private String waitForFateOperation(long opid) throws ThriftSecurityException, TException, ThriftTableOperationException {
-    while (true) {
-      MasterClientService.Iface client = null;
-      try {
-        client = MasterClient.getConnectionWithRetry(instance);
-        return client.waitForFateOperation(Tracer.traceInfo(), credentials.toThrift(instance), opid);
-      } catch (TTransportException tte) {
-        log.debug("Failed to call waitForFateOperation(), retrying ... ", tte);
-        UtilWaitThread.sleep(100);
-      } finally {
-        MasterClient.close(client);
-      }
-    }
-  }
-
-  private void finishFateOperation(long opid) throws ThriftSecurityException, TException {
-    while (true) {
-      MasterClientService.Iface client = null;
-      try {
-        client = MasterClient.getConnectionWithRetry(instance);
-        client.finishFateOperation(Tracer.traceInfo(), credentials.toThrift(instance), opid);
-        break;
-      } catch (TTransportException tte) {
-        log.debug("Failed to call finishFateOperation(), retrying ... ", tte);
-        UtilWaitThread.sleep(100);
-      } finally {
-        MasterClient.close(client);
-      }
-    }
-  }
-
-  String doFateOperation(FateOperation op, List<ByteBuffer> args, Map<String,String> opts) throws AccumuloSecurityException, TableExistsException,
-      TableNotFoundException, AccumuloException, NamespaceExistsException, NamespaceNotFoundException {
-    return doFateOperation(op, args, opts, true);
-  }
-
-  String doFateOperation(FateOperation op, List<ByteBuffer> args, Map<String,String> opts, boolean wait) throws AccumuloSecurityException,
-      TableExistsException, TableNotFoundException, AccumuloException, NamespaceExistsException, NamespaceNotFoundException {
-    Long opid = null;
-
-    try {
-      opid = beginFateOperation();
-      executeFateOperation(opid, op, args, opts, !wait);
-      if (!wait) {
-        opid = null;
-        return null;
-      }
-      String ret = waitForFateOperation(opid);
-      return ret;
-    } catch (ThriftSecurityException e) {
-      String tableName = ByteBufferUtil.toString(args.get(0));
-      switch (e.getCode()) {
-        case TABLE_DOESNT_EXIST:
-          throw new TableNotFoundException(null, tableName, "Target table does not exist");
-        case NAMESPACE_DOESNT_EXIST:
-          throw new NamespaceNotFoundException(null, tableName, "Target namespace does not exist");
-        default:
-          String tableInfo = Tables.getPrintableTableInfoFromName(instance, tableName);
-          throw new AccumuloSecurityException(e.user, e.code, tableInfo, e);
-      }
-    } catch (ThriftTableOperationException e) {
-      switch (e.getType()) {
-        case EXISTS:
-          throw new TableExistsException(e);
-        case NOTFOUND:
-          throw new TableNotFoundException(e);
-        case NAMESPACE_EXISTS:
-          throw new NamespaceExistsException(e);
-        case NAMESPACE_NOTFOUND:
-          throw new NamespaceNotFoundException(e);
-        case OFFLINE:
-          throw new TableOfflineException(instance, null);
-        default:
-          throw new AccumuloException(e.description, e);
-      }
-    } catch (Exception e) {
-      throw new AccumuloException(e.getMessage(), e);
-    } finally {
-      Tables.clearCache(instance);
-      // always finish table op, even when exception
-      if (opid != null)
-        try {
-          finishFateOperation(opid);
-        } catch (Exception e) {
-          log.warn(e.getMessage(), e);
-        }
-    }
-  }
-
-  private static class SplitEnv {
-    private String tableName;
-    private String tableId;
-    private ExecutorService executor;
-    private CountDownLatch latch;
-    private AtomicReference<Exception> exception;
-
-    SplitEnv(String tableName, String tableId, ExecutorService executor, CountDownLatch latch, AtomicReference<Exception> exception) {
-      this.tableName = tableName;
-      this.tableId = tableId;
-      this.executor = executor;
-      this.latch = latch;
-      this.exception = exception;
-    }
-  }
-
-  private class SplitTask implements Runnable {
-
-    private List<Text> splits;
-    private SplitEnv env;
-
-    SplitTask(SplitEnv env, List<Text> splits) {
-      this.env = env;
-      this.splits = splits;
-    }
-
-    @Override
-    public void run() {
-      try {
-        if (env.exception.get() != null)
-          return;
-
-        if (splits.size() <= 2) {
-          addSplits(env.tableName, new TreeSet<Text>(splits), env.tableId);
-          for (int i = 0; i < splits.size(); i++)
-            env.latch.countDown();
-          return;
-        }
-
-        int mid = splits.size() / 2;
-
-        // split the middle split point to ensure that child task split different tablets and can therefore
-        // run in parallel
-        addSplits(env.tableName, new TreeSet<Text>(splits.subList(mid, mid + 1)), env.tableId);
-        env.latch.countDown();
-
-        env.executor.submit(new SplitTask(env, splits.subList(0, mid)));
-        env.executor.submit(new SplitTask(env, splits.subList(mid + 1, splits.size())));
-
-      } catch (Exception e) {
-        env.exception.compareAndSet(null, e);
-      }
-    }
-
-  }
-
-  /**
-   * @param tableName
-   *          the name of the table
-   * @param partitionKeys
-   *          a sorted set of row key values to pre-split the table on
-   * @throws AccumuloException
-   *           if a general error occurs
-   * @throws AccumuloSecurityException
-   *           if the user does not have permission
-   * @throws TableNotFoundException
-   *           if the table does not exist
-   */
-  @Override
-  public void addSplits(String tableName, SortedSet<Text> partitionKeys) throws TableNotFoundException, AccumuloException, AccumuloSecurityException {
-    String tableId = Tables.getTableId(instance, tableName);
-
-    List<Text> splits = new ArrayList<Text>(partitionKeys);
-    // should be sorted because we copied from a sorted set, but that makes assumptions about
-    // how the copy was done so resort to be sure.
-    Collections.sort(splits);
-
-    CountDownLatch latch = new CountDownLatch(splits.size());
-    AtomicReference<Exception> exception = new AtomicReference<Exception>(null);
-
-    ExecutorService executor = Executors.newFixedThreadPool(16, new NamingThreadFactory("addSplits"));
-    try {
-      executor.submit(new SplitTask(new SplitEnv(tableName, tableId, executor, latch, exception), splits));
-
-      while (!latch.await(100, TimeUnit.MILLISECONDS)) {
-        if (exception.get() != null) {
-          executor.shutdownNow();
-          Exception excep = exception.get();
-          if (excep instanceof TableNotFoundException)
-            throw (TableNotFoundException) excep;
-          else if (excep instanceof AccumuloException)
-            throw (AccumuloException) excep;
-          else if (excep instanceof AccumuloSecurityException)
-            throw (AccumuloSecurityException) excep;
-          else if (excep instanceof RuntimeException)
-            throw (RuntimeException) excep;
-          else
-            throw new RuntimeException(excep);
-        }
-      }
-    } catch (InterruptedException e) {
-      throw new RuntimeException(e);
-    } finally {
-      executor.shutdown();
-    }
-  }
-
-  private void addSplits(String tableName, SortedSet<Text> partitionKeys, String tableId) throws AccumuloException, AccumuloSecurityException,
-      TableNotFoundException, AccumuloServerException {
-    TabletLocator tabLocator = TabletLocator.getLocator(instance, new Text(tableId));
-
-    for (Text split : partitionKeys) {
-      boolean successful = false;
-      int attempt = 0;
-
-      while (!successful) {
-
-        if (attempt > 0)
-          UtilWaitThread.sleep(100);
-
-        attempt++;
-
-        TabletLocation tl = tabLocator.locateTablet(credentials, split, false, false);
-
-        if (tl == null) {
-          if (!Tables.exists(instance, tableId))
-            throw new TableNotFoundException(tableId, tableName, null);
-          else if (Tables.getTableState(instance, tableId) == TableState.OFFLINE)
-            throw new TableOfflineException(instance, tableId);
-          continue;
-        }
-
-        try {
-          TabletClientService.Client client = ThriftUtil.getTServerClient(tl.tablet_location, ServerConfigurationUtil.getConfiguration(instance));
-          try {
-            OpTimer opTimer = null;
-            if (log.isTraceEnabled())
-              opTimer = new OpTimer(log, Level.TRACE).start("Splitting tablet " + tl.tablet_extent + " on " + tl.tablet_location + " at " + split);
-
-            client.splitTablet(Tracer.traceInfo(), credentials.toThrift(instance), tl.tablet_extent.toThrift(), TextUtil.getByteBuffer(split));
-
-            // just split it, might as well invalidate it in the cache
-            tabLocator.invalidateCache(tl.tablet_extent);
-
-            if (opTimer != null)
-              opTimer.stop("Split tablet in %DURATION%");
-          } finally {
-            ThriftUtil.returnClient(client);
-          }
-
-        } catch (TApplicationException tae) {
-          throw new AccumuloServerException(tl.tablet_location, tae);
-        } catch (TTransportException e) {
-          tabLocator.invalidateCache(tl.tablet_location);
-          continue;
-        } catch (ThriftSecurityException e) {
-          Tables.clearCache(instance);
-          if (!Tables.exists(instance, tableId))
-            throw new TableNotFoundException(tableId, tableName, null);
-          throw new AccumuloSecurityException(e.user, e.code, e);
-        } catch (NotServingTabletException e) {
-          tabLocator.invalidateCache(tl.tablet_extent);
-          continue;
-        } catch (TException e) {
-          tabLocator.invalidateCache(tl.tablet_location);
-          continue;
-        }
-
-        successful = true;
-      }
-    }
-  }
-
-  @Override
-  public void merge(String tableName, Text start, Text end) throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
-
-    ArgumentChecker.notNull(tableName);
-    ByteBuffer EMPTY = ByteBuffer.allocate(0);
-    List<ByteBuffer> args = Arrays.asList(ByteBuffer.wrap(tableName.getBytes(Constants.UTF8)), start == null ? EMPTY : TextUtil.getByteBuffer(start),
-        end == null ? EMPTY : TextUtil.getByteBuffer(end));
-    Map<String,String> opts = new HashMap<String,String>();
-    try {
-      doTableFateOperation(tableName, TableNotFoundException.class, FateOperation.TABLE_MERGE, args, opts);
-    } catch (TableExistsException e) {
-      // should not happen
-      throw new AssertionError(e);
-    }
-  }
-
-  @Override
-  public void deleteRows(String tableName, Text start, Text end) throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
-
-    ArgumentChecker.notNull(tableName);
-    ByteBuffer EMPTY = ByteBuffer.allocate(0);
-    List<ByteBuffer> args = Arrays.asList(ByteBuffer.wrap(tableName.getBytes(Constants.UTF8)), start == null ? EMPTY : TextUtil.getByteBuffer(start),
-        end == null ? EMPTY : TextUtil.getByteBuffer(end));
-    Map<String,String> opts = new HashMap<String,String>();
-    try {
-      doTableFateOperation(tableName, TableNotFoundException.class, FateOperation.TABLE_DELETE_RANGE, args, opts);
-    } catch (TableExistsException e) {
-      // should not happen
-      throw new AssertionError(e);
-    }
-  }
-
-  /**
-   * @param tableName
-   *          the name of the table
-   * @return the split points (end-row names) for the table's current split profile
-   */
-  @Override
-  public Collection<Text> listSplits(String tableName) throws TableNotFoundException, AccumuloSecurityException {
-
-    ArgumentChecker.notNull(tableName);
-
-    String tableId = Tables.getTableId(instance, tableName);
-
-    TreeMap<KeyExtent,String> tabletLocations = new TreeMap<KeyExtent,String>();
-
-    while (true) {
-      try {
-        tabletLocations.clear();
-        // the following method throws AccumuloException for some conditions that should be retried
-        MetadataServicer.forTableId(instance, credentials, tableId).getTabletLocations(tabletLocations);
-        break;
-      } catch (AccumuloSecurityException ase) {
-        throw ase;
-      } catch (Exception e) {
-        if (!Tables.exists(instance, tableId)) {
-          throw new TableNotFoundException(tableId, tableName, null);
-        }
-
-        if (e instanceof RuntimeException && e.getCause() instanceof AccumuloSecurityException) {
-          throw (AccumuloSecurityException) e.getCause();
-        }
-
-        log.info(e.getMessage() + " ... retrying ...");
-        UtilWaitThread.sleep(3000);
-      }
-    }
-
-    ArrayList<Text> endRows = new ArrayList<Text>(tabletLocations.size());
-
-    for (KeyExtent ke : tabletLocations.keySet())
-      if (ke.getEndRow() != null)
-        endRows.add(ke.getEndRow());
-
-    return endRows;
-  }
-
-  @Deprecated
-  @Override
-  public Collection<Text> getSplits(String tableName) throws TableNotFoundException {
-    try {
-      return listSplits(tableName);
-    } catch (AccumuloSecurityException e) {
-      throw new RuntimeException(e);
-    }
-  }
-
-  /**
-   * @param tableName
-   *          the name of the table
-   * @param maxSplits
-   *          specifies the maximum number of splits to return
-   * @return the split points (end-row names) for the table's current split profile, grouped into fewer splits so as not to exceed maxSplits
-   */
-  @Override
-  public Collection<Text> listSplits(String tableName, int maxSplits) throws TableNotFoundException, AccumuloSecurityException {
-    Collection<Text> endRows = listSplits(tableName);
-
-    if (endRows.size() <= maxSplits)
-      return endRows;
-
-    double r = (maxSplits + 1) / (double) (endRows.size());
-    double pos = 0;
-
-    ArrayList<Text> subset = new ArrayList<Text>(maxSplits);
-
-    int j = 0;
-    for (int i = 0; i < endRows.size() && j < maxSplits; i++) {
-      pos += r;
-      while (pos > 1) {
-        subset.add(((ArrayList<Text>) endRows).get(i));
-        j++;
-        pos -= 1;
-      }
-    }
-
-    return subset;
-  }
-
-  @Deprecated
-  @Override
-  public Collection<Text> getSplits(String tableName, int maxSplits) throws TableNotFoundException {
-    try {
-      return listSplits(tableName, maxSplits);
-    } catch (AccumuloSecurityException e) {
-      throw new RuntimeException(e);
-    }
-  }
-
-  /**
-   * Delete a table
-   * 
-   * @param tableName
-   *          the name of the table
-   * @throws AccumuloException
-   *           if a general error occurs
-   * @throws AccumuloSecurityException
-   *           if the user does not have permission
-   * @throws TableNotFoundException
-   *           if the table does not exist
-   */
-  @Override
-  public void delete(String tableName) throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
-    ArgumentChecker.notNull(tableName);
-
-    List<ByteBuffer> args = Arrays.asList(ByteBuffer.wrap(tableName.getBytes(Constants.UTF8)));
-    Map<String,String> opts = new HashMap<String,String>();
-
-    try {
-      doTableFateOperation(tableName, TableNotFoundException.class, FateOperation.TABLE_DELETE, args, opts);
-    } catch (TableExistsException e) {
-      // should not happen
-      throw new AssertionError(e);
-    }
-
-  }
-
-  @Override
-  public void clone(String srcTableName, String newTableName, boolean flush, Map<String,String> propertiesToSet, Set<String> propertiesToExclude)
-      throws AccumuloSecurityException, TableNotFoundException, AccumuloException, TableExistsException {
-
-    ArgumentChecker.notNull(srcTableName, newTableName);
-
-    String srcTableId = Tables.getTableId(instance, srcTableName);
-
-    if (flush)
-      _flush(srcTableId, null, null, true);
-
-    if (propertiesToExclude == null)
-      propertiesToExclude = Collections.emptySet();
-
-    if (propertiesToSet == null)
-      propertiesToSet = Collections.emptyMap();
-
-    List<ByteBuffer> args = Arrays.asList(ByteBuffer.wrap(srcTableId.getBytes(Constants.UTF8)), ByteBuffer.wrap(newTableName.getBytes(Constants.UTF8)));
-    Map<String,String> opts = new HashMap<String,String>();
-    for (Entry<String,String> entry : propertiesToSet.entrySet()) {
-      if (entry.getKey().startsWith(CLONE_EXCLUDE_PREFIX))
-        throw new IllegalArgumentException("Property can not start with " + CLONE_EXCLUDE_PREFIX);
-      opts.put(entry.getKey(), entry.getValue());
-    }
-
-    for (String prop : propertiesToExclude) {
-      opts.put(CLONE_EXCLUDE_PREFIX + prop, "");
-    }
-
-    doTableFateOperation(newTableName, AccumuloException.class, FateOperation.TABLE_CLONE, args, opts);
-  }
-
-  /**
-   * Rename a table
-   * 
-   * @param oldTableName
-   *          the old table name
-   * @param newTableName
-   *          the new table name
-   * @throws AccumuloException
-   *           if a general error occurs
-   * @throws AccumuloSecurityException
-   *           if the user does not have permission
-   * @throws TableNotFoundException
-   *           if the old table name does not exist
-   * @throws TableExistsException
-   *           if the new table name already exists
-   */
-  @Override
-  public void rename(String oldTableName, String newTableName) throws AccumuloSecurityException, TableNotFoundException, AccumuloException,
-      TableExistsException {
-
-    List<ByteBuffer> args = Arrays.asList(ByteBuffer.wrap(oldTableName.getBytes(Constants.UTF8)), ByteBuffer.wrap(newTableName.getBytes(Constants.UTF8)));
-    Map<String,String> opts = new HashMap<String,String>();
-    doTableFateOperation(oldTableName, TableNotFoundException.class, FateOperation.TABLE_RENAME, args, opts);
-  }
-
-  /**
-   * @deprecated since 1.4 {@link #flush(String, Text, Text, boolean)}
-   */
-  @Override
-  @Deprecated
-  public void flush(String tableName) throws AccumuloException, AccumuloSecurityException {
-    try {
-      flush(tableName, null, null, false);
-    } catch (TableNotFoundException e) {
-      throw new AccumuloException(e.getMessage(), e);
-    }
-  }
-
-  /**
-   * Flush a table
-   * 
-   * @param tableName
-   *          the name of the table
-   * @throws AccumuloException
-   *           if a general error occurs
-   * @throws AccumuloSecurityException
-   *           if the user does not have permission
-   */
-  @Override
-  public void flush(String tableName, Text start, Text end, boolean wait) throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
-    ArgumentChecker.notNull(tableName);
-
-    String tableId = Tables.getTableId(instance, tableName);
-    _flush(tableId, start, end, wait);
-  }
-
-  @Override
-  public void compact(String tableName, Text start, Text end, boolean flush, boolean wait) throws AccumuloSecurityException, TableNotFoundException,
-      AccumuloException {
-    compact(tableName, start, end, new ArrayList<IteratorSetting>(), flush, wait);
-  }
-
-  @Override
-  public void compact(String tableName, Text start, Text end, List<IteratorSetting> iterators, boolean flush, boolean wait) throws AccumuloSecurityException,
-      TableNotFoundException, AccumuloException {
-    ArgumentChecker.notNull(tableName);
-    ByteBuffer EMPTY = ByteBuffer.allocate(0);
-
-    String tableId = Tables.getTableId(instance, tableName);
-
-    if (flush)
-      _flush(tableId, start, end, true);
-
-    List<ByteBuffer> args = Arrays.asList(ByteBuffer.wrap(tableId.getBytes(Constants.UTF8)), start == null ? EMPTY : TextUtil.getByteBuffer(start),
-        end == null ? EMPTY : TextUtil.getByteBuffer(end), ByteBuffer.wrap(IteratorUtil.encodeIteratorSettings(iterators)));
-
-    Map<String,String> opts = new HashMap<String,String>();
-    try {
-      doFateOperation(FateOperation.TABLE_COMPACT, args, opts, wait);
-    } catch (TableExistsException e) {
-      // should not happen
-      throw new AssertionError(e);
-    } catch (NamespaceExistsException e) {
-      // should not happen
-      throw new AssertionError(e);
-    } catch (NamespaceNotFoundException e) {
-      throw new TableNotFoundException(null, tableName, "Namespace not found", e);
-    }
-  }
-
-  @Override
-  public void cancelCompaction(String tableName) throws AccumuloSecurityException, TableNotFoundException, AccumuloException {
-    String tableId = Tables.getTableId(instance, tableName);
-
-    List<ByteBuffer> args = Arrays.asList(ByteBuffer.wrap(tableId.getBytes(Constants.UTF8)));
-
-    Map<String,String> opts = new HashMap<String,String>();
-    try {
-      doTableFateOperation(tableName, TableNotFoundException.class, FateOperation.TABLE_CANCEL_COMPACT, args, opts);
-    } catch (TableExistsException e) {
-      // should not happen
-      throw new AssertionError(e);
-    }
-
-  }
-
-  private void _flush(String tableId, Text start, Text end, boolean wait) throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
-
-    try {
-      long flushID;
-
-      // used to pass the table name. but the tableid associated with a table name could change between calls.
-      // so pass the tableid to both calls
-
-      while (true) {
-        MasterClientService.Iface client = null;
-        try {
-          client = MasterClient.getConnectionWithRetry(instance);
-          flushID = client.initiateFlush(Tracer.traceInfo(), credentials.toThrift(instance), tableId);
-          break;
-        } catch (TTransportException tte) {
-          log.debug("Failed to call initiateFlush, retrying ... ", tte);
-          UtilWaitThread.sleep(100);
-        } finally {
-          MasterClient.close(client);
-        }
-      }
-
-      while (true) {
-        MasterClientService.Iface client = null;
-        try {
-          client = MasterClient.getConnectionWithRetry(instance);
-          client.waitForFlush(Tracer.traceInfo(), credentials.toThrift(instance), tableId, TextUtil.getByteBuffer(start), TextUtil.getByteBuffer(end), flushID,
-              wait ? Long.MAX_VALUE : 1);
-          break;
-        } catch (TTransportException tte) {
-          log.debug("Failed to call initiateFlush, retrying ... ", tte);
-          UtilWaitThread.sleep(100);
-        } finally {
-          MasterClient.close(client);
-        }
-      }
-    } catch (ThriftSecurityException e) {
-      switch (e.getCode()) {
-        case TABLE_DOESNT_EXIST:
-          throw new TableNotFoundException(tableId, null, e.getMessage(), e);
-        default:
-          log.debug("flush security exception on table id " + tableId);
-          throw new AccumuloSecurityException(e.user, e.code, e);
-      }
-    } catch (ThriftTableOperationException e) {
-      switch (e.getType()) {
-        case NOTFOUND:
-          throw new TableNotFoundException(e);
-        default:
-          throw new AccumuloException(e.description, e);
-      }
-    } catch (Exception e) {
-      throw new AccumuloException(e);
-    }
-  }
-
-  /**
-   * Sets a property on a table
-   * 
-   * @param tableName
-   *          the name of the table
-   * @param property
-   *          the name of a per-table property
-   * @param value
-   *          the value to set a per-table property to
-   * @throws AccumuloException
-   *           if a general error occurs
-   * @throws AccumuloSecurityException
-   *           if the user does not have permission
-   */
-  @Override
-  public void setProperty(final String tableName, final String property, final String value) throws AccumuloException, AccumuloSecurityException {
-    ArgumentChecker.notNull(tableName, property, value);
-    try {
-      MasterClient.executeTable(instance, new ClientExec<MasterClientService.Client>() {
-        @Override
-        public void execute(MasterClientService.Client client) throws Exception {
-          client.setTableProperty(Tracer.traceInfo(), credentials.toThrift(instance), tableName, property, value);
-        }
-      });
-    } catch (TableNotFoundException e) {
-      throw new AccumuloException(e);
-    }
-  }
-
-  /**
-   * Removes a property from a table
-   * 
-   * @param tableName
-   *          the name of the table
-   * @param property
-   *          the name of a per-table property
-   * @throws AccumuloException
-   *           if a general error occurs
-   * @throws AccumuloSecurityException
-   *           if the user does not have permission
-   */
-  @Override
-  public void removeProperty(final String tableName, final String property) throws AccumuloException, AccumuloSecurityException {
-    ArgumentChecker.notNull(tableName, property);
-    try {
-      MasterClient.executeTable(instance, new ClientExec<MasterClientService.Client>() {
-        @Override
-        public void execute(MasterClientService.Client client) throws Exception {
-          client.removeTableProperty(Tracer.traceInfo(), credentials.toThrift(instance), tableName, property);
-        }
-      });
-    } catch (TableNotFoundException e) {
-      throw new AccumuloException(e);
-    }
-  }
-
-  /**
-   * Gets properties of a table
-   * 
-   * @param tableName
-   *          the name of the table
-   * @return all properties visible by this table (system and per-table properties)
-   * @throws TableNotFoundException
-   *           if the table does not exist
-   */
-  @Override
-  public Iterable<Entry<String,String>> getProperties(final String tableName) throws AccumuloException, TableNotFoundException {
-    ArgumentChecker.notNull(tableName);
-    try {
-      return ServerClient.executeRaw(instance, new ClientExecReturn<Map<String,String>,ClientService.Client>() {
-        @Override
-        public Map<String,String> execute(ClientService.Client client) throws Exception {
-          return client.getTableConfiguration(Tracer.traceInfo(), credentials.toThrift(instance), tableName);
-        }
-      }).entrySet();
-    } catch (ThriftTableOperationException e) {
-      switch (e.getType()) {
-        case NOTFOUND:
-          throw new TableNotFoundException(e);
-        case NAMESPACE_NOTFOUND:
-          throw new TableNotFoundException(tableName, new NamespaceNotFoundException(e));
-        default:
-          throw new AccumuloException(e.description, e);
-      }
-    } catch (AccumuloException e) {
-      throw e;
-    } catch (Exception e) {
-      throw new AccumuloException(e);
-    }
-
-  }
-
-  /**
-   * Sets a tables locality groups. A tables locality groups can be changed at any time.
-   * 
-   * @param tableName
-   *          the name of the table
-   * @param groups
-   *          mapping of locality group names to column families in the locality group
-   * @throws AccumuloException
-   *           if a general error occurs
-   * @throws AccumuloSecurityException
-   *           if the user does not have permission
-   * @throws TableNotFoundException
-   *           if the table does not exist
-   */
-  @Override
-  public void setLocalityGroups(String tableName, Map<String,Set<Text>> groups) throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
-    // ensure locality groups do not overlap
-    HashSet<Text> all = new HashSet<Text>();
-    for (Entry<String,Set<Text>> entry : groups.entrySet()) {
-
-      if (!Collections.disjoint(all, entry.getValue())) {
-        throw new IllegalArgumentException("Group " + entry.getKey() + " overlaps with another group");
-      }
-
-      all.addAll(entry.getValue());
-    }
-
-    for (Entry<String,Set<Text>> entry : groups.entrySet()) {
-      Set<Text> colFams = entry.getValue();
-      String value = LocalityGroupUtil.encodeColumnFamilies(colFams);
-      setProperty(tableName, Property.TABLE_LOCALITY_GROUP_PREFIX + entry.getKey(), value);
-    }
-
-    try {
-      setProperty(tableName, Property.TABLE_LOCALITY_GROUPS.getKey(), StringUtil.join(groups.keySet(), ","));
-    } catch (AccumuloException e) {
-      if (e.getCause() instanceof TableNotFoundException)
-        throw (TableNotFoundException) e.getCause();
-      throw e;
-    }
-
-    // remove anything extraneous
-    String prefix = Property.TABLE_LOCALITY_GROUP_PREFIX.getKey();
-    for (Entry<String,String> entry : getProperties(tableName)) {
-      String property = entry.getKey();
-      if (property.startsWith(prefix)) {
-        // this property configures a locality group, find out which
-        // one:
-        String[] parts = property.split("\\.");
-        String group = parts[parts.length - 1];
-
-        if (!groups.containsKey(group)) {
-          removeProperty(tableName, property);
-        }
-      }
-    }
-  }
-
-  /**
-   * 
-   * Gets the locality groups currently set for a table.
-   * 
-   * @param tableName
-   *          the name of the table
-   * @return mapping of locality group names to column families in the locality group
-   * @throws AccumuloException
-   *           if a general error occurs
-   * @throws TableNotFoundException
-   *           if the table does not exist
-   */
-  @Override
-  public Map<String,Set<Text>> getLocalityGroups(String tableName) throws AccumuloException, TableNotFoundException {
-    AccumuloConfiguration conf = new ConfigurationCopy(this.getProperties(tableName));
-    Map<String,Set<ByteSequence>> groups = LocalityGroupUtil.getLocalityGroups(conf);
-
-    Map<String,Set<Text>> groups2 = new HashMap<String,Set<Text>>();
-    for (Entry<String,Set<ByteSequence>> entry : groups.entrySet()) {
-
-      HashSet<Text> colFams = new HashSet<Text>();
-
-      for (ByteSequence bs : entry.getValue()) {
-        colFams.add(new Text(bs.toArray()));
-      }
-
-      groups2.put(entry.getKey(), colFams);
-    }
-
-    return groups2;
-  }
-
-  /**
-   * @param tableName
-   *          the name of the table
-   * @param range
-   *          a range to split
-   * @param maxSplits
-   *          the maximum number of splits
-   * @return the range, split into smaller ranges that fall on boundaries of the table's split points as evenly as possible
-   * @throws AccumuloException
-   *           if a general error occurs
-   * @throws AccumuloSecurityException
-   *           if the user does not have permission
-   * @throws TableNotFoundException
-   *           if the table does not exist
-   */
-  @Override
-  public Set<Range> splitRangeByTablets(String tableName, Range range, int maxSplits) throws AccumuloException, AccumuloSecurityException,
-      TableNotFoundException {
-    ArgumentChecker.notNull(tableName, range);
-    if (maxSplits < 1)
-      throw new IllegalArgumentException("maximum splits must be >= 1");
-    if (maxSplits == 1)
-      return Collections.singleton(range);
-
-    Random random = new Random();
-    Map<String,Map<KeyExtent,List<Range>>> binnedRanges = new HashMap<String,Map<KeyExtent,List<Range>>>();
-    String tableId = Tables.getTableId(instance, tableName);
-    TabletLocator tl = TabletLocator.getLocator(instance, new Text(tableId));
-    // its possible that the cache could contain complete, but old information about a tables tablets... so clear it
-    tl.invalidateCache();
-    while (!tl.binRanges(credentials, Collections.singletonList(range), binnedRanges).isEmpty()) {
-      if (!Tables.exists(instance, tableId))
-        throw new TableDeletedException(tableId);
-      if (Tables.getTableState(instance, tableId) == TableState.OFFLINE)
-        throw new TableOfflineException(instance, tableId);
-
-      log.warn("Unable to locate bins for specified range. Retrying.");
-      // sleep randomly between 100 and 200ms
-      UtilWaitThread.sleep(100 + random.nextInt(100));
-      binnedRanges.clear();
-      tl.invalidateCache();
-    }
-
-    // group key extents to get <= maxSplits
-    LinkedList<KeyExtent> unmergedExtents = new LinkedList<KeyExtent>();
-    List<KeyExtent> mergedExtents = new ArrayList<KeyExtent>();
-
-    for (Map<KeyExtent,List<Range>> map : binnedRanges.values())
-      unmergedExtents.addAll(map.keySet());
-
-    // the sort method is efficient for linked list
-    Collections.sort(unmergedExtents);
-
-    while (unmergedExtents.size() + mergedExtents.size() > maxSplits) {
-      if (unmergedExtents.size() >= 2) {
-        KeyExtent first = unmergedExtents.removeFirst();
-        KeyExtent second = unmergedExtents.removeFirst();
-        first.setEndRow(second.getEndRow());
-        mergedExtents.add(first);
-      } else {
-        mergedExtents.addAll(unmergedExtents);
-        unmergedExtents.clear();
-        unmergedExtents.addAll(mergedExtents);
-        mergedExtents.clear();
-      }
-
-    }
-
-    mergedExtents.addAll(unmergedExtents);
-
-    Set<Range> ranges = new HashSet<Range>();
-    for (KeyExtent k : mergedExtents)
-      ranges.add(k.toDataRange().clip(range));
-
-    return ranges;
-  }
-
-  // TODO Remove deprecation warning surppression when Hadoop1 support is dropped
-  @SuppressWarnings("deprecation")
-  private Path checkPath(String dir, String kind, String type) throws IOException, AccumuloException {
-    Path ret;
-    FileSystem fs = VolumeConfiguration.getVolume(dir, CachedConfiguration.getInstance(), ServerConfigurationUtil.getConfiguration(instance)).getFileSystem();
-
-    if (dir.contains(":")) {
-      ret = new Path(dir);
-    } else {
-      ret = fs.makeQualified(new Path(dir));
-    }
-
-    if (!fs.exists(ret))
-      throw new AccumuloException(kind + " import " + type + " directory " + dir + " does not exist!");
-
-    if (!fs.getFileStatus(ret).isDir()) {
-      throw new AccumuloException(kind + " import " + type + " directory " + dir + " is not a directory!");
-    }
-
-    if (type.equals("failure")) {
-      FileStatus[] listStatus = fs.listStatus(ret);
-      if (listStatus != null && listStatus.length != 0) {
-        throw new AccumuloException("Bulk import failure directory " + ret + " is not empty");
-      }
-    }
-
-    return ret;
-  }
-
-  @Override
-  public void importDirectory(String tableName, String dir, String failureDir, boolean setTime) throws IOException, AccumuloSecurityException,
-      TableNotFoundException, AccumuloException {
-    ArgumentChecker.notNull(tableName, dir, failureDir);
-    // check for table existance
-    Tables.getTableId(instance, tableName);
-
-    Path dirPath = checkPath(dir, "Bulk", "");
-    Path failPath = checkPath(failureDir, "Bulk", "failure");
-
-    List<ByteBuffer> args = Arrays.asList(ByteBuffer.wrap(tableName.getBytes(Constants.UTF8)), ByteBuffer.wrap(dirPath.toString().getBytes(Constants.UTF8)),
-        ByteBuffer.wrap(failPath.toString().getBytes(Constants.UTF8)), ByteBuffer.wrap((setTime + "").getBytes(Constants.UTF8)));
-    Map<String,String> opts = new HashMap<String,String>();
-
-    try {
-      doTableFateOperation(tableName, TableNotFoundException.class, FateOperation.TABLE_BULK_IMPORT, args, opts);
-    } catch (TableExistsException e) {
-      // should not happen
-      throw new AssertionError(e);
-    }
-  }
-
-  private void waitForTableStateTransition(String tableId, TableState expectedState) throws AccumuloException, TableNotFoundException,
-      AccumuloSecurityException {
-
-    Text startRow = null;
-    Text lastRow = null;
-
-    while (true) {
-
-      if (Tables.getTableState(instance, tableId) != expectedState) {
-        Tables.clearCache(instance);
-        if (Tables.getTableState(instance, tableId) != expectedState) {
-          if (!Tables.exists(instance, tableId))
-            throw new TableDeletedException(tableId);
-          throw new AccumuloException("Unexpected table state " + tableId + " " + Tables.getTableState(instance, tableId) + " != " + expectedState);
-        }
-      }
-
-      Range range = new KeyExtent(new Text(tableId), null, null).toMetadataRange();
-      if (startRow == null || lastRow == null)
-        range = new KeyExtent(new Text(tableId), null, null).toMetadataRange();
-      else
-        range = new Range(startRow, lastRow);
-
-      String metaTable = MetadataTable.NAME;
-      if (tableId.equals(MetadataTable.ID))
-        metaTable = RootTable.NAME;
-      Scanner scanner = instance.getConnector(credentials.getPrincipal(), credentials.getToken()).createScanner(metaTable, Authorizations.EMPTY);
-      scanner = new IsolatedScanner(scanner);
-      TabletsSection.TabletColumnFamily.PREV_ROW_COLUMN.fetch(scanner);
-      scanner.fetchColumnFamily(TabletsSection.CurrentLocationColumnFamily.NAME);
-      scanner.setRange(range);
-
-      RowIterator rowIter = new RowIterator(scanner);
-
-      KeyExtent lastExtent = null;
-
-      int total = 0;
-      int waitFor = 0;
-      int holes = 0;
-      Text continueRow = null;
-      MapCounter<String> serverCounts = new MapCounter<String>();
-
-      while (rowIter.hasNext()) {
-        Iterator<Entry<Key,Value>> row = rowIter.next();
-
-        total++;
-
-        KeyExtent extent = null;
-        String future = null;
-        String current = null;
-
-        while (row.hasNext()) {
-          Entry<Key,Value> entry = row.next();
-          Key key = entry.getKey();
-
-          if (key.getColumnFamily().equals(TabletsSection.FutureLocationColumnFamily.NAME))
-            future = entry.getValue().toString();
-
-          if (key.getColumnFamily().equals(TabletsSection.CurrentLocationColumnFamily.NAME))
-            current = entry.getValue().toString();
-
-          if (TabletsSection.TabletColumnFamily.PREV_ROW_COLUMN.hasColumns(key))
-            extent = new KeyExtent(key.getRow(), entry.getValue());
-        }
-
-        if ((expectedState == TableState.ONLINE && current == null) || (expectedState == TableState.OFFLINE && (future != null || current != null))) {
-          if (continueRow == null)
-            continueRow = extent.getMetadataEntry();
-          waitFor++;
-          lastRow = extent.getMetadataEntry();
-
-          if (current != null)
-            serverCounts.increment(current, 1);
-          if (future != null)
-            serverCounts.increment(future, 1);
-        }
-
-        if (!extent.getTableId().toString().equals(tableId)) {
-          throw new AccumuloException("Saw unexpected table Id " + tableId + " " + extent);
-        }
-
-        if (lastExtent != null && !extent.isPreviousExtent(lastExtent)) {
-          holes++;
-        }
-
-        lastExtent = extent;
-      }
-
-      if (continueRow != null) {
-        startRow = continueRow;
-      }
-
-      if (holes > 0 || total == 0) {
-        startRow = null;
-        lastRow = null;
-      }
-
-      if (waitFor > 0 || holes > 0 || total == 0) {
-        long waitTime;
-        long maxPerServer = 0;
-        if (serverCounts.size() > 0) {
-          maxPerServer = Collections.max(serverCounts.values());
-          waitTime = maxPerServer * 10;
-        } else
-          waitTime = waitFor * 10;
-        waitTime = Math.max(100, waitTime);
-        waitTime = Math.min(5000, waitTime);
-        log.trace("Waiting for " + waitFor + "(" + maxPerServer + ") tablets, startRow = " + startRow + " lastRow = " + lastRow + ", holes=" + holes
-            + " sleeping:" + waitTime + "ms");
-        UtilWaitThread.sleep(waitTime);
-      } else {
-        break;
-      }
-
-    }
-  }
-
-  @Override
-  public void offline(String tableName) throws AccumuloSecurityException, AccumuloException, TableNotFoundException {
-    offline(tableName, false);
-  }
-
-  /**
-   * 
-   * @param tableName
-   *          the table to take offline
-   * @throws AccumuloException
-   *           when there is a general accumulo error
-   * @throws AccumuloSecurityException
-   *           when the user does not have the proper permissions
-   */
-  @Override
-  public void offline(String tableName, boolean wait) throws AccumuloSecurityException, AccumuloException, TableNotFoundException {
-
-    ArgumentChecker.notNull(tableName);
-    String tableId = Tables.getTableId(instance, tableName);
-    List<ByteBuffer> args = Arrays.asList(ByteBuffer.wrap(tableId.getBytes(Constants.UTF8)));
-    Map<String,String> opts = new HashMap<String,String>();
-
-    try {
-      doTableFateOperation(tableName, TableNotFoundException.class, FateOperation.TABLE_OFFLINE, args, opts);
-    } catch (TableExistsException e) {
-      // should not happen
-      throw new AssertionError(e);
-    }
-
-    if (wait)
-      waitForTableStateTransition(tableId, TableState.OFFLINE);
-  }
-
-  @Override
-  public void online(String tableName) throws AccumuloSecurityException, AccumuloException, TableNotFoundException {
-    online(tableName, false);
-  }
-
-  /**
-   * 
-   * @param tableName
-   *          the table to take online
-   * @throws AccumuloException
-   *           when there is a general accumulo error
-   * @throws AccumuloSecurityException
-   *           when the user does not have the proper permissions
-   */
-  @Override
-  public void online(String tableName, boolean wait) throws AccumuloSecurityException, AccumuloException, TableNotFoundException {
-    ArgumentChecker.notNull(tableName);
-    String tableId = Tables.getTableId(instance, tableName);
-    List<ByteBuffer> args = Arrays.asList(ByteBuffer.wrap(tableId.getBytes(Constants.UTF8)));
-    Map<String,String> opts = new HashMap<String,String>();
-
-    try {
-      doTableFateOperation(tableName, TableNotFoundException.class, FateOperation.TABLE_ONLINE, args, opts);
-    } catch (TableExistsException e) {
-      // should not happen
-      throw new AssertionError(e);
-    }
-
-    if (wait)
-      waitForTableStateTransition(tableId, TableState.ONLINE);
-  }
-
-  /**
-   * Clears the tablet locator cache for a specified table
-   * 
-   * @param tableName
-   *          the name of the table
-   * @throws TableNotFoundException
-   *           if table does not exist
-   */
-  @Override
-  public void clearLocatorCache(String tableName) throws TableNotFoundException {
-    ArgumentChecker.notNull(tableName);
-    TabletLocator tabLocator = TabletLocator.getLocator(instance, new Text(Tables.getTableId(instance, tableName)));
-    tabLocator.invalidateCache();
-  }
-
-  /**
-   * Get a mapping of table name to internal table id.
-   * 
-   * @return the map from table name to internal table id
+   * @param instance
+   *          the connection information for this instance
+   * @param credentials
+   *          the username/password for this connection
    */
-  @Override
-  public Map<String,String> tableIdMap() {
-    return Tables.getNameToIdMap(instance);
-  }
-
-  @Override
-  public Text getMaxRow(String tableName, Authorizations auths, Text startRow, boolean startInclusive, Text endRow, boolean endInclusive)
-      throws TableNotFoundException, AccumuloException, AccumuloSecurityException {
-    ArgumentChecker.notNull(tableName, auths);
-    Scanner scanner = instance.getConnector(credentials.getPrincipal(), credentials.getToken()).createScanner(tableName, auths);
-    return FindMax.findMax(scanner, startRow, startInclusive, endRow, endInclusive);
-  }
-
-  @Override
-  public List<DiskUsage> getDiskUsage(Set<String> tableNames) throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
-
-    List<TDiskUsage> diskUsages = null;
-    while (diskUsages == null) {
-      Pair<String,Client> pair = null;
-      try {
-        // this operation may us a lot of memory... its likely that connections to tabletservers hosting metadata tablets will be cached, so do not use cached
-        // connections
-        pair = ServerClient.getConnection(instance, false);
-        diskUsages = pair.getSecond().getDiskUsage(tableNames, credentials.toThrift(instance));
-      } catch (ThriftTableOperationException e) {
-        switch (e.getType()) {
-          case NOTFOUND:
-            throw new TableNotFoundException(e);
-          case NAMESPACE_NOTFOUND:
-            throw new TableNotFoundException(e.getTableName(), new NamespaceNotFoundException(e));
-          default:
-            throw new AccumuloException(e.description, e);
-        }
-      } catch (ThriftSecurityException e) {
-        throw new AccumuloSecurityException(e.getUser(), e.getCode());
-      } catch (TTransportException e) {
-        // some sort of communication error occurred, retry
-        log.debug("disk usage request failed " + pair.getFirst() + ", retrying ... ", e);
-        UtilWaitThread.sleep(100);
-      } catch (TException e) {
-        // may be a TApplicationException which indicates error on the server side
-        throw new AccumuloException(e);
-      } finally {
-        // must always return thrift connection
-        if (pair != null)
-          ServerClient.close(pair.getSecond());
-      }
-    }
-
-    List<DiskUsage> finalUsages = new ArrayList<DiskUsage>();
-    for (TDiskUsage diskUsage : diskUsages) {
-      finalUsages.add(new DiskUsage(new TreeSet<String>(diskUsage.getTables()), diskUsage.getUsage()));
-    }
-
-    return finalUsages;
-  }
-
-  public static Map<String,String> getExportedProps(FileSystem fs, Path path) throws IOException {
-    HashMap<String,String> props = new HashMap<String,String>();
-
-    ZipInputStream zis = new ZipInputStream(fs.open(path));
-    try {
-      ZipEntry zipEntry;
-      while ((zipEntry = zis.getNextEntry()) != null) {
-        if (zipEntry.getName().equals(Constants.EXPORT_TABLE_CONFIG_FILE)) {
-          BufferedReader in = new BufferedReader(new InputStreamReader(zis, Constants.UTF8));
-          try {
-            String line;
-            while ((line = in.readLine()) != null) {
-              String sa[] = line.split("=", 2);
-              props.put(sa[0], sa[1]);
-            }
-          } finally {
-            in.close();
-          }
-
-          break;
-        }
-      }
-    } finally {
-      zis.close();
-    }
-    return props;
-  }
-
-  @Override
-  public void importTable(String tableName, String importDir) throws TableExistsException, AccumuloException, AccumuloSecurityException {
-    ArgumentChecker.notNull(tableName, importDir);
-
-    try {
-      importDir = checkPath(importDir, "Table", "").toString();
-    } catch (IOException e) {
-      throw new AccumuloException(e);
-    }
-
-    try {
-      FileSystem fs = new Path(importDir).getFileSystem(CachedConfiguration.getInstance());
-      Map<String,String> props = getExportedProps(fs, new Path(importDir, Constants.EXPORT_FILE));
-
-      for (Entry<String,String> entry : props.entrySet()) {
-        if (Property.isClassProperty(entry.getKey()) && !entry.getValue().contains(Constants.CORE_PACKAGE_NAME)) {
-          Logger.getLogger(this.getClass()).info(
-              "Imported table sets '" + entry.getKey() + "' to '" + entry.getValue() + "'.  Ensure this class is on Accumulo classpath.");
-        }
-      }
-
-    } catch (IOException ioe) {
-      Logger.getLogger(this.getClass()).warn("Failed to check if imported table references external java classes : " + ioe.getMessage());
-    }
-
-    List<ByteBuffer> args = Arrays.asList(ByteBuffer.wrap(tableName.getBytes(Constants.UTF8)), ByteBuffer.wrap(importDir.getBytes(Constants.UTF8)));
-
-    Map<String,String> opts = Collections.emptyMap();
-
-    try {
-      doTableFateOperation(tableName, AccumuloException.class, FateOperation.TABLE_IMPORT, args, opts);
-    } catch (TableNotFoundException e) {
-      // should not happen
-      throw new AssertionError(e);
-    }
-
-  }
-
-  @Override
-  public void exportTable(String tableName, String exportDir) throws TableNotFoundException, AccumuloException, AccumuloSecurityException {
-    ArgumentChecker.notNull(tableName, exportDir);
-
-    List<ByteBuffer> args = Arrays.asList(ByteBuffer.wrap(tableName.getBytes(Constants.UTF8)), ByteBuffer.wrap(exportDir.getBytes(Constants.UTF8)));
-
-    Map<String,String> opts = Collections.emptyMap();
-
-    try {
-      doTableFateOperation(tableName, TableNotFoundException.class, FateOperation.TABLE_EXPORT, args, opts);
-    } catch (TableExistsException e) {
-      // should not happen
-      throw new AssertionError(e);
-    }
+  public TableOperationsImpl(Instance instance, TCredentials credentials) {
+    this(instance, Credentials.fromThrift(credentials));
   }
-
-  @Override
-  public boolean testClassLoad(final String tableName, final String className, final String asTypeName) throws TableNotFoundException, AccumuloException,
-      AccumuloSecurityException {
-    ArgumentChecker.notNull(tableName, className, asTypeName);
-
-    try {
-      return ServerClient.executeRaw(instance, new ClientExecReturn<Boolean,ClientService.Client>() {
-        @Override
-        public Boolean execute(ClientService.Client client) throws Exception {
-          return client.checkTableClass(Tracer.traceInfo(), credentials.toThrift(instance), tableName, className, asTypeName);
-        }
-      });
-    } catch (ThriftTableOperationException e) {
-      switch (e.getType()) {
-        case NOTFOUND:
-          throw new TableNotFoundException(e);
-        case NAMESPACE_NOTFOUND:
-          throw new TableNotFoundException(tableName, new NamespaceNotFoundException(e));
-        default:
-          throw new AccumuloException(e.description, e);
-      }
-    } catch (ThriftSecurityException e) {
-      throw new AccumuloSecurityException(e.user, e.code, e);
-    } catch (AccumuloException e) {
-      throw e;
-    } catch (Exception e) {
-      throw new AccumuloException(e);
-    }
-  }
-
-  @Override
-  public void attachIterator(String tableName, IteratorSetting setting, EnumSet<IteratorScope> scopes) throws AccumuloSecurityException, AccumuloException,
-      TableNotFoundException {
-    testClassLoad(tableName, setting.getIteratorClass(), SortedKeyValueIterator.class.getName());
-    super.attachIterator(tableName, setting, scopes);
-  }
-
-  @Override
-  public int addConstraint(String tableName, String constraintClassName) throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
-    testClassLoad(tableName, constraintClassName, Constraint.class.getName());
-    return super.addConstraint(tableName, constraintClassName);
-  }
-
-  private void doTableFateOperation(String tableName, Class<? extends Exception> namespaceNotFoundExceptionClass, FateOperation op, List<ByteBuffer> args,
-      Map<String,String> opts) throws AccumuloSecurityException, AccumuloException, TableExistsException, TableNotFoundException {
-    try {
-      doFateOperation(op, args, opts);
-    } catch (NamespaceExistsException e) {
-      // should not happen
-      throw new AssertionError(e);
-    } catch (NamespaceNotFoundException e) {
-      if (namespaceNotFoundExceptionClass == null) {
-        // should not happen
-        throw new AssertionError(e);
-      } else if (AccumuloException.class.isAssignableFrom(namespaceNotFoundExceptionClass)) {
-        throw new AccumuloException("Cannot create table in non-existent namespace", e);
-      } else if (TableNotFoundException.class.isAssignableFrom(namespaceNotFoundExceptionClass)) {
-        throw new TableNotFoundException(null, tableName, "Namespace not found", e);
-      } else {
-        // should not happen
-        throw new AssertionError(e);
-      }
-    }
-  }
-
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/f4454a06/core/src/main/java/org/apache/accumulo/core/client/impl/ActiveCompactionImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/ActiveCompactionImpl.java b/core/src/main/java/org/apache/accumulo/core/client/impl/ActiveCompactionImpl.java
new file mode 100644
index 0000000..a413195
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/ActiveCompactionImpl.java
@@ -0,0 +1,109 @@
+/*
+ * 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.core.client.impl;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.accumulo.core.client.Instance;
+import org.apache.accumulo.core.client.IteratorSetting;
+import org.apache.accumulo.core.client.TableNotFoundException;
+import org.apache.accumulo.core.client.admin.ActiveCompaction;
+import org.apache.accumulo.core.data.KeyExtent;
+import org.apache.accumulo.core.data.thrift.IterInfo;
+
+
+/**
+ *
+ * @since 1.6.0
+ */
+public class ActiveCompactionImpl extends ActiveCompaction {
+
+  private org.apache.accumulo.core.tabletserver.thrift.ActiveCompaction tac;
+  private Instance instance;
+
+  ActiveCompactionImpl(Instance instance, org.apache.accumulo.core.tabletserver.thrift.ActiveCompaction tac) {
+    this.tac = tac;
+    this.instance = instance;
+  }
+
+  @Override
+  public String getTable() throws TableNotFoundException {
+    return Tables.getTableName(instance, getExtent().getTableId().toString());
+  }
+
+  @Override
+  public KeyExtent getExtent() {
+    return new KeyExtent(tac.getExtent());
+  }
+
+  @Override
+  public long getAge() {
+    return tac.getAge();
+  }
+
+  @Override
+  public List<String> getInputFiles() {
+    return tac.getInputFiles();
+  }
+
+  @Override
+  public String getOutputFile() {
+    return tac.getOutputFile();
+  }
+
+  @Override
+  public CompactionType getType() {
+    return CompactionType.valueOf(tac.getType().name());
+  }
+
+  @Override
+  public CompactionReason getReason() {
+    return CompactionReason.valueOf(tac.getReason().name());
+  }
+
+  @Override
+  public String getLocalityGroup() {
+    return tac.getLocalityGroup();
+  }
+
+  @Override
+  public long getEntriesRead() {
+    return tac.getEntriesRead();
+  }
+
+  @Override
+  public long getEntriesWritten() {
+    return tac.getEntriesWritten();
+  }
+
+  @Override
+  public List<IteratorSetting> getIterators() {
+    ArrayList<IteratorSetting> ret = new ArrayList<IteratorSetting>();
+
+    for (IterInfo ii : tac.getSsiList()) {
+      IteratorSetting settings = new IteratorSetting(ii.getPriority(), ii.getIterName(), ii.getClassName());
+      Map<String,String> options = tac.getSsio().get(ii.getIterName());
+      settings.addOptions(options);
+
+      ret.add(settings);
+    }
+
+    return ret;
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/f4454a06/core/src/main/java/org/apache/accumulo/core/client/impl/ActiveScanImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/ActiveScanImpl.java b/core/src/main/java/org/apache/accumulo/core/client/impl/ActiveScanImpl.java
new file mode 100644
index 0000000..4f6fa33
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/ActiveScanImpl.java
@@ -0,0 +1,146 @@
+/*
+ * 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.core.client.impl;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.accumulo.core.client.Instance;
+import org.apache.accumulo.core.client.TableNotFoundException;
+import org.apache.accumulo.core.client.admin.ActiveScan;
+import org.apache.accumulo.core.client.admin.ScanType;
+import org.apache.accumulo.core.client.admin.ScanState;
+import org.apache.accumulo.core.data.Column;
+import org.apache.accumulo.core.data.KeyExtent;
+import org.apache.accumulo.core.data.thrift.IterInfo;
+import org.apache.accumulo.core.data.thrift.TColumn;
+import org.apache.accumulo.core.security.Authorizations;
+
+/**
+ * A class that contains information about an ActiveScan
+ * @since 1.6.0
+ */
+public class ActiveScanImpl extends ActiveScan {
+
+  private long scanid;
+  private String client;
+  private String table;
+  private long age;
+  private long idle;
+  private ScanType type;
+  private ScanState state;
+  private KeyExtent extent;
+  private List<Column> columns;
+  private List<String> ssiList;
+  private Map<String,Map<String,String>> ssio;
+  private String user;
+  private Authorizations authorizations;
+
+  ActiveScanImpl(Instance instance, org.apache.accumulo.core.tabletserver.thrift.ActiveScan activeScan) throws TableNotFoundException {
+    this.client = activeScan.client;
+    this.user = activeScan.user;
+    this.age = activeScan.age;
+    this.idle = activeScan.idleTime;
+    this.table = Tables.getTableName(instance, activeScan.tableId);
+    this.type = ScanType.valueOf(activeScan.getType().name());
+    this.state = ScanState.valueOf(activeScan.state.name());
+    this.extent = new KeyExtent(activeScan.extent);
+    this.authorizations = new Authorizations(activeScan.authorizations);
+
+    this.columns = new ArrayList<Column>(activeScan.columns.size());
+
+    for (TColumn tcolumn : activeScan.columns)
+      this.columns.add(new Column(tcolumn));
+
+    this.ssiList = new ArrayList<String>();
+    for (IterInfo ii : activeScan.ssiList) {
+      this.ssiList.add(ii.iterName + "=" + ii.priority + "," + ii.className);
+    }
+    this.ssio = activeScan.ssio;
+  }
+
+  @Override
+  public long getScanid() {
+    return scanid;
+  }
+
+  @Override
+  public String getClient() {
+    return client;
+  }
+
+  @Override
+  public String getUser() {
+    return user;
+  }
+
+  @Override
+  public String getTable() {
+    return table;
+  }
+
+  @Override
+  public long getAge() {
+    return age;
+  }
+
+  @Override
+  public long getLastContactTime() {
+    return idle;
+  }
+
+  @Override
+  public ScanType getType() {
+    return type;
+  }
+
+  @Override
+  public ScanState getState() {
+    return state;
+  }
+
+  @Override
+  public KeyExtent getExtent() {
+    return extent;
+  }
+
+  @Override
+  public List<Column> getColumns() {
+    return columns;
+  }
+
+  @Override
+  public List<String> getSsiList() {
+    return ssiList;
+  }
+
+  @Override
+  public Map<String,Map<String,String>> getSsio() {
+    return ssio;
+  }
+
+  @Override
+  public Authorizations getAuthorizations() {
+    return authorizations;
+  }
+
+  @Override
+  public long getIdleTime() {
+    return idle;
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/f4454a06/core/src/main/java/org/apache/accumulo/core/client/impl/ConnectorImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/ConnectorImpl.java b/core/src/main/java/org/apache/accumulo/core/client/impl/ConnectorImpl.java
index 7d9d3ab..cdad4e5 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/impl/ConnectorImpl.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/ConnectorImpl.java
@@ -33,13 +33,13 @@ import org.apache.accumulo.core.client.Scanner;
 import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.client.TableOfflineException;
 import org.apache.accumulo.core.client.admin.InstanceOperations;
-import org.apache.accumulo.core.client.admin.InstanceOperationsImpl;
 import org.apache.accumulo.core.client.admin.NamespaceOperations;
-import org.apache.accumulo.core.client.admin.NamespaceOperationsImpl;
 import org.apache.accumulo.core.client.admin.SecurityOperations;
-import org.apache.accumulo.core.client.admin.SecurityOperationsImpl;
 import org.apache.accumulo.core.client.admin.TableOperations;
-import org.apache.accumulo.core.client.admin.TableOperationsImpl;
+import org.apache.accumulo.core.client.impl.InstanceOperationsImpl;
+import org.apache.accumulo.core.client.impl.NamespaceOperationsImpl;
+import org.apache.accumulo.core.client.impl.SecurityOperationsImpl;
+import org.apache.accumulo.core.client.impl.TableOperationsImpl;
 import org.apache.accumulo.core.client.impl.thrift.ClientService;
 import org.apache.accumulo.core.client.impl.thrift.SecurityErrorCode;
 import org.apache.accumulo.core.master.state.tables.TableState;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/f4454a06/core/src/main/java/org/apache/accumulo/core/client/impl/InstanceOperationsImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/InstanceOperationsImpl.java b/core/src/main/java/org/apache/accumulo/core/client/impl/InstanceOperationsImpl.java
new file mode 100644
index 0000000..12e4912
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/InstanceOperationsImpl.java
@@ -0,0 +1,209 @@
+/*
+ * 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.core.client.impl;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+
+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.Instance;
+import org.apache.accumulo.core.client.TableNotFoundException;
+import org.apache.accumulo.core.client.admin.ActiveScan;
+import org.apache.accumulo.core.client.admin.ActiveCompaction;
+import org.apache.accumulo.core.client.admin.InstanceOperations;
+import org.apache.accumulo.core.client.impl.thrift.ClientService;
+import org.apache.accumulo.core.client.impl.thrift.ConfigurationType;
+import org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException;
+import org.apache.accumulo.core.master.thrift.MasterClientService;
+import org.apache.accumulo.core.security.Credentials;
+import org.apache.accumulo.core.tabletserver.thrift.TabletClientService;
+import org.apache.accumulo.core.tabletserver.thrift.TabletClientService.Client;
+import org.apache.accumulo.core.util.AddressUtil;
+import org.apache.accumulo.core.util.ArgumentChecker;
+import org.apache.accumulo.core.util.ThriftUtil;
+import org.apache.accumulo.core.zookeeper.ZooUtil;
+import org.apache.accumulo.fate.zookeeper.ZooCache;
+import org.apache.accumulo.trace.instrument.Tracer;
+import org.apache.thrift.TException;
+import org.apache.thrift.transport.TTransport;
+import org.apache.thrift.transport.TTransportException;
+
+/**
+ * Provides a class for administering the accumulo instance
+ */
+public class InstanceOperationsImpl implements InstanceOperations {
+  private Instance instance;
+  private Credentials credentials;
+
+  /**
+   * @param instance
+   *          the connection information for this instance
+   * @param credentials
+   *          the Credential, containing principal and Authentication Token
+   */
+  public InstanceOperationsImpl(Instance instance, Credentials credentials) {
+    ArgumentChecker.notNull(instance, credentials);
+    this.instance = instance;
+    this.credentials = credentials;
+  }
+
+  @Override
+  public void setProperty(final String property, final String value) throws AccumuloException, AccumuloSecurityException {
+    ArgumentChecker.notNull(property, value);
+    MasterClient.execute(instance, new ClientExec<MasterClientService.Client>() {
+      @Override
+      public void execute(MasterClientService.Client client) throws Exception {
+        client.setSystemProperty(Tracer.traceInfo(), credentials.toThrift(instance), property, value);
+      }
+    });
+  }
+
+  @Override
+  public void removeProperty(final String property) throws AccumuloException, AccumuloSecurityException {
+    ArgumentChecker.notNull(property);
+    MasterClient.execute(instance, new ClientExec<MasterClientService.Client>() {
+      @Override
+      public void execute(MasterClientService.Client client) throws Exception {
+        client.removeSystemProperty(Tracer.traceInfo(), credentials.toThrift(instance), property);
+      }
+    });
+  }
+
+  @Override
+  public Map<String,String> getSystemConfiguration() throws AccumuloException, AccumuloSecurityException {
+    return ServerClient.execute(instance, new ClientExecReturn<Map<String,String>,ClientService.Client>() {
+      @Override
+      public Map<String,String> execute(ClientService.Client client) throws Exception {
+        return client.getConfiguration(Tracer.traceInfo(), credentials.toThrift(instance), ConfigurationType.CURRENT);
+      }
+    });
+  }
+
+  @Override
+  public Map<String,String> getSiteConfiguration() throws AccumuloException, AccumuloSecurityException {
+    return ServerClient.execute(instance, new ClientExecReturn<Map<String,String>,ClientService.Client>() {
+      @Override
+      public Map<String,String> execute(ClientService.Client client) throws Exception {
+        return client.getConfiguration(Tracer.traceInfo(), credentials.toThrift(instance), ConfigurationType.SITE);
+      }
+    });
+  }
+
+  @Override
+  public List<String> getTabletServers() {
+    ZooCache cache = ZooCache.getInstance(instance.getZooKeepers(), instance.getZooKeepersSessionTimeOut());
+    String path = ZooUtil.getRoot(instance) + Constants.ZTSERVERS;
+    List<String> results = new ArrayList<String>();
+    for (String candidate : cache.getChildren(path)) {
+      List<String> children = cache.getChildren(path + "/" + candidate);
+      if (children != null && children.size() > 0) {
+        List<String> copy = new ArrayList<String>(children);
+        Collections.sort(copy);
+        byte[] data = cache.get(path + "/" + candidate + "/" + copy.get(0));
+        if (data != null && !"master".equals(new String(data, Constants.UTF8))) {
+          results.add(candidate);
+        }
+      }
+    }
+    return results;
+  }
+
+  @Override
+  public List<ActiveScan> getActiveScans(String tserver) throws AccumuloException, AccumuloSecurityException {
+    Client client = null;
+    try {
+      client = ThriftUtil.getTServerClient(tserver, ServerConfigurationUtil.getConfiguration(instance));
+
+      List<ActiveScan> as = new ArrayList<ActiveScan>();
+      for (org.apache.accumulo.core.tabletserver.thrift.ActiveScan activeScan : client.getActiveScans(Tracer.traceInfo(), credentials.toThrift(instance))) {
+        try {
+          as.add(new ActiveScanImpl(instance, activeScan));
+        } catch (TableNotFoundException e) {
+          throw new AccumuloException(e);
+        }
+      }
+      return as;
+    } catch (TTransportException e) {
+      throw new AccumuloException(e);
+    } catch (ThriftSecurityException e) {
+      throw new AccumuloSecurityException(e.user, e.code, e);
+    } catch (TException e) {
+      throw new AccumuloException(e);
+    } finally {
+      if (client != null)
+        ThriftUtil.returnClient(client);
+    }
+  }
+
+  @Override
+  public boolean testClassLoad(final String className, final String asTypeName) throws AccumuloException, AccumuloSecurityException {
+    return ServerClient.execute(instance, new ClientExecReturn<Boolean,ClientService.Client>() {
+      @Override
+      public Boolean execute(ClientService.Client client) throws Exception {
+        return client.checkClass(Tracer.traceInfo(), credentials.toThrift(instance), className, asTypeName);
+      }
+    });
+  }
+
+  @Override
+  public List<ActiveCompaction> getActiveCompactions(String tserver) throws AccumuloException, AccumuloSecurityException {
+    Client client = null;
+    try {
+      client = ThriftUtil.getTServerClient(tserver, ServerConfigurationUtil.getConfiguration(instance));
+
+      List<ActiveCompaction> as = new ArrayList<ActiveCompaction>();
+      for (org.apache.accumulo.core.tabletserver.thrift.ActiveCompaction activeCompaction : client.getActiveCompactions(Tracer.traceInfo(),
+          credentials.toThrift(instance))) {
+        as.add(new ActiveCompactionImpl(instance, activeCompaction));
+      }
+      return as;
+    } catch (TTransportException e) {
+      throw new AccumuloException(e);
+    } catch (ThriftSecurityException e) {
+      throw new AccumuloSecurityException(e.user, e.code, e);
+    } catch (TException e) {
+      throw new AccumuloException(e);
+    } finally {
+      if (client != null)
+        ThriftUtil.returnClient(client);
+    }
+  }
+
+  @Override
+  public void ping(String tserver) throws AccumuloException {
+    TTransport transport = null;
+    try {
+      transport = ThriftUtil.createTransport(AddressUtil.parseAddress(tserver, false), ServerConfigurationUtil.getConfiguration(instance));
+      TabletClientService.Client client = ThriftUtil.createClient(new TabletClientService.Client.Factory(), transport);
+      client.getTabletServerStatus(Tracer.traceInfo(), credentials.toThrift(instance));
+    } catch (TTransportException e) {
+      throw new AccumuloException(e);
+    } catch (ThriftSecurityException e) {
+      throw new AccumuloException(e);
+    } catch (TException e) {
+      throw new AccumuloException(e);
+    } finally {
+      if (transport != null) {
+        transport.close();
+      }
+    }
+  }
+}