You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@accumulo.apache.org by kt...@apache.org on 2013/07/23 18:55:08 UTC

[43/50] git commit: ACCUMULO-1533 Clean up deprecated mapreduce and thrift code

ACCUMULO-1533 Clean up deprecated mapreduce and thrift code


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

Branch: refs/heads/ACCUMULO-1000
Commit: a69a9d68fb8f1e2e7455b75a5a1130e964d96a7d
Parents: 47d1f5f
Author: Christopher Tubbs <ct...@apache.org>
Authored: Mon Jul 22 14:33:03 2013 -0400
Committer: Christopher Tubbs <ct...@apache.org>
Committed: Mon Jul 22 14:33:03 2013 -0400

----------------------------------------------------------------------
 .../apache/accumulo/core/cli/ClientOpts.java    |   4 +-
 .../core/client/AccumuloSecurityException.java  |  18 +-
 .../apache/accumulo/core/client/Instance.java   |  15 -
 .../accumulo/core/client/ZooKeeperInstance.java |  61 +-
 .../core/client/impl/ConnectorImpl.java         |   1 -
 .../mapreduce/AccumuloFileOutputFormat.java     |  49 --
 .../client/mapreduce/AccumuloOutputFormat.java  | 164 -----
 .../core/client/mapreduce/InputFormatBase.java  | 385 ------------
 .../accumulo/core/client/mock/MockInstance.java |   6 -
 .../core/conf/AccumuloConfiguration.java        |  18 +-
 .../apache/accumulo/core/data/ColumnUpdate.java |  10 -
 .../accumulo/core/security/thrift/AuthInfo.java | 616 -------------------
 .../core/security/thrift/SecurityErrorCode.java | 112 ----
 .../thrift/ThriftSecurityException.java         | 521 ----------------
 .../apache/accumulo/core/util/shell/Shell.java  |   4 +-
 .../core/util/shell/commands/FateCommand.java   |   7 +-
 .../accumulo/core/zookeeper/ZooCache.java       |  40 --
 .../apache/accumulo/core/zookeeper/ZooUtil.java |  42 ++
 core/src/main/thrift/security.thrift            |  40 --
 .../core/client/impl/TabletLocatorImplTest.java |   6 -
 .../mapreduce/AccumuloInputFormatTest.java      |  40 --
 .../client/security/SecurityErrorCodeTest.java  |   8 +-
 .../accumulo/core/file/rfile/RFileTest.java     |   2 +-
 .../accumulo/server/client/HdfsZooInstance.java |  16 +-
 .../accumulo/server/conf/ZooConfiguration.java  |   5 +-
 25 files changed, 71 insertions(+), 2119 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/a69a9d68/core/src/main/java/org/apache/accumulo/core/cli/ClientOpts.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/cli/ClientOpts.java b/core/src/main/java/org/apache/accumulo/core/cli/ClientOpts.java
index 8a2d6f0..ec6b198 100644
--- a/core/src/main/java/org/apache/accumulo/core/cli/ClientOpts.java
+++ b/core/src/main/java/org/apache/accumulo/core/cli/ClientOpts.java
@@ -42,6 +42,7 @@ import org.apache.accumulo.core.conf.DefaultConfiguration;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.security.ColumnVisibility;
+import org.apache.accumulo.core.zookeeper.ZooUtil;
 import org.apache.accumulo.trace.instrument.Trace;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
@@ -221,8 +222,7 @@ public class ClientOpts extends Help {
       };
       this.zookeepers = config.get(Property.INSTANCE_ZK_HOST);
       Path instanceDir = new Path(config.get(Property.INSTANCE_DFS_DIR), "instance_id");
-      @SuppressWarnings("deprecation")
-      String instanceIDFromFile = ZooKeeperInstance.getInstanceIDFromHdfs(instanceDir);
+      String instanceIDFromFile = ZooUtil.getInstanceIDFromHdfs(instanceDir);
       return cachedInstance = new ZooKeeperInstance(UUID.fromString(instanceIDFromFile), zookeepers);
     }
     return cachedInstance = new ZooKeeperInstance(this.instance, this.zookeepers);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/a69a9d68/core/src/main/java/org/apache/accumulo/core/client/AccumuloSecurityException.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/AccumuloSecurityException.java b/core/src/main/java/org/apache/accumulo/core/client/AccumuloSecurityException.java
index 04ba4e7..f626f4d 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/AccumuloSecurityException.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/AccumuloSecurityException.java
@@ -94,7 +94,7 @@ public class AccumuloSecurityException extends Exception {
    * @param errorcode
    *          the specific reason for this exception
    * @param tableInfo
-   *          the relevant tableInfo for the security violation 
+   *          the relevant tableInfo for the security violation
    * @param cause
    *          the exception that caused this violation
    */
@@ -123,7 +123,7 @@ public class AccumuloSecurityException extends Exception {
    * @param errorcode
    *          the specific reason for this exception
    * @param tableInfo
-   *          the relevant tableInfo for the security violation 
+   *          the relevant tableInfo for the security violation
    */
   public AccumuloSecurityException(final String user, final SecurityErrorCode errorcode, final String tableInfo) {
     super(getDefaultErrorMessage(errorcode));
@@ -158,25 +158,17 @@ public class AccumuloSecurityException extends Exception {
    * @return the specific reason for this exception
    * @since 1.5.0
    */
-
+  
   public org.apache.accumulo.core.client.security.SecurityErrorCode getSecurityErrorCode() {
     return org.apache.accumulo.core.client.security.SecurityErrorCode.valueOf(errorCode.name());
   }
-
-  /**
-   * @return the specific reason for this exception
-   * 
-   * @deprecated since 1.5.0; Use {@link #getSecurityErrorCode()} instead.
-   */
-  public org.apache.accumulo.core.security.thrift.SecurityErrorCode getErrorCode() {
-    return org.apache.accumulo.core.security.thrift.SecurityErrorCode.valueOf(errorCode.name());
-  }
   
+  @Override
   public String getMessage() {
     StringBuilder message = new StringBuilder();
     message.append("Error ").append(errorCode);
     message.append(" for user ").append(user);
-    if(!StringUtils.isEmpty(tableInfo)) {
+    if (!StringUtils.isEmpty(tableInfo)) {
       message.append(" on table ").append(tableInfo);
     }
     message.append(" - ").append(super.getMessage());

http://git-wip-us.apache.org/repos/asf/accumulo/blob/a69a9d68/core/src/main/java/org/apache/accumulo/core/client/Instance.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/Instance.java b/core/src/main/java/org/apache/accumulo/core/client/Instance.java
index 3b04281..0796059 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/Instance.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/Instance.java
@@ -90,21 +90,6 @@ public interface Instance {
   /**
    * Returns a connection to accumulo.
    * 
-   * @param auth
-   *          An Credentials object.
-   * @return the accumulo Connector
-   * @throws AccumuloException
-   *           when a generic exception occurs
-   * @throws AccumuloSecurityException
-   *           when a user's credentials are invalid
-   * @deprecated since 1.5, use {@link #getConnector(String, AuthenticationToken)} with {@link PasswordToken}
-   */
-  @Deprecated
-  public abstract Connector getConnector(org.apache.accumulo.core.security.thrift.AuthInfo auth) throws AccumuloException, AccumuloSecurityException;
-  
-  /**
-   * Returns a connection to accumulo.
-   * 
    * @param user
    *          a valid accumulo user
    * @param pass

http://git-wip-us.apache.org/repos/asf/accumulo/blob/a69a9d68/core/src/main/java/org/apache/accumulo/core/client/ZooKeeperInstance.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/ZooKeeperInstance.java b/core/src/main/java/org/apache/accumulo/core/client/ZooKeeperInstance.java
index 5b56adb..07f5fd4 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/ZooKeeperInstance.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/ZooKeeperInstance.java
@@ -16,8 +16,6 @@
  */
 package org.apache.accumulo.core.client;
 
-import java.io.FileNotFoundException;
-import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.util.Collections;
 import java.util.List;
@@ -29,20 +27,14 @@ import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
 import org.apache.accumulo.core.client.security.tokens.PasswordToken;
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
 import org.apache.accumulo.core.conf.Property;
-import org.apache.accumulo.core.file.FileUtil;
 import org.apache.accumulo.core.metadata.RootTable;
 import org.apache.accumulo.core.security.CredentialHelper;
-import org.apache.accumulo.core.security.thrift.TCredentials;
 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.OpTimer;
 import org.apache.accumulo.core.util.TextUtil;
 import org.apache.accumulo.core.zookeeper.ZooUtil;
 import org.apache.accumulo.fate.zookeeper.ZooCache;
-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;
@@ -220,12 +212,7 @@ public class ZooKeeperInstance implements Instance {
   
   @Override
   public Connector getConnector(String principal, AuthenticationToken token) throws AccumuloException, AccumuloSecurityException {
-    return getConnector(CredentialHelper.create(principal, token, getInstanceID()));
-  }
-  
-  @SuppressWarnings("deprecation")
-  private Connector getConnector(TCredentials credential) throws AccumuloException, AccumuloSecurityException {
-    return new ConnectorImpl(this, credential);
+    return new ConnectorImpl(this, CredentialHelper.create(principal, token, getInstanceID()));
   }
   
   @Override
@@ -249,19 +236,7 @@ public class ZooKeeperInstance implements Instance {
   }
   
   /**
-   * @deprecated Use {@link #lookupInstanceName(org.apache.accumulo.fate.zookeeper.ZooCache, UUID)} instead
-   */
-  @Deprecated
-  public static String lookupInstanceName(org.apache.accumulo.core.zookeeper.ZooCache zooCache, UUID instanceId) {
-    return lookupInstanceName((ZooCache) zooCache, instanceId);
-  }
-  
-  /**
    * Given a zooCache and instanceId, look up the instance name.
-   * 
-   * @param zooCache
-   * @param instanceId
-   * @return the instance name
    */
   public static String lookupInstanceName(ZooCache zooCache, UUID instanceId) {
     ArgumentChecker.notNull(zooCache, instanceId);
@@ -275,38 +250,4 @@ public class ZooKeeperInstance implements Instance {
     return null;
   }
   
-  /**
-   * To be moved to server code. Only lives here to support certain client side utilities to minimize command-line options.
-   */
-  @Deprecated
-  public static String getInstanceIDFromHdfs(Path instanceDirectory) {
-    try {
-      FileSystem fs = FileUtil.getFileSystem(CachedConfiguration.getInstance(), AccumuloConfiguration.getSiteConfiguration());
-      FileStatus[] files = null;
-      try {
-        files = fs.listStatus(instanceDirectory);
-      } catch (FileNotFoundException ex) {
-        // ignored
-      }
-      log.debug("Trying to read instance id from " + instanceDirectory);
-      if (files == null || files.length == 0) {
-        log.error("unable obtain instance id at " + instanceDirectory);
-        throw new RuntimeException("Accumulo not initialized, there is no instance id at " + instanceDirectory);
-      } else if (files.length != 1) {
-        log.error("multiple potential instances in " + instanceDirectory);
-        throw new RuntimeException("Accumulo found multiple possible instance ids in " + instanceDirectory);
-      } else {
-        String result = files[0].getPath().getName();
-        return result;
-      }
-    } catch (IOException e) {
-      throw new RuntimeException("Accumulo not initialized, there is no instance id at " + instanceDirectory, e);
-    }
-  }
-  
-  @Deprecated
-  @Override
-  public Connector getConnector(org.apache.accumulo.core.security.thrift.AuthInfo auth) throws AccumuloException, AccumuloSecurityException {
-    return getConnector(auth.user, auth.password);
-  }
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/a69a9d68/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 3c6e445..3858cdc 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
@@ -51,7 +51,6 @@ public class ConnectorImpl extends Connector {
   private TableOperations tableops = null;
   private InstanceOperations instanceops = null;
   
-  @Deprecated
   public ConnectorImpl(Instance instance, TCredentials cred) throws AccumuloException, AccumuloSecurityException {
     ArgumentChecker.notNull(instance, cred);
     this.instance = instance;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/a69a9d68/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloFileOutputFormat.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloFileOutputFormat.java b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloFileOutputFormat.java
index cfcefda..d78219c 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloFileOutputFormat.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloFileOutputFormat.java
@@ -19,7 +19,6 @@ package org.apache.accumulo.core.client.mapreduce;
 import java.io.IOException;
 import java.util.Arrays;
 
-import org.apache.accumulo.core.client.Instance;
 import org.apache.accumulo.core.client.mapreduce.lib.util.FileOutputConfigurator;
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
 import org.apache.accumulo.core.conf.Property;
@@ -28,14 +27,12 @@ import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.file.FileOperations;
 import org.apache.accumulo.core.file.FileSKVWriter;
-import org.apache.accumulo.core.file.rfile.RFile;
 import org.apache.accumulo.core.security.ColumnVisibility;
 import org.apache.commons.collections.map.LRUMap;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.mapreduce.Job;
 import org.apache.hadoop.mapreduce.JobContext;
-import org.apache.hadoop.mapreduce.OutputFormat;
 import org.apache.hadoop.mapreduce.RecordWriter;
 import org.apache.hadoop.mapreduce.TaskAttemptContext;
 import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
@@ -177,50 +174,4 @@ public class AccumuloFileOutputFormat extends FileOutputFormat<Key,Value> {
     };
   }
   
-  // ----------------------------------------------------------------------------------------------------
-  // Everything below this line is deprecated and should go away in future versions
-  // ----------------------------------------------------------------------------------------------------
-  
-  /**
-   * @deprecated since 1.5.0; Retrieve the relevant block size from {@link #getAccumuloConfiguration(JobContext)} and configure hadoop's
-   *             io.seqfile.compress.blocksize with the same value. No longer needed, as {@link RFile} does not use this field.
-   */
-  @Deprecated
-  protected static void handleBlockSize(Configuration conf) {
-    conf.setInt("io.seqfile.compress.blocksize",
-        (int) FileOutputConfigurator.getAccumuloConfiguration(CLASS, conf).getMemoryInBytes(Property.TABLE_FILE_COMPRESSED_BLOCK_SIZE));
-  }
-  
-  /**
-   * @deprecated since 1.5.0; This method does nothing. Only 'rf' type is supported.
-   */
-  @Deprecated
-  public static void setFileType(Configuration conf, String type) {}
-  
-  /**
-   * @deprecated since 1.5.0; Use {@link #setFileBlockSize(Job, long)}, {@link #setDataBlockSize(Job, long)}, or {@link #setIndexBlockSize(Job, long)} instead.
-   */
-  @Deprecated
-  public static void setBlockSize(Configuration conf, int blockSize) {
-    FileOutputConfigurator.setDataBlockSize(CLASS, conf, blockSize);
-  }
-  
-  /**
-   * @deprecated since 1.5.0; This {@link OutputFormat} does not communicate with Accumulo. If this is needed, subclasses must implement their own
-   *             configuration.
-   */
-  @Deprecated
-  public static void setZooKeeperInstance(Configuration conf, String instanceName, String zooKeepers) {
-    FileOutputConfigurator.setZooKeeperInstance(CLASS, conf, instanceName, zooKeepers);
-  }
-  
-  /**
-   * @deprecated since 1.5.0; This {@link OutputFormat} does not communicate with Accumulo. If this is needed, subclasses must implement their own
-   *             configuration.
-   */
-  @Deprecated
-  protected static Instance getInstance(Configuration conf) {
-    return FileOutputConfigurator.getInstance(CLASS, conf);
-  }
-  
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/a69a9d68/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloOutputFormat.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloOutputFormat.java b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloOutputFormat.java
index 69e7e2e..49bd3e1 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloOutputFormat.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloOutputFormat.java
@@ -21,7 +21,6 @@ import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Map.Entry;
 import java.util.Set;
-import java.util.concurrent.TimeUnit;
 
 import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
@@ -38,13 +37,11 @@ import org.apache.accumulo.core.client.mapreduce.lib.util.OutputConfigurator;
 import org.apache.accumulo.core.client.mock.MockInstance;
 import org.apache.accumulo.core.client.security.SecurityErrorCode;
 import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
-import org.apache.accumulo.core.client.security.tokens.PasswordToken;
 import org.apache.accumulo.core.data.ColumnUpdate;
 import org.apache.accumulo.core.data.KeyExtent;
 import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.security.ColumnVisibility;
 import org.apache.accumulo.core.security.CredentialHelper;
-import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.mapreduce.Job;
 import org.apache.hadoop.mapreduce.JobContext;
@@ -536,165 +533,4 @@ public class AccumuloOutputFormat extends OutputFormat<Text,Mutation> {
     }
   }
   
-  // ----------------------------------------------------------------------------------------------------
-  // Everything below this line is deprecated and should go away in future versions
-  // ----------------------------------------------------------------------------------------------------
-  
-  /**
-   * @deprecated since 1.5.0; Use {@link #setConnectorInfo(Job, String, AuthenticationToken)}, {@link #setCreateTables(Job, boolean)}, and
-   *             {@link #setDefaultTableName(Job, String)} instead.
-   */
-  @Deprecated
-  public static void setOutputInfo(Configuration conf, String user, byte[] passwd, boolean createTables, String defaultTable) {
-    try {
-      OutputConfigurator.setConnectorInfo(CLASS, conf, user, new PasswordToken(passwd));
-    } catch (AccumuloSecurityException e) {
-      throw new RuntimeException(e);
-    }
-    OutputConfigurator.setCreateTables(CLASS, conf, createTables);
-    OutputConfigurator.setDefaultTableName(CLASS, conf, defaultTable);
-  }
-  
-  /**
-   * @deprecated since 1.5.0; Use {@link #setZooKeeperInstance(Job, String, String)} instead.
-   */
-  @Deprecated
-  public static void setZooKeeperInstance(Configuration conf, String instanceName, String zooKeepers) {
-    OutputConfigurator.setZooKeeperInstance(CLASS, conf, instanceName, zooKeepers);
-  }
-  
-  /**
-   * @deprecated since 1.5.0; Use {@link #setMockInstance(Job, String)} instead.
-   */
-  @Deprecated
-  public static void setMockInstance(Configuration conf, String instanceName) {
-    OutputConfigurator.setMockInstance(CLASS, conf, instanceName);
-  }
-  
-  /**
-   * @deprecated since 1.5.0; Use {@link #setBatchWriterOptions(Job, BatchWriterConfig)} instead.
-   */
-  @Deprecated
-  public static void setMaxMutationBufferSize(Configuration conf, long numberOfBytes) {
-    BatchWriterConfig bwConfig = OutputConfigurator.getBatchWriterOptions(CLASS, conf);
-    bwConfig.setMaxMemory(numberOfBytes);
-    OutputConfigurator.setBatchWriterOptions(CLASS, conf, bwConfig);
-  }
-  
-  /**
-   * @deprecated since 1.5.0; Use {@link #setBatchWriterOptions(Job, BatchWriterConfig)} instead.
-   */
-  @Deprecated
-  public static void setMaxLatency(Configuration conf, int numberOfMilliseconds) {
-    BatchWriterConfig bwConfig = OutputConfigurator.getBatchWriterOptions(CLASS, conf);
-    bwConfig.setMaxLatency(numberOfMilliseconds, TimeUnit.MILLISECONDS);
-    OutputConfigurator.setBatchWriterOptions(CLASS, conf, bwConfig);
-  }
-  
-  /**
-   * @deprecated since 1.5.0; Use {@link #setBatchWriterOptions(Job, BatchWriterConfig)} instead.
-   */
-  @Deprecated
-  public static void setMaxWriteThreads(Configuration conf, int numberOfThreads) {
-    BatchWriterConfig bwConfig = OutputConfigurator.getBatchWriterOptions(CLASS, conf);
-    bwConfig.setMaxWriteThreads(numberOfThreads);
-    OutputConfigurator.setBatchWriterOptions(CLASS, conf, bwConfig);
-  }
-  
-  /**
-   * @deprecated since 1.5.0; Use {@link #setLogLevel(Job, Level)} instead.
-   */
-  @Deprecated
-  public static void setLogLevel(Configuration conf, Level level) {
-    OutputConfigurator.setLogLevel(CLASS, conf, level);
-  }
-  
-  /**
-   * @deprecated since 1.5.0; Use {@link #setSimulationMode(Job, boolean)} instead.
-   */
-  @Deprecated
-  public static void setSimulationMode(Configuration conf) {
-    OutputConfigurator.setSimulationMode(CLASS, conf, true);
-  }
-  
-  /**
-   * @deprecated since 1.5.0; Use {@link #getToken(JobContext)} instead.
-   */
-  @Deprecated
-  protected static String getPrincipal(Configuration conf) {
-    return OutputConfigurator.getPrincipal(CLASS, conf);
-  }
-  
-  /**
-   * @deprecated since 1.5.0; Use {@link #getToken(JobContext)} instead.
-   */
-  @Deprecated
-  protected static byte[] getToken(Configuration conf) {
-    return OutputConfigurator.getToken(CLASS, conf);
-  }
-  
-  /**
-   * @deprecated since 1.5.0; Use {@link #canCreateTables(JobContext)} instead.
-   */
-  @Deprecated
-  protected static boolean canCreateTables(Configuration conf) {
-    return OutputConfigurator.canCreateTables(CLASS, conf);
-  }
-  
-  /**
-   * @deprecated since 1.5.0; Use {@link #getDefaultTableName(JobContext)} instead.
-   */
-  @Deprecated
-  protected static String getDefaultTableName(Configuration conf) {
-    return OutputConfigurator.getDefaultTableName(CLASS, conf);
-  }
-  
-  /**
-   * @deprecated since 1.5.0; Use {@link #getInstance(JobContext)} instead.
-   */
-  @Deprecated
-  protected static Instance getInstance(Configuration conf) {
-    return OutputConfigurator.getInstance(CLASS, conf);
-  }
-  
-  /**
-   * @deprecated since 1.5.0; Use {@link #getBatchWriterOptions(JobContext)} instead.
-   */
-  @Deprecated
-  protected static long getMaxMutationBufferSize(Configuration conf) {
-    return OutputConfigurator.getBatchWriterOptions(CLASS, conf).getMaxMemory();
-  }
-  
-  /**
-   * @deprecated since 1.5.0; Use {@link #getBatchWriterOptions(JobContext)} instead.
-   */
-  @Deprecated
-  protected static int getMaxLatency(Configuration conf) {
-    return (int) OutputConfigurator.getBatchWriterOptions(CLASS, conf).getMaxLatency(TimeUnit.MILLISECONDS);
-  }
-  
-  /**
-   * @deprecated since 1.5.0; Use {@link #getBatchWriterOptions(JobContext)} instead.
-   */
-  @Deprecated
-  protected static int getMaxWriteThreads(Configuration conf) {
-    return OutputConfigurator.getBatchWriterOptions(CLASS, conf).getMaxWriteThreads();
-  }
-  
-  /**
-   * @deprecated since 1.5.0; Use {@link #getLogLevel(JobContext)} instead.
-   */
-  @Deprecated
-  protected static Level getLogLevel(Configuration conf) {
-    return OutputConfigurator.getLogLevel(CLASS, conf);
-  }
-  
-  /**
-   * @deprecated since 1.5.0; Use {@link #getSimulationMode(JobContext)} instead.
-   */
-  @Deprecated
-  protected static boolean getSimulationMode(Configuration conf) {
-    return OutputConfigurator.getSimulationMode(CLASS, conf);
-  }
-  
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/a69a9d68/core/src/main/java/org/apache/accumulo/core/client/mapreduce/InputFormatBase.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/InputFormatBase.java b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/InputFormatBase.java
index ea40e02..8308a63 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/InputFormatBase.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/InputFormatBase.java
@@ -19,12 +19,9 @@ package org.apache.accumulo.core.client.mapreduce;
 import java.io.DataInput;
 import java.io.DataOutput;
 import java.io.IOException;
-import java.io.UnsupportedEncodingException;
 import java.lang.reflect.Method;
 import java.math.BigInteger;
 import java.net.InetAddress;
-import java.net.URLDecoder;
-import java.net.URLEncoder;
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.Collection;
@@ -34,7 +31,6 @@ import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Set;
-import java.util.StringTokenizer;
 
 import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
@@ -55,14 +51,12 @@ import org.apache.accumulo.core.client.impl.TabletLocator;
 import org.apache.accumulo.core.client.mapreduce.lib.util.InputConfigurator;
 import org.apache.accumulo.core.client.mock.MockInstance;
 import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
-import org.apache.accumulo.core.client.security.tokens.PasswordToken;
 import org.apache.accumulo.core.data.ByteSequence;
 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.user.VersioningIterator;
 import org.apache.accumulo.core.master.state.tables.TableState;
 import org.apache.accumulo.core.metadata.MetadataTable;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection;
@@ -999,385 +993,6 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
     }
   }
   
-  // ----------------------------------------------------------------------------------------------------
-  // Everything below this line is deprecated and should go away in future versions
-  // ----------------------------------------------------------------------------------------------------
-  
-  /**
-   * @deprecated since 1.5.0; Use {@link #setScanIsolation(Job, boolean)} instead.
-   */
-  @Deprecated
-  public static void setIsolated(Configuration conf, boolean enable) {
-    InputConfigurator.setScanIsolation(CLASS, conf, enable);
-  }
-  
-  /**
-   * @deprecated since 1.5.0; Use {@link #setLocalIterators(Job, boolean)} instead.
-   */
-  @Deprecated
-  public static void setLocalIterators(Configuration conf, boolean enable) {
-    InputConfigurator.setLocalIterators(CLASS, conf, enable);
-  }
-  
-  /**
-   * @deprecated since 1.5.0; Use {@link #setConnectorInfo(Job, String, AuthenticationToken)}, {@link #setInputTableName(Job, String)}, and
-   *             {@link #setScanAuthorizations(Job, Authorizations)} instead.
-   */
-  @Deprecated
-  public static void setInputInfo(Configuration conf, String user, byte[] passwd, String table, Authorizations auths) {
-    try {
-      InputConfigurator.setConnectorInfo(CLASS, conf, user, new PasswordToken(passwd));
-    } catch (AccumuloSecurityException e) {
-      throw new RuntimeException(e);
-    }
-    InputConfigurator.setInputTableName(CLASS, conf, table);
-    InputConfigurator.setScanAuthorizations(CLASS, conf, auths);
-  }
-  
-  /**
-   * @deprecated since 1.5.0; Use {@link #setZooKeeperInstance(Job, String, String)} instead.
-   */
-  @Deprecated
-  public static void setZooKeeperInstance(Configuration conf, String instanceName, String zooKeepers) {
-    InputConfigurator.setZooKeeperInstance(CLASS, conf, instanceName, zooKeepers);
-  }
-  
-  /**
-   * @deprecated since 1.5.0; Use {@link #setMockInstance(Job, String)} instead.
-   */
-  @Deprecated
-  public static void setMockInstance(Configuration conf, String instanceName) {
-    InputConfigurator.setMockInstance(CLASS, conf, instanceName);
-  }
-  
-  /**
-   * @deprecated since 1.5.0; Use {@link #setRanges(Job, Collection)} instead.
-   */
-  @Deprecated
-  public static void setRanges(Configuration conf, Collection<Range> ranges) {
-    InputConfigurator.setRanges(CLASS, conf, ranges);
-  }
-  
-  /**
-   * @deprecated since 1.5.0; Use {@link #setAutoAdjustRanges(Job, boolean)} instead.
-   */
-  @Deprecated
-  public static void disableAutoAdjustRanges(Configuration conf) {
-    InputConfigurator.setAutoAdjustRanges(CLASS, conf, false);
-  }
-  
-  /**
-   * @deprecated since 1.5.0; Use {@link #addIterator(Job, IteratorSetting)} to add the {@link VersioningIterator} instead.
-   */
-  @Deprecated
-  public static void setMaxVersions(Configuration conf, int maxVersions) throws IOException {
-    IteratorSetting vers = new IteratorSetting(1, "vers", VersioningIterator.class);
-    try {
-      VersioningIterator.setMaxVersions(vers, maxVersions);
-    } catch (IllegalArgumentException e) {
-      throw new IOException(e);
-    }
-    InputConfigurator.addIterator(CLASS, conf, vers);
-  }
-  
-  /**
-   * @deprecated since 1.5.0; Use {@link #setOfflineTableScan(Job, boolean)} instead.
-   */
-  @Deprecated
-  public static void setScanOffline(Configuration conf, boolean scanOff) {
-    InputConfigurator.setOfflineTableScan(CLASS, conf, scanOff);
-  }
-  
-  /**
-   * @deprecated since 1.5.0; Use {@link #fetchColumns(Job, Collection)} instead.
-   */
-  @Deprecated
-  public static void fetchColumns(Configuration conf, Collection<Pair<Text,Text>> columnFamilyColumnQualifierPairs) {
-    InputConfigurator.fetchColumns(CLASS, conf, columnFamilyColumnQualifierPairs);
-  }
-  
-  /**
-   * @deprecated since 1.5.0; Use {@link #setLogLevel(Job, Level)} instead.
-   */
-  @Deprecated
-  public static void setLogLevel(Configuration conf, Level level) {
-    InputConfigurator.setLogLevel(CLASS, conf, level);
-  }
-  
-  /**
-   * @deprecated since 1.5.0; Use {@link #addIterator(Job, IteratorSetting)} instead.
-   */
-  @Deprecated
-  public static void addIterator(Configuration conf, IteratorSetting cfg) {
-    InputConfigurator.addIterator(CLASS, conf, cfg);
-  }
-  
-  /**
-   * @deprecated since 1.5.0; Use {@link #isIsolated(JobContext)} instead.
-   */
-  @Deprecated
-  protected static boolean isIsolated(Configuration conf) {
-    return InputConfigurator.isIsolated(CLASS, conf);
-  }
-  
-  /**
-   * @deprecated since 1.5.0; Use {@link #usesLocalIterators(JobContext)} instead.
-   */
-  @Deprecated
-  protected static boolean usesLocalIterators(Configuration conf) {
-    return InputConfigurator.usesLocalIterators(CLASS, conf);
-  }
-  
-  /**
-   * @deprecated since 1.5.0; Use {@link #getPrincipal(JobContext)} instead.
-   */
-  @Deprecated
-  protected static String getPrincipal(Configuration conf) {
-    return InputConfigurator.getPrincipal(CLASS, conf);
-  }
-  
-  /**
-   * @deprecated since 1.5.0; Use {@link #getToken(JobContext)} instead.
-   */
-  @Deprecated
-  protected static byte[] getToken(Configuration conf) {
-    return InputConfigurator.getToken(CLASS, conf);
-  }
-  
-  /**
-   * @deprecated since 1.5.0; Use {@link #getInputTableName(JobContext)} instead.
-   */
-  @Deprecated
-  protected static String getTablename(Configuration conf) {
-    return InputConfigurator.getInputTableName(CLASS, conf);
-  }
-  
-  /**
-   * @deprecated since 1.5.0; Use {@link #getScanAuthorizations(JobContext)} instead.
-   */
-  @Deprecated
-  protected static Authorizations getAuthorizations(Configuration conf) {
-    return InputConfigurator.getScanAuthorizations(CLASS, conf);
-  }
-  
-  /**
-   * @deprecated since 1.5.0; Use {@link #getInstance(JobContext)} instead.
-   */
-  @Deprecated
-  protected static Instance getInstance(Configuration conf) {
-    return InputConfigurator.getInstance(CLASS, conf);
-  }
-  
-  /**
-   * @deprecated since 1.5.0; Use {@link #getTabletLocator(JobContext)} instead.
-   */
-  @Deprecated
-  protected static TabletLocator getTabletLocator(Configuration conf) throws TableNotFoundException {
-    return InputConfigurator.getTabletLocator(CLASS, conf);
-  }
-  
-  /**
-   * @deprecated since 1.5.0; Use {@link #getRanges(JobContext)} instead.
-   */
-  @Deprecated
-  protected static List<Range> getRanges(Configuration conf) throws IOException {
-    return InputConfigurator.getRanges(CLASS, conf);
-  }
-  
-  /**
-   * @deprecated since 1.5.0; Use {@link #getFetchedColumns(JobContext)} instead.
-   */
-  @Deprecated
-  protected static Set<Pair<Text,Text>> getFetchedColumns(Configuration conf) {
-    return InputConfigurator.getFetchedColumns(CLASS, conf);
-  }
-  
-  /**
-   * @deprecated since 1.5.0; Use {@link #getAutoAdjustRanges(JobContext)} instead.
-   */
-  @Deprecated
-  protected static boolean getAutoAdjustRanges(Configuration conf) {
-    return InputConfigurator.getAutoAdjustRanges(CLASS, conf);
-  }
-  
-  /**
-   * @deprecated since 1.5.0; Use {@link #getLogLevel(JobContext)} instead.
-   */
-  @Deprecated
-  protected static Level getLogLevel(Configuration conf) {
-    return InputConfigurator.getLogLevel(CLASS, conf);
-  }
-  
-  /**
-   * @deprecated since 1.5.0; Use {@link #validateOptions(JobContext)} instead.
-   */
-  @Deprecated
-  protected static void validateOptions(Configuration conf) throws IOException {
-    InputConfigurator.validateOptions(CLASS, conf);
-  }
-  
-  /**
-   * @deprecated since 1.5.0; Use {@link #addIterator(Job, IteratorSetting)} to add the {@link VersioningIterator} instead.
-   */
-  @Deprecated
-  protected static int getMaxVersions(Configuration conf) {
-    // This is so convoluted, because the only reason to get the number of maxVersions is to construct the same type of IteratorSetting object we have to
-    // deconstruct to get at this option in the first place, but to preserve correct behavior, this appears necessary.
-    List<IteratorSetting> iteratorSettings = InputConfigurator.getIterators(CLASS, conf);
-    for (IteratorSetting setting : iteratorSettings) {
-      if ("vers".equals(setting.getName()) && 1 == setting.getPriority() && VersioningIterator.class.getName().equals(setting.getIteratorClass())) {
-        if (setting.getOptions().containsKey("maxVersions"))
-          return Integer.parseInt(setting.getOptions().get("maxVersions"));
-        else
-          return -1;
-      }
-    }
-    return -1;
-  }
-  
-  /**
-   * @deprecated since 1.5.0; Use {@link #isOfflineScan(JobContext)} instead.
-   */
-  @Deprecated
-  protected static boolean isOfflineScan(Configuration conf) {
-    return InputConfigurator.isOfflineScan(CLASS, conf);
-  }
-  
-  /**
-   * @deprecated since 1.5.0; Use {@link #getIterators(JobContext)} instead.
-   */
-  @Deprecated
-  protected static List<AccumuloIterator> getIterators(Configuration conf) {
-    List<IteratorSetting> iteratorSettings = InputConfigurator.getIterators(CLASS, conf);
-    List<AccumuloIterator> deprecatedIterators = new ArrayList<AccumuloIterator>(iteratorSettings.size());
-    for (IteratorSetting setting : iteratorSettings) {
-      AccumuloIterator deprecatedIter = new AccumuloIterator(new String(setting.getPriority() + AccumuloIterator.FIELD_SEP + setting.getIteratorClass()
-          + AccumuloIterator.FIELD_SEP + setting.getName()));
-      deprecatedIterators.add(deprecatedIter);
-    }
-    return deprecatedIterators;
-  }
-  
-  /**
-   * @deprecated since 1.5.0; Use {@link #getIterators(JobContext)} instead.
-   */
-  @Deprecated
-  protected static List<AccumuloIteratorOption> getIteratorOptions(Configuration conf) {
-    List<IteratorSetting> iteratorSettings = InputConfigurator.getIterators(CLASS, conf);
-    List<AccumuloIteratorOption> deprecatedIteratorOptions = new ArrayList<AccumuloIteratorOption>(iteratorSettings.size());
-    for (IteratorSetting setting : iteratorSettings) {
-      for (Entry<String,String> opt : setting.getOptions().entrySet()) {
-        String deprecatedOption;
-        try {
-          deprecatedOption = new String(setting.getName() + AccumuloIteratorOption.FIELD_SEP + URLEncoder.encode(opt.getKey(), "UTF-8")
-              + AccumuloIteratorOption.FIELD_SEP + URLEncoder.encode(opt.getValue(), "UTF-8"));
-        } catch (UnsupportedEncodingException e) {
-          throw new RuntimeException(e);
-        }
-        deprecatedIteratorOptions.add(new AccumuloIteratorOption(deprecatedOption));
-      }
-    }
-    return deprecatedIteratorOptions;
-  }
-  
-  /**
-   * @deprecated since 1.5.0; Use {@link IteratorSetting} instead.
-   */
-  @Deprecated
-  static class AccumuloIterator {
-    
-    private static final String FIELD_SEP = ":";
-    
-    private int priority;
-    private String iteratorClass;
-    private String iteratorName;
-    
-    public AccumuloIterator(int priority, String iteratorClass, String iteratorName) {
-      this.priority = priority;
-      this.iteratorClass = iteratorClass;
-      this.iteratorName = iteratorName;
-    }
-    
-    // Parses out a setting given an string supplied from an earlier toString() call
-    public AccumuloIterator(String iteratorSetting) {
-      // Parse the string to expand the iterator
-      StringTokenizer tokenizer = new StringTokenizer(iteratorSetting, FIELD_SEP);
-      priority = Integer.parseInt(tokenizer.nextToken());
-      iteratorClass = tokenizer.nextToken();
-      iteratorName = tokenizer.nextToken();
-    }
-    
-    public int getPriority() {
-      return priority;
-    }
-    
-    public String getIteratorClass() {
-      return iteratorClass;
-    }
-    
-    public String getIteratorName() {
-      return iteratorName;
-    }
-    
-    @Override
-    public String toString() {
-      return new String(priority + FIELD_SEP + iteratorClass + FIELD_SEP + iteratorName);
-    }
-    
-  }
-  
-  /**
-   * @deprecated since 1.5.0; Use {@link IteratorSetting} instead.
-   */
-  @Deprecated
-  static class AccumuloIteratorOption {
-    private static final String FIELD_SEP = ":";
-    
-    private String iteratorName;
-    private String key;
-    private String value;
-    
-    public AccumuloIteratorOption(String iteratorName, String key, String value) {
-      this.iteratorName = iteratorName;
-      this.key = key;
-      this.value = value;
-    }
-    
-    // Parses out an option given a string supplied from an earlier toString() call
-    public AccumuloIteratorOption(String iteratorOption) {
-      StringTokenizer tokenizer = new StringTokenizer(iteratorOption, FIELD_SEP);
-      this.iteratorName = tokenizer.nextToken();
-      try {
-        this.key = URLDecoder.decode(tokenizer.nextToken(), "UTF-8");
-        this.value = URLDecoder.decode(tokenizer.nextToken(), "UTF-8");
-      } catch (UnsupportedEncodingException e) {
-        throw new RuntimeException(e);
-      }
-    }
-    
-    public String getIteratorName() {
-      return iteratorName;
-    }
-    
-    public String getKey() {
-      return key;
-    }
-    
-    public String getValue() {
-      return value;
-    }
-    
-    @Override
-    public String toString() {
-      try {
-        return new String(iteratorName + FIELD_SEP + URLEncoder.encode(key, "UTF-8") + FIELD_SEP + URLEncoder.encode(value, "UTF-8"));
-      } catch (UnsupportedEncodingException e) {
-        throw new RuntimeException(e);
-      }
-    }
-    
-  }
-  
   // use reflection to pull the Configuration out of the JobContext for Hadoop 1 and Hadoop 2 compatibility
   static Configuration getConfiguration(JobContext context) {
     try {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/a69a9d68/core/src/main/java/org/apache/accumulo/core/client/mock/MockInstance.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mock/MockInstance.java b/core/src/main/java/org/apache/accumulo/core/client/mock/MockInstance.java
index c0829df..f37994d 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mock/MockInstance.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mock/MockInstance.java
@@ -148,12 +148,6 @@ public class MockInstance implements Instance {
     this.conf = conf;
   }
   
-  @Deprecated
-  @Override
-  public Connector getConnector(org.apache.accumulo.core.security.thrift.AuthInfo auth) throws AccumuloException, AccumuloSecurityException {
-    return getConnector(auth.user, auth.password);
-  }
-  
   @Override
   public Connector getConnector(String principal, AuthenticationToken token) throws AccumuloException, AccumuloSecurityException {
     Connector conn = new MockConnector(principal, acu, this);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/a69a9d68/core/src/main/java/org/apache/accumulo/core/conf/AccumuloConfiguration.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/conf/AccumuloConfiguration.java b/core/src/main/java/org/apache/accumulo/core/conf/AccumuloConfiguration.java
index 28f24ef..28cb0bd 100644
--- a/core/src/main/java/org/apache/accumulo/core/conf/AccumuloConfiguration.java
+++ b/core/src/main/java/org/apache/accumulo/core/conf/AccumuloConfiguration.java
@@ -32,6 +32,7 @@ public abstract class AccumuloConfiguration implements Iterable<Entry<String,Str
   
   public abstract String get(Property property);
   
+  @Override
   public abstract Iterator<Entry<String,String>> iterator();
   
   private void checkType(Property property, PropertyType type) {
@@ -45,15 +46,17 @@ public abstract class AccumuloConfiguration implements Iterable<Entry<String,Str
   
   /**
    * This method returns all properties in a map of string->string under the given prefix property.
-   * @param property the prefix property, and must be of type PropertyType.PREFIX
+   * 
+   * @param property
+   *          the prefix property, and must be of type PropertyType.PREFIX
    * @return a map of strings to strings of the resulting properties
    */
-  public Map<String, String> getAllPropertiesWithPrefix(Property property) {
+  public Map<String,String> getAllPropertiesWithPrefix(Property property) {
     checkType(property, PropertyType.PREFIX);
     
-    Map<String, String> propMap = new HashMap<String, String>(); 
+    Map<String,String> propMap = new HashMap<String,String>();
     
-    for (Entry<String, String> entry : this) {
+    for (Entry<String,String> entry : this) {
       if (entry.getKey().startsWith(property.getKey())) {
         propMap.put(entry.getKey(), entry.getValue());
       }
@@ -153,8 +156,8 @@ public abstract class AccumuloConfiguration implements Iterable<Entry<String,Str
     return DefaultConfiguration.getInstance();
   }
   
-  // Only here for Shell option-free start-up
   /**
+   * Only here for Shell option-free start-up
    * 
    * @deprecated not for client use
    */
@@ -178,7 +181,6 @@ public abstract class AccumuloConfiguration implements Iterable<Entry<String,Str
     return maxFilesPerTablet;
   }
   
-  public void invalidateCache() {
-    // overridden in ZooConfiguration
-  }
+  // overridden in ZooConfiguration
+  public void invalidateCache() {}
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/a69a9d68/core/src/main/java/org/apache/accumulo/core/data/ColumnUpdate.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/data/ColumnUpdate.java b/core/src/main/java/org/apache/accumulo/core/data/ColumnUpdate.java
index 691ec0e..641ca3a 100644
--- a/core/src/main/java/org/apache/accumulo/core/data/ColumnUpdate.java
+++ b/core/src/main/java/org/apache/accumulo/core/data/ColumnUpdate.java
@@ -43,16 +43,6 @@ public class ColumnUpdate {
     this.val = val;
   }
   
-  /**
-   * @deprecated use setTimestamp(long);
-   * @param timestamp
-   */
-  @Deprecated
-  public void setSystemTimestamp(long timestamp) {
-    if (hasTimestamp)
-      throw new IllegalStateException("Cannot set system timestamp when user set a timestamp");
-  }
-  
   public boolean hasTimestamp() {
     return hasTimestamp;
   }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/a69a9d68/core/src/main/java/org/apache/accumulo/core/security/thrift/AuthInfo.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/security/thrift/AuthInfo.java b/core/src/main/java/org/apache/accumulo/core/security/thrift/AuthInfo.java
deleted file mode 100644
index 9df1d29..0000000
--- a/core/src/main/java/org/apache/accumulo/core/security/thrift/AuthInfo.java
+++ /dev/null
@@ -1,616 +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.
- */
-/**
- * Autogenerated by Thrift Compiler (0.9.0)
- *
- * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
- *  @generated
- */
-package org.apache.accumulo.core.security.thrift;
-
-import org.apache.thrift.scheme.IScheme;
-import org.apache.thrift.scheme.SchemeFactory;
-import org.apache.thrift.scheme.StandardScheme;
-
-import org.apache.thrift.scheme.TupleScheme;
-import org.apache.thrift.protocol.TTupleProtocol;
-import org.apache.thrift.protocol.TProtocolException;
-import org.apache.thrift.EncodingUtils;
-import org.apache.thrift.TException;
-import java.util.List;
-import java.util.ArrayList;
-import java.util.Map;
-import java.util.HashMap;
-import java.util.EnumMap;
-import java.util.Set;
-import java.util.HashSet;
-import java.util.EnumSet;
-import java.util.Collections;
-import java.util.BitSet;
-import java.nio.ByteBuffer;
-import java.util.Arrays;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * @deprecated since 1.5
- */
-@SuppressWarnings("all") public class AuthInfo implements org.apache.thrift.TBase<AuthInfo, AuthInfo._Fields>, java.io.Serializable, Cloneable {
-  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("AuthInfo");
-
-  private static final org.apache.thrift.protocol.TField USER_FIELD_DESC = new org.apache.thrift.protocol.TField("user", org.apache.thrift.protocol.TType.STRING, (short)1);
-  private static final org.apache.thrift.protocol.TField PASSWORD_FIELD_DESC = new org.apache.thrift.protocol.TField("password", org.apache.thrift.protocol.TType.STRING, (short)2);
-  private static final org.apache.thrift.protocol.TField INSTANCE_ID_FIELD_DESC = new org.apache.thrift.protocol.TField("instanceId", org.apache.thrift.protocol.TType.STRING, (short)3);
-
-  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
-  static {
-    schemes.put(StandardScheme.class, new AuthInfoStandardSchemeFactory());
-    schemes.put(TupleScheme.class, new AuthInfoTupleSchemeFactory());
-  }
-
-  public String user; // required
-  public ByteBuffer password; // required
-  public String instanceId; // required
-
-  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
-  @SuppressWarnings("all") public enum _Fields implements org.apache.thrift.TFieldIdEnum {
-    USER((short)1, "user"),
-    PASSWORD((short)2, "password"),
-    INSTANCE_ID((short)3, "instanceId");
-
-    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
-
-    static {
-      for (_Fields field : EnumSet.allOf(_Fields.class)) {
-        byName.put(field.getFieldName(), field);
-      }
-    }
-
-    /**
-     * Find the _Fields constant that matches fieldId, or null if its not found.
-     */
-    public static _Fields findByThriftId(int fieldId) {
-      switch(fieldId) {
-        case 1: // USER
-          return USER;
-        case 2: // PASSWORD
-          return PASSWORD;
-        case 3: // INSTANCE_ID
-          return INSTANCE_ID;
-        default:
-          return null;
-      }
-    }
-
-    /**
-     * Find the _Fields constant that matches fieldId, throwing an exception
-     * if it is not found.
-     */
-    public static _Fields findByThriftIdOrThrow(int fieldId) {
-      _Fields fields = findByThriftId(fieldId);
-      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
-      return fields;
-    }
-
-    /**
-     * Find the _Fields constant that matches name, or null if its not found.
-     */
-    public static _Fields findByName(String name) {
-      return byName.get(name);
-    }
-
-    private final short _thriftId;
-    private final String _fieldName;
-
-    _Fields(short thriftId, String fieldName) {
-      _thriftId = thriftId;
-      _fieldName = fieldName;
-    }
-
-    public short getThriftFieldId() {
-      return _thriftId;
-    }
-
-    public String getFieldName() {
-      return _fieldName;
-    }
-  }
-
-  // isset id assignments
-  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
-  static {
-    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
-    tmpMap.put(_Fields.USER, new org.apache.thrift.meta_data.FieldMetaData("user", org.apache.thrift.TFieldRequirementType.DEFAULT, 
-        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
-    tmpMap.put(_Fields.PASSWORD, new org.apache.thrift.meta_data.FieldMetaData("password", org.apache.thrift.TFieldRequirementType.DEFAULT, 
-        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING        , true)));
-    tmpMap.put(_Fields.INSTANCE_ID, new org.apache.thrift.meta_data.FieldMetaData("instanceId", org.apache.thrift.TFieldRequirementType.DEFAULT, 
-        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
-    metaDataMap = Collections.unmodifiableMap(tmpMap);
-    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(AuthInfo.class, metaDataMap);
-  }
-
-  public AuthInfo() {
-  }
-
-  public AuthInfo(
-    String user,
-    ByteBuffer password,
-    String instanceId)
-  {
-    this();
-    this.user = user;
-    this.password = password;
-    this.instanceId = instanceId;
-  }
-
-  /**
-   * Performs a deep copy on <i>other</i>.
-   */
-  public AuthInfo(AuthInfo other) {
-    if (other.isSetUser()) {
-      this.user = other.user;
-    }
-    if (other.isSetPassword()) {
-      this.password = org.apache.thrift.TBaseHelper.copyBinary(other.password);
-;
-    }
-    if (other.isSetInstanceId()) {
-      this.instanceId = other.instanceId;
-    }
-  }
-
-  public AuthInfo deepCopy() {
-    return new AuthInfo(this);
-  }
-
-  @Override
-  public void clear() {
-    this.user = null;
-    this.password = null;
-    this.instanceId = null;
-  }
-
-  public String getUser() {
-    return this.user;
-  }
-
-  public AuthInfo setUser(String user) {
-    this.user = user;
-    return this;
-  }
-
-  public void unsetUser() {
-    this.user = null;
-  }
-
-  /** Returns true if field user is set (has been assigned a value) and false otherwise */
-  public boolean isSetUser() {
-    return this.user != null;
-  }
-
-  public void setUserIsSet(boolean value) {
-    if (!value) {
-      this.user = null;
-    }
-  }
-
-  public byte[] getPassword() {
-    setPassword(org.apache.thrift.TBaseHelper.rightSize(password));
-    return password == null ? null : password.array();
-  }
-
-  public ByteBuffer bufferForPassword() {
-    return password;
-  }
-
-  public AuthInfo setPassword(byte[] password) {
-    setPassword(password == null ? (ByteBuffer)null : ByteBuffer.wrap(password));
-    return this;
-  }
-
-  public AuthInfo setPassword(ByteBuffer password) {
-    this.password = password;
-    return this;
-  }
-
-  public void unsetPassword() {
-    this.password = null;
-  }
-
-  /** Returns true if field password is set (has been assigned a value) and false otherwise */
-  public boolean isSetPassword() {
-    return this.password != null;
-  }
-
-  public void setPasswordIsSet(boolean value) {
-    if (!value) {
-      this.password = null;
-    }
-  }
-
-  public String getInstanceId() {
-    return this.instanceId;
-  }
-
-  public AuthInfo setInstanceId(String instanceId) {
-    this.instanceId = instanceId;
-    return this;
-  }
-
-  public void unsetInstanceId() {
-    this.instanceId = null;
-  }
-
-  /** Returns true if field instanceId is set (has been assigned a value) and false otherwise */
-  public boolean isSetInstanceId() {
-    return this.instanceId != null;
-  }
-
-  public void setInstanceIdIsSet(boolean value) {
-    if (!value) {
-      this.instanceId = null;
-    }
-  }
-
-  public void setFieldValue(_Fields field, Object value) {
-    switch (field) {
-    case USER:
-      if (value == null) {
-        unsetUser();
-      } else {
-        setUser((String)value);
-      }
-      break;
-
-    case PASSWORD:
-      if (value == null) {
-        unsetPassword();
-      } else {
-        setPassword((ByteBuffer)value);
-      }
-      break;
-
-    case INSTANCE_ID:
-      if (value == null) {
-        unsetInstanceId();
-      } else {
-        setInstanceId((String)value);
-      }
-      break;
-
-    }
-  }
-
-  public Object getFieldValue(_Fields field) {
-    switch (field) {
-    case USER:
-      return getUser();
-
-    case PASSWORD:
-      return getPassword();
-
-    case INSTANCE_ID:
-      return getInstanceId();
-
-    }
-    throw new IllegalStateException();
-  }
-
-  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
-  public boolean isSet(_Fields field) {
-    if (field == null) {
-      throw new IllegalArgumentException();
-    }
-
-    switch (field) {
-    case USER:
-      return isSetUser();
-    case PASSWORD:
-      return isSetPassword();
-    case INSTANCE_ID:
-      return isSetInstanceId();
-    }
-    throw new IllegalStateException();
-  }
-
-  @Override
-  public boolean equals(Object that) {
-    if (that == null)
-      return false;
-    if (that instanceof AuthInfo)
-      return this.equals((AuthInfo)that);
-    return false;
-  }
-
-  public boolean equals(AuthInfo that) {
-    if (that == null)
-      return false;
-
-    boolean this_present_user = true && this.isSetUser();
-    boolean that_present_user = true && that.isSetUser();
-    if (this_present_user || that_present_user) {
-      if (!(this_present_user && that_present_user))
-        return false;
-      if (!this.user.equals(that.user))
-        return false;
-    }
-
-    boolean this_present_password = true && this.isSetPassword();
-    boolean that_present_password = true && that.isSetPassword();
-    if (this_present_password || that_present_password) {
-      if (!(this_present_password && that_present_password))
-        return false;
-      if (!this.password.equals(that.password))
-        return false;
-    }
-
-    boolean this_present_instanceId = true && this.isSetInstanceId();
-    boolean that_present_instanceId = true && that.isSetInstanceId();
-    if (this_present_instanceId || that_present_instanceId) {
-      if (!(this_present_instanceId && that_present_instanceId))
-        return false;
-      if (!this.instanceId.equals(that.instanceId))
-        return false;
-    }
-
-    return true;
-  }
-
-  @Override
-  public int hashCode() {
-    return 0;
-  }
-
-  public int compareTo(AuthInfo other) {
-    if (!getClass().equals(other.getClass())) {
-      return getClass().getName().compareTo(other.getClass().getName());
-    }
-
-    int lastComparison = 0;
-    AuthInfo typedOther = (AuthInfo)other;
-
-    lastComparison = Boolean.valueOf(isSetUser()).compareTo(typedOther.isSetUser());
-    if (lastComparison != 0) {
-      return lastComparison;
-    }
-    if (isSetUser()) {
-      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.user, typedOther.user);
-      if (lastComparison != 0) {
-        return lastComparison;
-      }
-    }
-    lastComparison = Boolean.valueOf(isSetPassword()).compareTo(typedOther.isSetPassword());
-    if (lastComparison != 0) {
-      return lastComparison;
-    }
-    if (isSetPassword()) {
-      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.password, typedOther.password);
-      if (lastComparison != 0) {
-        return lastComparison;
-      }
-    }
-    lastComparison = Boolean.valueOf(isSetInstanceId()).compareTo(typedOther.isSetInstanceId());
-    if (lastComparison != 0) {
-      return lastComparison;
-    }
-    if (isSetInstanceId()) {
-      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.instanceId, typedOther.instanceId);
-      if (lastComparison != 0) {
-        return lastComparison;
-      }
-    }
-    return 0;
-  }
-
-  public _Fields fieldForId(int fieldId) {
-    return _Fields.findByThriftId(fieldId);
-  }
-
-  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
-    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
-  }
-
-  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
-    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
-  }
-
-  @Override
-  public String toString() {
-    StringBuilder sb = new StringBuilder("AuthInfo(");
-    boolean first = true;
-
-    sb.append("user:");
-    if (this.user == null) {
-      sb.append("null");
-    } else {
-      sb.append(this.user);
-    }
-    first = false;
-    if (!first) sb.append(", ");
-    sb.append("password:");
-    if (this.password == null) {
-      sb.append("null");
-    } else {
-      org.apache.thrift.TBaseHelper.toString(this.password, sb);
-    }
-    first = false;
-    if (!first) sb.append(", ");
-    sb.append("instanceId:");
-    if (this.instanceId == null) {
-      sb.append("null");
-    } else {
-      sb.append(this.instanceId);
-    }
-    first = false;
-    sb.append(")");
-    return sb.toString();
-  }
-
-  public void validate() throws org.apache.thrift.TException {
-    // check for required fields
-    // check for sub-struct validity
-  }
-
-  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
-    try {
-      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
-    } catch (org.apache.thrift.TException te) {
-      throw new java.io.IOException(te);
-    }
-  }
-
-  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
-    try {
-      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
-    } catch (org.apache.thrift.TException te) {
-      throw new java.io.IOException(te);
-    }
-  }
-
-  private static class AuthInfoStandardSchemeFactory implements SchemeFactory {
-    public AuthInfoStandardScheme getScheme() {
-      return new AuthInfoStandardScheme();
-    }
-  }
-
-  private static class AuthInfoStandardScheme extends StandardScheme<AuthInfo> {
-
-    public void read(org.apache.thrift.protocol.TProtocol iprot, AuthInfo struct) throws org.apache.thrift.TException {
-      org.apache.thrift.protocol.TField schemeField;
-      iprot.readStructBegin();
-      while (true)
-      {
-        schemeField = iprot.readFieldBegin();
-        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
-          break;
-        }
-        switch (schemeField.id) {
-          case 1: // USER
-            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
-              struct.user = iprot.readString();
-              struct.setUserIsSet(true);
-            } else { 
-              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
-            }
-            break;
-          case 2: // PASSWORD
-            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
-              struct.password = iprot.readBinary();
-              struct.setPasswordIsSet(true);
-            } else { 
-              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
-            }
-            break;
-          case 3: // INSTANCE_ID
-            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
-              struct.instanceId = iprot.readString();
-              struct.setInstanceIdIsSet(true);
-            } else { 
-              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
-            }
-            break;
-          default:
-            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
-        }
-        iprot.readFieldEnd();
-      }
-      iprot.readStructEnd();
-
-      // check for required fields of primitive type, which can't be checked in the validate method
-      struct.validate();
-    }
-
-    public void write(org.apache.thrift.protocol.TProtocol oprot, AuthInfo struct) throws org.apache.thrift.TException {
-      struct.validate();
-
-      oprot.writeStructBegin(STRUCT_DESC);
-      if (struct.user != null) {
-        oprot.writeFieldBegin(USER_FIELD_DESC);
-        oprot.writeString(struct.user);
-        oprot.writeFieldEnd();
-      }
-      if (struct.password != null) {
-        oprot.writeFieldBegin(PASSWORD_FIELD_DESC);
-        oprot.writeBinary(struct.password);
-        oprot.writeFieldEnd();
-      }
-      if (struct.instanceId != null) {
-        oprot.writeFieldBegin(INSTANCE_ID_FIELD_DESC);
-        oprot.writeString(struct.instanceId);
-        oprot.writeFieldEnd();
-      }
-      oprot.writeFieldStop();
-      oprot.writeStructEnd();
-    }
-
-  }
-
-  private static class AuthInfoTupleSchemeFactory implements SchemeFactory {
-    public AuthInfoTupleScheme getScheme() {
-      return new AuthInfoTupleScheme();
-    }
-  }
-
-  private static class AuthInfoTupleScheme extends TupleScheme<AuthInfo> {
-
-    @Override
-    public void write(org.apache.thrift.protocol.TProtocol prot, AuthInfo struct) throws org.apache.thrift.TException {
-      TTupleProtocol oprot = (TTupleProtocol) prot;
-      BitSet optionals = new BitSet();
-      if (struct.isSetUser()) {
-        optionals.set(0);
-      }
-      if (struct.isSetPassword()) {
-        optionals.set(1);
-      }
-      if (struct.isSetInstanceId()) {
-        optionals.set(2);
-      }
-      oprot.writeBitSet(optionals, 3);
-      if (struct.isSetUser()) {
-        oprot.writeString(struct.user);
-      }
-      if (struct.isSetPassword()) {
-        oprot.writeBinary(struct.password);
-      }
-      if (struct.isSetInstanceId()) {
-        oprot.writeString(struct.instanceId);
-      }
-    }
-
-    @Override
-    public void read(org.apache.thrift.protocol.TProtocol prot, AuthInfo struct) throws org.apache.thrift.TException {
-      TTupleProtocol iprot = (TTupleProtocol) prot;
-      BitSet incoming = iprot.readBitSet(3);
-      if (incoming.get(0)) {
-        struct.user = iprot.readString();
-        struct.setUserIsSet(true);
-      }
-      if (incoming.get(1)) {
-        struct.password = iprot.readBinary();
-        struct.setPasswordIsSet(true);
-      }
-      if (incoming.get(2)) {
-        struct.instanceId = iprot.readString();
-        struct.setInstanceIdIsSet(true);
-      }
-    }
-  }
-
-}
-

http://git-wip-us.apache.org/repos/asf/accumulo/blob/a69a9d68/core/src/main/java/org/apache/accumulo/core/security/thrift/SecurityErrorCode.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/security/thrift/SecurityErrorCode.java b/core/src/main/java/org/apache/accumulo/core/security/thrift/SecurityErrorCode.java
deleted file mode 100644
index 7aa0dd2..0000000
--- a/core/src/main/java/org/apache/accumulo/core/security/thrift/SecurityErrorCode.java
+++ /dev/null
@@ -1,112 +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.
- */
-/**
- * Autogenerated by Thrift Compiler (0.9.0)
- *
- * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
- *  @generated
- */
-package org.apache.accumulo.core.security.thrift;
-
-
-import java.util.Map;
-import java.util.HashMap;
-import org.apache.thrift.TEnum;
-
-/**
- * @deprecated since 1.5, see org.apache.accumulo.core.client.impl.thrift.SecurityErrorCode
- */
-@SuppressWarnings("all") public enum SecurityErrorCode implements org.apache.thrift.TEnum {
-  DEFAULT_SECURITY_ERROR(0),
-  BAD_CREDENTIALS(1),
-  PERMISSION_DENIED(2),
-  USER_DOESNT_EXIST(3),
-  CONNECTION_ERROR(4),
-  USER_EXISTS(5),
-  GRANT_INVALID(6),
-  BAD_AUTHORIZATIONS(7),
-  INVALID_INSTANCEID(8),
-  TABLE_DOESNT_EXIST(9),
-  UNSUPPORTED_OPERATION(10),
-  INVALID_TOKEN(11),
-  AUTHENTICATOR_FAILED(12),
-  AUTHORIZOR_FAILED(13),
-  PERMISSIONHANDLER_FAILED(14),
-  TOKEN_EXPIRED(15),
-  SERIALIZATION_ERROR(16),
-  INSUFFICIENT_PROPERTIES(17);
-
-  private final int value;
-
-  private SecurityErrorCode(int value) {
-    this.value = value;
-  }
-
-  /**
-   * Get the integer value of this enum value, as defined in the Thrift IDL.
-   */
-  public int getValue() {
-    return value;
-  }
-
-  /**
-   * Find a the enum type by its integer value, as defined in the Thrift IDL.
-   * @return null if the value is not found.
-   */
-  public static SecurityErrorCode findByValue(int value) { 
-    switch (value) {
-      case 0:
-        return DEFAULT_SECURITY_ERROR;
-      case 1:
-        return BAD_CREDENTIALS;
-      case 2:
-        return PERMISSION_DENIED;
-      case 3:
-        return USER_DOESNT_EXIST;
-      case 4:
-        return CONNECTION_ERROR;
-      case 5:
-        return USER_EXISTS;
-      case 6:
-        return GRANT_INVALID;
-      case 7:
-        return BAD_AUTHORIZATIONS;
-      case 8:
-        return INVALID_INSTANCEID;
-      case 9:
-        return TABLE_DOESNT_EXIST;
-      case 10:
-        return UNSUPPORTED_OPERATION;
-      case 11:
-        return INVALID_TOKEN;
-      case 12:
-        return AUTHENTICATOR_FAILED;
-      case 13:
-        return AUTHORIZOR_FAILED;
-      case 14:
-        return PERMISSIONHANDLER_FAILED;
-      case 15:
-        return TOKEN_EXPIRED;
-      case 16:
-        return SERIALIZATION_ERROR;
-      case 17:
-        return INSUFFICIENT_PROPERTIES;
-      default:
-        return null;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/a69a9d68/core/src/main/java/org/apache/accumulo/core/security/thrift/ThriftSecurityException.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/security/thrift/ThriftSecurityException.java b/core/src/main/java/org/apache/accumulo/core/security/thrift/ThriftSecurityException.java
deleted file mode 100644
index c7b2fcb..0000000
--- a/core/src/main/java/org/apache/accumulo/core/security/thrift/ThriftSecurityException.java
+++ /dev/null
@@ -1,521 +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.
- */
-/**
- * Autogenerated by Thrift Compiler (0.9.0)
- *
- * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
- *  @generated
- */
-package org.apache.accumulo.core.security.thrift;
-
-import org.apache.thrift.scheme.IScheme;
-import org.apache.thrift.scheme.SchemeFactory;
-import org.apache.thrift.scheme.StandardScheme;
-
-import org.apache.thrift.scheme.TupleScheme;
-import org.apache.thrift.protocol.TTupleProtocol;
-import org.apache.thrift.protocol.TProtocolException;
-import org.apache.thrift.EncodingUtils;
-import org.apache.thrift.TException;
-import java.util.List;
-import java.util.ArrayList;
-import java.util.Map;
-import java.util.HashMap;
-import java.util.EnumMap;
-import java.util.Set;
-import java.util.HashSet;
-import java.util.EnumSet;
-import java.util.Collections;
-import java.util.BitSet;
-import java.nio.ByteBuffer;
-import java.util.Arrays;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * @deprecated since 1.5, see org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException
- */
-@SuppressWarnings("all") public class ThriftSecurityException extends TException implements org.apache.thrift.TBase<ThriftSecurityException, ThriftSecurityException._Fields>, java.io.Serializable, Cloneable {
-  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("ThriftSecurityException");
-
-  private static final org.apache.thrift.protocol.TField USER_FIELD_DESC = new org.apache.thrift.protocol.TField("user", org.apache.thrift.protocol.TType.STRING, (short)1);
-  private static final org.apache.thrift.protocol.TField CODE_FIELD_DESC = new org.apache.thrift.protocol.TField("code", org.apache.thrift.protocol.TType.I32, (short)2);
-
-  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
-  static {
-    schemes.put(StandardScheme.class, new ThriftSecurityExceptionStandardSchemeFactory());
-    schemes.put(TupleScheme.class, new ThriftSecurityExceptionTupleSchemeFactory());
-  }
-
-  public String user; // required
-  /**
-   * 
-   * @see SecurityErrorCode
-   */
-  public SecurityErrorCode code; // required
-
-  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
-  @SuppressWarnings("all") public enum _Fields implements org.apache.thrift.TFieldIdEnum {
-    USER((short)1, "user"),
-    /**
-     * 
-     * @see SecurityErrorCode
-     */
-    CODE((short)2, "code");
-
-    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
-
-    static {
-      for (_Fields field : EnumSet.allOf(_Fields.class)) {
-        byName.put(field.getFieldName(), field);
-      }
-    }
-
-    /**
-     * Find the _Fields constant that matches fieldId, or null if its not found.
-     */
-    public static _Fields findByThriftId(int fieldId) {
-      switch(fieldId) {
-        case 1: // USER
-          return USER;
-        case 2: // CODE
-          return CODE;
-        default:
-          return null;
-      }
-    }
-
-    /**
-     * Find the _Fields constant that matches fieldId, throwing an exception
-     * if it is not found.
-     */
-    public static _Fields findByThriftIdOrThrow(int fieldId) {
-      _Fields fields = findByThriftId(fieldId);
-      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
-      return fields;
-    }
-
-    /**
-     * Find the _Fields constant that matches name, or null if its not found.
-     */
-    public static _Fields findByName(String name) {
-      return byName.get(name);
-    }
-
-    private final short _thriftId;
-    private final String _fieldName;
-
-    _Fields(short thriftId, String fieldName) {
-      _thriftId = thriftId;
-      _fieldName = fieldName;
-    }
-
-    public short getThriftFieldId() {
-      return _thriftId;
-    }
-
-    public String getFieldName() {
-      return _fieldName;
-    }
-  }
-
-  // isset id assignments
-  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
-  static {
-    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
-    tmpMap.put(_Fields.USER, new org.apache.thrift.meta_data.FieldMetaData("user", org.apache.thrift.TFieldRequirementType.DEFAULT, 
-        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
-    tmpMap.put(_Fields.CODE, new org.apache.thrift.meta_data.FieldMetaData("code", org.apache.thrift.TFieldRequirementType.DEFAULT, 
-        new org.apache.thrift.meta_data.EnumMetaData(org.apache.thrift.protocol.TType.ENUM, SecurityErrorCode.class)));
-    metaDataMap = Collections.unmodifiableMap(tmpMap);
-    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(ThriftSecurityException.class, metaDataMap);
-  }
-
-  public ThriftSecurityException() {
-  }
-
-  public ThriftSecurityException(
-    String user,
-    SecurityErrorCode code)
-  {
-    this();
-    this.user = user;
-    this.code = code;
-  }
-
-  /**
-   * Performs a deep copy on <i>other</i>.
-   */
-  public ThriftSecurityException(ThriftSecurityException other) {
-    if (other.isSetUser()) {
-      this.user = other.user;
-    }
-    if (other.isSetCode()) {
-      this.code = other.code;
-    }
-  }
-
-  public ThriftSecurityException deepCopy() {
-    return new ThriftSecurityException(this);
-  }
-
-  @Override
-  public void clear() {
-    this.user = null;
-    this.code = null;
-  }
-
-  public String getUser() {
-    return this.user;
-  }
-
-  public ThriftSecurityException setUser(String user) {
-    this.user = user;
-    return this;
-  }
-
-  public void unsetUser() {
-    this.user = null;
-  }
-
-  /** Returns true if field user is set (has been assigned a value) and false otherwise */
-  public boolean isSetUser() {
-    return this.user != null;
-  }
-
-  public void setUserIsSet(boolean value) {
-    if (!value) {
-      this.user = null;
-    }
-  }
-
-  /**
-   * 
-   * @see SecurityErrorCode
-   */
-  public SecurityErrorCode getCode() {
-    return this.code;
-  }
-
-  /**
-   * 
-   * @see SecurityErrorCode
-   */
-  public ThriftSecurityException setCode(SecurityErrorCode code) {
-    this.code = code;
-    return this;
-  }
-
-  public void unsetCode() {
-    this.code = null;
-  }
-
-  /** Returns true if field code is set (has been assigned a value) and false otherwise */
-  public boolean isSetCode() {
-    return this.code != null;
-  }
-
-  public void setCodeIsSet(boolean value) {
-    if (!value) {
-      this.code = null;
-    }
-  }
-
-  public void setFieldValue(_Fields field, Object value) {
-    switch (field) {
-    case USER:
-      if (value == null) {
-        unsetUser();
-      } else {
-        setUser((String)value);
-      }
-      break;
-
-    case CODE:
-      if (value == null) {
-        unsetCode();
-      } else {
-        setCode((SecurityErrorCode)value);
-      }
-      break;
-
-    }
-  }
-
-  public Object getFieldValue(_Fields field) {
-    switch (field) {
-    case USER:
-      return getUser();
-
-    case CODE:
-      return getCode();
-
-    }
-    throw new IllegalStateException();
-  }
-
-  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
-  public boolean isSet(_Fields field) {
-    if (field == null) {
-      throw new IllegalArgumentException();
-    }
-
-    switch (field) {
-    case USER:
-      return isSetUser();
-    case CODE:
-      return isSetCode();
-    }
-    throw new IllegalStateException();
-  }
-
-  @Override
-  public boolean equals(Object that) {
-    if (that == null)
-      return false;
-    if (that instanceof ThriftSecurityException)
-      return this.equals((ThriftSecurityException)that);
-    return false;
-  }
-
-  public boolean equals(ThriftSecurityException that) {
-    if (that == null)
-      return false;
-
-    boolean this_present_user = true && this.isSetUser();
-    boolean that_present_user = true && that.isSetUser();
-    if (this_present_user || that_present_user) {
-      if (!(this_present_user && that_present_user))
-        return false;
-      if (!this.user.equals(that.user))
-        return false;
-    }
-
-    boolean this_present_code = true && this.isSetCode();
-    boolean that_present_code = true && that.isSetCode();
-    if (this_present_code || that_present_code) {
-      if (!(this_present_code && that_present_code))
-        return false;
-      if (!this.code.equals(that.code))
-        return false;
-    }
-
-    return true;
-  }
-
-  @Override
-  public int hashCode() {
-    return 0;
-  }
-
-  public int compareTo(ThriftSecurityException other) {
-    if (!getClass().equals(other.getClass())) {
-      return getClass().getName().compareTo(other.getClass().getName());
-    }
-
-    int lastComparison = 0;
-    ThriftSecurityException typedOther = (ThriftSecurityException)other;
-
-    lastComparison = Boolean.valueOf(isSetUser()).compareTo(typedOther.isSetUser());
-    if (lastComparison != 0) {
-      return lastComparison;
-    }
-    if (isSetUser()) {
-      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.user, typedOther.user);
-      if (lastComparison != 0) {
-        return lastComparison;
-      }
-    }
-    lastComparison = Boolean.valueOf(isSetCode()).compareTo(typedOther.isSetCode());
-    if (lastComparison != 0) {
-      return lastComparison;
-    }
-    if (isSetCode()) {
-      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.code, typedOther.code);
-      if (lastComparison != 0) {
-        return lastComparison;
-      }
-    }
-    return 0;
-  }
-
-  public _Fields fieldForId(int fieldId) {
-    return _Fields.findByThriftId(fieldId);
-  }
-
-  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
-    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
-  }
-
-  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
-    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
-  }
-
-  @Override
-  public String toString() {
-    StringBuilder sb = new StringBuilder("ThriftSecurityException(");
-    boolean first = true;
-
-    sb.append("user:");
-    if (this.user == null) {
-      sb.append("null");
-    } else {
-      sb.append(this.user);
-    }
-    first = false;
-    if (!first) sb.append(", ");
-    sb.append("code:");
-    if (this.code == null) {
-      sb.append("null");
-    } else {
-      sb.append(this.code);
-    }
-    first = false;
-    sb.append(")");
-    return sb.toString();
-  }
-
-  public void validate() throws org.apache.thrift.TException {
-    // check for required fields
-    // check for sub-struct validity
-  }
-
-  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
-    try {
-      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
-    } catch (org.apache.thrift.TException te) {
-      throw new java.io.IOException(te);
-    }
-  }
-
-  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
-    try {
-      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
-    } catch (org.apache.thrift.TException te) {
-      throw new java.io.IOException(te);
-    }
-  }
-
-  private static class ThriftSecurityExceptionStandardSchemeFactory implements SchemeFactory {
-    public ThriftSecurityExceptionStandardScheme getScheme() {
-      return new ThriftSecurityExceptionStandardScheme();
-    }
-  }
-
-  private static class ThriftSecurityExceptionStandardScheme extends StandardScheme<ThriftSecurityException> {
-
-    public void read(org.apache.thrift.protocol.TProtocol iprot, ThriftSecurityException struct) throws org.apache.thrift.TException {
-      org.apache.thrift.protocol.TField schemeField;
-      iprot.readStructBegin();
-      while (true)
-      {
-        schemeField = iprot.readFieldBegin();
-        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
-          break;
-        }
-        switch (schemeField.id) {
-          case 1: // USER
-            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
-              struct.user = iprot.readString();
-              struct.setUserIsSet(true);
-            } else { 
-              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
-            }
-            break;
-          case 2: // CODE
-            if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
-              struct.code = SecurityErrorCode.findByValue(iprot.readI32());
-              struct.setCodeIsSet(true);
-            } else { 
-              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
-            }
-            break;
-          default:
-            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
-        }
-        iprot.readFieldEnd();
-      }
-      iprot.readStructEnd();
-
-      // check for required fields of primitive type, which can't be checked in the validate method
-      struct.validate();
-    }
-
-    public void write(org.apache.thrift.protocol.TProtocol oprot, ThriftSecurityException struct) throws org.apache.thrift.TException {
-      struct.validate();
-
-      oprot.writeStructBegin(STRUCT_DESC);
-      if (struct.user != null) {
-        oprot.writeFieldBegin(USER_FIELD_DESC);
-        oprot.writeString(struct.user);
-        oprot.writeFieldEnd();
-      }
-      if (struct.code != null) {
-        oprot.writeFieldBegin(CODE_FIELD_DESC);
-        oprot.writeI32(struct.code.getValue());
-        oprot.writeFieldEnd();
-      }
-      oprot.writeFieldStop();
-      oprot.writeStructEnd();
-    }
-
-  }
-
-  private static class ThriftSecurityExceptionTupleSchemeFactory implements SchemeFactory {
-    public ThriftSecurityExceptionTupleScheme getScheme() {
-      return new ThriftSecurityExceptionTupleScheme();
-    }
-  }
-
-  private static class ThriftSecurityExceptionTupleScheme extends TupleScheme<ThriftSecurityException> {
-
-    @Override
-    public void write(org.apache.thrift.protocol.TProtocol prot, ThriftSecurityException struct) throws org.apache.thrift.TException {
-      TTupleProtocol oprot = (TTupleProtocol) prot;
-      BitSet optionals = new BitSet();
-      if (struct.isSetUser()) {
-        optionals.set(0);
-      }
-      if (struct.isSetCode()) {
-        optionals.set(1);
-      }
-      oprot.writeBitSet(optionals, 2);
-      if (struct.isSetUser()) {
-        oprot.writeString(struct.user);
-      }
-      if (struct.isSetCode()) {
-        oprot.writeI32(struct.code.getValue());
-      }
-    }
-
-    @Override
-    public void read(org.apache.thrift.protocol.TProtocol prot, ThriftSecurityException struct) throws org.apache.thrift.TException {
-      TTupleProtocol iprot = (TTupleProtocol) prot;
-      BitSet incoming = iprot.readBitSet(2);
-      if (incoming.get(0)) {
-        struct.user = iprot.readString();
-        struct.setUserIsSet(true);
-      }
-      if (incoming.get(1)) {
-        struct.code = SecurityErrorCode.findByValue(iprot.readI32());
-        struct.setCodeIsSet(true);
-      }
-    }
-  }
-
-}
-

http://git-wip-us.apache.org/repos/asf/accumulo/blob/a69a9d68/core/src/main/java/org/apache/accumulo/core/util/shell/Shell.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/util/shell/Shell.java b/core/src/main/java/org/apache/accumulo/core/util/shell/Shell.java
index ef3e519..2b8d96e 100644
--- a/core/src/main/java/org/apache/accumulo/core/util/shell/Shell.java
+++ b/core/src/main/java/org/apache/accumulo/core/util/shell/Shell.java
@@ -140,6 +140,7 @@ import org.apache.accumulo.core.util.shell.commands.UserCommand;
 import org.apache.accumulo.core.util.shell.commands.UserPermissionsCommand;
 import org.apache.accumulo.core.util.shell.commands.UsersCommand;
 import org.apache.accumulo.core.util.shell.commands.WhoAmICommand;
+import org.apache.accumulo.core.zookeeper.ZooUtil;
 import org.apache.accumulo.fate.zookeeper.ZooReader;
 import org.apache.commons.cli.BasicParser;
 import org.apache.commons.cli.CommandLine;
@@ -396,11 +397,10 @@ public class Shell extends ShellOptions {
     }
   }
   
-  @SuppressWarnings("deprecation")
   private static Instance getDefaultInstance(AccumuloConfiguration conf) {
     String keepers = conf.get(Property.INSTANCE_ZK_HOST);
     Path instanceDir = new Path(conf.get(Property.INSTANCE_DFS_DIR), "instance_id");
-    return new ZooKeeperInstance(UUID.fromString(ZooKeeperInstance.getInstanceIDFromHdfs(instanceDir)), keepers);
+    return new ZooKeeperInstance(UUID.fromString(ZooUtil.getInstanceIDFromHdfs(instanceDir)), keepers);
   }
   
   public Connector getConnector() {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/a69a9d68/core/src/main/java/org/apache/accumulo/core/util/shell/commands/FateCommand.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/util/shell/commands/FateCommand.java b/core/src/main/java/org/apache/accumulo/core/util/shell/commands/FateCommand.java
index 8a4d0ce..6bf4f30 100644
--- a/core/src/main/java/org/apache/accumulo/core/util/shell/commands/FateCommand.java
+++ b/core/src/main/java/org/apache/accumulo/core/util/shell/commands/FateCommand.java
@@ -137,16 +137,15 @@ public class FateCommand extends Command {
     return failedCommand ? 1 : 0;
   }
   
-  @SuppressWarnings("deprecation")
   protected synchronized IZooReaderWriter getZooReaderWriter(Instance instance, String secret) {
-
+    
     if (secret == null) {
+      @SuppressWarnings("deprecation")
       AccumuloConfiguration conf = AccumuloConfiguration.getSiteConfiguration();
       secret = conf.get(Property.INSTANCE_SECRET);
     }
     
-    return new ZooReaderWriter(instance.getZooKeepers(), instance.getZooKeepersSessionTimeOut(), SCHEME,
-        (USER + ":" + secret).getBytes());
+    return new ZooReaderWriter(instance.getZooKeepers(), instance.getZooKeepersSessionTimeOut(), SCHEME, (USER + ":" + secret).getBytes());
   }
   
   @Override