You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@accumulo.apache.org by ct...@apache.org on 2018/10/12 14:47:32 UTC

[accumulo] branch master updated: Remove aggregators (#690)

This is an automated email from the ASF dual-hosted git repository.

ctubbsii pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/accumulo.git


The following commit(s) were added to refs/heads/master by this push:
     new 489213e  Remove aggregators (#690)
489213e is described below

commit 489213ef40eede44a491c607376d3c9e8a88ab46
Author: Christopher Tubbs <ct...@apache.org>
AuthorDate: Fri Oct 12 10:47:27 2018 -0400

    Remove aggregators (#690)
    
    Primarily, this removes aggregators, which have been deprecated since
    1.4 and have been replaced by combiners. This removal sets us up to
    provide aggregators as a separate jar artifact, outside of the main
    Accumulo build, if there is a continued demand for them. In future, all
    such "user library" type add-ons to Accumulo could be provided as
    separate, optional component jars.
    
    Also cleanup other minor deprecated items.
    
    Also create a delegating ConnectorImpl to allow removal of old deprecated
    methods from new AccumuloClientImpl. This ConnectorImpl reuses the
    implementation moved to AccumuloClientImpl via a delegate field, but
    keeps the old ConnectorImpl methods that weren't carried over to the new
    interface.
---
 core/pom.xml                                       |   5 -
 .../java/org/apache/accumulo/core/Constants.java   |   8 -
 .../core/client/ClientSideIteratorScanner.java     |  18 -
 .../org/apache/accumulo/core/client/Connector.java |  10 +-
 .../accumulo/core/client/IsolatedScanner.java      |  18 -
 .../org/apache/accumulo/core/client/Scanner.java   |  20 -
 .../accumulo/core/client/ZooKeeperInstance.java    |   7 +-
 .../core/client/admin/SecurityOperations.java      |  86 ----
 .../core/client/admin/TableOperations.java         |  61 ---
 .../core/client/impl/AccumuloClientImpl.java       |  43 +-
 .../accumulo/core/client/impl/ConnectorImpl.java   | 156 +++++++
 .../accumulo/core/client/impl/OfflineScanner.java  |  14 -
 .../accumulo/core/client/impl/ScannerImpl.java     |  22 +-
 .../accumulo/core/client/impl/ScannerIterator.java |   4 +-
 .../core/client/impl/SecurityOperationsImpl.java   |  34 --
 .../core/client/impl/TableOperationsImpl.java      |  44 --
 .../accumulo/core/client/impl/ThriftScanner.java   |   2 +-
 .../accumulo/core/client/rfile/RFileScanner.java   |  19 -
 .../core/client/security/tokens/KerberosToken.java |  21 -
 .../accumulo/core/constraints/Constraint.java      |  10 -
 .../org/apache/accumulo/core/data/PartialKey.java  |  18 -
 .../java/org/apache/accumulo/core/data/Value.java  |  35 --
 .../apache/accumulo/core/data/impl/KeyExtent.java  |  78 ----
 .../core/iterators/AggregatingIterator.java        | 221 ----------
 .../core/iterators/FamilyIntersectingIterator.java |  30 --
 .../core/iterators/aggregation/Aggregator.java     |  31 --
 .../core/iterators/aggregation/LongSummation.java  |  81 ----
 .../iterators/aggregation/NumArraySummation.java   |  97 -----
 .../core/iterators/aggregation/NumSummation.java   |  92 ----
 .../core/iterators/aggregation/StringMax.java      |  48 ---
 .../core/iterators/aggregation/StringMin.java      |  48 ---
 .../iterators/aggregation/StringSummation.java     |  46 --
 .../aggregation/conf/AggregatorConfiguration.java  |  35 --
 .../iterators/aggregation/conf/AggregatorSet.java  |  43 --
 .../iterators/conf/PerColumnIteratorConfig.java    |  83 ----
 .../core/iterators/user/IntersectingIterator.java  |  25 --
 .../java/org/apache/accumulo/core/trace/Trace.java |  19 -
 .../client/impl/TableOperationsHelperTest.java     |  27 --
 .../apache/accumulo/core/data/KeyExtentTest.java   |  58 ---
 .../org/apache/accumulo/core/data/ValueTest.java   |  26 --
 .../core/iterators/AggregatingIteratorTest.java    | 474 ---------------------
 .../iterators/aggregation/NumSummationTest.java    | 149 -------
 .../conf/AggregatorConfigurationTest.java          |  79 ----
 .../org/apache/accumulo/proxy/ProxyServer.java     |   3 +-
 .../org/apache/accumulo/tracer/TracerTest.java     |   4 +-
 .../accumulo/tserver/TservConstraintEnv.java       |  14 -
 .../accumulo/shell/commands/SetIterCommand.java    |  38 +-
 .../shell/commands/SetIterCommandTest.java         |   1 -
 .../accumulo/test/NewTableConfigurationIT.java     | 159 -------
 .../accumulo/test/functional/AccumuloClientIT.java |   9 +
 50 files changed, 186 insertions(+), 2487 deletions(-)

diff --git a/core/pom.xml b/core/pom.xml
index 4535f50..d9ca17a 100644
--- a/core/pom.xml
+++ b/core/pom.xml
@@ -128,11 +128,6 @@
       <scope>test</scope>
     </dependency>
     <dependency>
-      <groupId>org.hamcrest</groupId>
-      <artifactId>hamcrest-core</artifactId>
-      <scope>test</scope>
-    </dependency>
-    <dependency>
       <groupId>org.slf4j</groupId>
       <artifactId>slf4j-log4j12</artifactId>
       <scope>test</scope>
diff --git a/core/src/main/java/org/apache/accumulo/core/Constants.java b/core/src/main/java/org/apache/accumulo/core/Constants.java
index f8cf8e2..46fcb7f 100644
--- a/core/src/main/java/org/apache/accumulo/core/Constants.java
+++ b/core/src/main/java/org/apache/accumulo/core/Constants.java
@@ -22,8 +22,6 @@ import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
 
-import org.apache.accumulo.core.security.Authorizations;
-
 public class Constants {
 
   public static final String VERSION = FilteredConstants.VERSION;
@@ -111,12 +109,6 @@ public class Constants {
   // Security configuration
   public static final String PW_HASH_ALGORITHM = "SHA-256";
 
-  /**
-   * @deprecated since 1.6.0; Use {@link Authorizations#EMPTY} instead
-   */
-  @Deprecated
-  public static final Authorizations NO_AUTHS = Authorizations.EMPTY;
-
   public static final int MAX_DATA_TO_PRINT = 64;
   public static final String CORE_PACKAGE_NAME = "org.apache.accumulo.core";
   public static final String MAPFILE_EXTENSION = "map";
diff --git a/core/src/main/java/org/apache/accumulo/core/client/ClientSideIteratorScanner.java b/core/src/main/java/org/apache/accumulo/core/client/ClientSideIteratorScanner.java
index c9ebb8e..e65913d 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/ClientSideIteratorScanner.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/ClientSideIteratorScanner.java
@@ -291,24 +291,6 @@ public class ClientSideIteratorScanner extends ScannerOptions implements Scanner
     return smi.scanner.getAuthorizations();
   }
 
-  @Deprecated
-  @Override
-  public void setTimeOut(int timeOut) {
-    if (timeOut == Integer.MAX_VALUE)
-      setTimeout(Long.MAX_VALUE, TimeUnit.MILLISECONDS);
-    else
-      setTimeout(timeOut, TimeUnit.SECONDS);
-  }
-
-  @Deprecated
-  @Override
-  public int getTimeOut() {
-    long timeout = getTimeout(TimeUnit.SECONDS);
-    if (timeout >= Integer.MAX_VALUE)
-      return Integer.MAX_VALUE;
-    return (int) timeout;
-  }
-
   @Override
   public void setRange(final Range range) {
     this.range = range;
diff --git a/core/src/main/java/org/apache/accumulo/core/client/Connector.java b/core/src/main/java/org/apache/accumulo/core/client/Connector.java
index 33c4af7..42d61ac 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/Connector.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/Connector.java
@@ -21,6 +21,8 @@ import org.apache.accumulo.core.client.admin.NamespaceOperations;
 import org.apache.accumulo.core.client.admin.ReplicationOperations;
 import org.apache.accumulo.core.client.admin.SecurityOperations;
 import org.apache.accumulo.core.client.admin.TableOperations;
+import org.apache.accumulo.core.client.impl.AccumuloClientImpl;
+import org.apache.accumulo.core.client.impl.ConnectorImpl;
 import org.apache.accumulo.core.security.Authorizations;
 
 /**
@@ -30,9 +32,10 @@ import org.apache.accumulo.core.security.Authorizations;
  * The Connector enforces security on the client side by forcing all API calls to be accompanied by
  * user credentials.
  *
- * @deprecated since 2.0.0. Use {@link AccumuloClient} for wiriting new code. Connector is available
+ * @deprecated since 2.0.0. Use {@link AccumuloClient} for writing new code. Connector is available
  *             for existing code. Use {@link #from(AccumuloClient)} as a bridge between the two.
  */
+@Deprecated
 public abstract class Connector {
 
   /**
@@ -106,7 +109,6 @@ public abstract class Connector {
    * @return BatchDeleter object for configuring and deleting
    * @since 1.5.0
    */
-
   public abstract BatchDeleter createBatchDeleter(String tableName, Authorizations authorizations,
       int numQueryThreads, BatchWriterConfig config) throws TableNotFoundException;
 
@@ -221,9 +223,7 @@ public abstract class Connector {
    * Accessor method for internal instance object.
    *
    * @return the internal instance object
-   * @deprecated since 2.0.0, use {@link AccumuloClient#info()} instead
    */
-  @Deprecated
   public abstract Instance getInstance();
 
   /**
@@ -277,6 +277,6 @@ public abstract class Connector {
    * @since 2.0
    */
   public static Connector from(AccumuloClient client) {
-    return (Connector) client;
+    return new ConnectorImpl((AccumuloClientImpl) client);
   }
 }
diff --git a/core/src/main/java/org/apache/accumulo/core/client/IsolatedScanner.java b/core/src/main/java/org/apache/accumulo/core/client/IsolatedScanner.java
index 78b22ba..8cd100d 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/IsolatedScanner.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/IsolatedScanner.java
@@ -239,24 +239,6 @@ public class IsolatedScanner extends ScannerOptions implements Scanner {
         bufferFactory);
   }
 
-  @Deprecated
-  @Override
-  public void setTimeOut(int timeOut) {
-    if (timeOut == Integer.MAX_VALUE)
-      setTimeout(Long.MAX_VALUE, TimeUnit.MILLISECONDS);
-    else
-      setTimeout(timeOut, TimeUnit.SECONDS);
-  }
-
-  @Deprecated
-  @Override
-  public int getTimeOut() {
-    long timeout = getTimeout(TimeUnit.SECONDS);
-    if (timeout >= Integer.MAX_VALUE)
-      return Integer.MAX_VALUE;
-    return (int) timeout;
-  }
-
   @Override
   public void setRange(Range range) {
     this.range = range;
diff --git a/core/src/main/java/org/apache/accumulo/core/client/Scanner.java b/core/src/main/java/org/apache/accumulo/core/client/Scanner.java
index 03139e4..b784e4c 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/Scanner.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/Scanner.java
@@ -29,26 +29,6 @@ import org.apache.accumulo.core.data.Range;
 public interface Scanner extends ScannerBase {
 
   /**
-   * This setting determines how long a scanner will automatically retry when a failure occurs. By
-   * default a scanner will retry forever.
-   *
-   * @param timeOut
-   *          in seconds
-   * @deprecated Since 1.5. See {@link ScannerBase#setTimeout(long, java.util.concurrent.TimeUnit)}
-   */
-  @Deprecated
-  void setTimeOut(int timeOut);
-
-  /**
-   * Returns the setting for how long a scanner will automatically retry when a failure occurs.
-   *
-   * @return the timeout configured for this scanner
-   * @deprecated Since 1.5. See {@link ScannerBase#getTimeout(java.util.concurrent.TimeUnit)}
-   */
-  @Deprecated
-  int getTimeOut();
-
-  /**
    * Sets the range of keys to scan over.
    *
    * @param range
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 bad3a78..3799138 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
@@ -30,6 +30,7 @@ import org.apache.accumulo.core.client.impl.AccumuloClientImpl;
 import org.apache.accumulo.core.client.impl.ClientConfConverter;
 import org.apache.accumulo.core.client.impl.ClientContext;
 import org.apache.accumulo.core.client.impl.ClientInfoImpl;
+import org.apache.accumulo.core.client.impl.ConnectorImpl;
 import org.apache.accumulo.core.client.impl.InstanceOperationsImpl;
 import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
 import org.apache.accumulo.core.conf.ClientProperty;
@@ -82,7 +83,6 @@ public class ZooKeeperInstance implements Instance {
    * @param zooKeepers
    *          A comma separated list of zoo keeper server locations. Each location can contain an
    *          optional port, of the format host:port.
-   * @deprecated since 2.0.0; use {@link Accumulo#newClient()} instead
    */
   public ZooKeeperInstance(String instanceName, String zooKeepers) {
     this(ClientConfiguration.loadDefault().withInstance(instanceName).withZkHosts(zooKeepers));
@@ -113,9 +113,7 @@ public class ZooKeeperInstance implements Instance {
    *          {@link ClientConfiguration} which extends Configuration with convenience methods
    *          specific to Accumulo.
    * @since 1.9.0
-   * @deprecated since 2.0.0; use {@link Accumulo#newClient()} instead
    */
-  @Deprecated
   public ZooKeeperInstance(ClientConfiguration config) {
     this(config, new ZooCacheFactory());
   }
@@ -225,7 +223,8 @@ public class ZooKeeperInstance implements Instance {
     Properties properties = ClientConfConverter.toProperties(clientConf);
     properties.setProperty(ClientProperty.AUTH_PRINCIPAL.getKey(), principal);
     properties.setProperty(ClientProperty.INSTANCE_NAME.getKey(), getInstanceName());
-    return new AccumuloClientImpl(new ClientContext(new ClientInfoImpl(properties, token)));
+    return new ConnectorImpl(
+        new AccumuloClientImpl(new ClientContext(new ClientInfoImpl(properties, token))));
   }
 
   @Override
diff --git a/core/src/main/java/org/apache/accumulo/core/client/admin/SecurityOperations.java b/core/src/main/java/org/apache/accumulo/core/client/admin/SecurityOperations.java
index 066fd58..5deb22a 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/admin/SecurityOperations.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/admin/SecurityOperations.java
@@ -36,26 +36,6 @@ public interface SecurityOperations {
   /**
    * Create a user
    *
-   * @param user
-   *          the name of the user to create
-   * @param password
-   *          the plaintext password for the user
-   * @param authorizations
-   *          the authorizations that the user has for scanning
-   * @throws AccumuloException
-   *           if a general error occurs
-   * @throws AccumuloSecurityException
-   *           if the user does not have permission to create a user
-   * @deprecated since 1.5.0; use {@link #createLocalUser(String, PasswordToken)} or the user
-   *             management functions of your configured authenticator instead.
-   */
-  @Deprecated
-  void createUser(String user, byte[] password, Authorizations authorizations)
-      throws AccumuloException, AccumuloSecurityException;
-
-  /**
-   * Create a user
-   *
    * @param principal
    *          the name of the user to create
    * @param password
@@ -72,21 +52,6 @@ public interface SecurityOperations {
   /**
    * Delete a user
    *
-   * @param user
-   *          the user name to delete
-   * @throws AccumuloException
-   *           if a general error occurs
-   * @throws AccumuloSecurityException
-   *           if the user does not have permission to delete a user
-   * @deprecated since 1.5.0; use {@link #dropLocalUser(String)} or the user management functions of
-   *             your configured authenticator instead.
-   */
-  @Deprecated
-  void dropUser(String user) throws AccumuloException, AccumuloSecurityException;
-
-  /**
-   * Delete a user
-   *
    * @param principal
    *          the user name to delete
    * @throws AccumuloException
@@ -100,25 +65,6 @@ public interface SecurityOperations {
   /**
    * Verify a username/password combination is valid
    *
-   * @param user
-   *          the name of the user to authenticate
-   * @param password
-   *          the plaintext password for the user
-   * @return true if the user asking is allowed to know and the specified user/password is valid,
-   *         false otherwise
-   * @throws AccumuloException
-   *           if a general error occurs
-   * @throws AccumuloSecurityException
-   *           if the user does not have permission to ask
-   * @deprecated since 1.5.0; use {@link #authenticateUser(String, AuthenticationToken)} instead.
-   */
-  @Deprecated
-  boolean authenticateUser(String user, byte[] password)
-      throws AccumuloException, AccumuloSecurityException;
-
-  /**
-   * Verify a username/password combination is valid
-   *
    * @param principal
    *          the name of the user to authenticate
    * @param token
@@ -137,24 +83,6 @@ public interface SecurityOperations {
   /**
    * Set the user's password
    *
-   * @param user
-   *          the name of the user to modify
-   * @param password
-   *          the plaintext password for the user
-   * @throws AccumuloException
-   *           if a general error occurs
-   * @throws AccumuloSecurityException
-   *           if the user does not have permission to modify a user
-   * @deprecated since 1.5.0; use {@link #changeLocalUserPassword(String, PasswordToken)} or the
-   *             user management functions of your configured authenticator instead.
-   */
-  @Deprecated
-  void changeUserPassword(String user, byte[] password)
-      throws AccumuloException, AccumuloSecurityException;
-
-  /**
-   * Set the user's password
-   *
    * @param principal
    *          the name of the user to modify
    * @param token
@@ -355,20 +283,6 @@ public interface SecurityOperations {
    *           if a general error occurs
    * @throws AccumuloSecurityException
    *           if the user does not have permission to query users
-   * @deprecated since 1.5.0; use {@link #listLocalUsers()} or the user management functions of your
-   *             configured authenticator instead.
-   */
-  @Deprecated
-  Set<String> listUsers() throws AccumuloException, AccumuloSecurityException;
-
-  /**
-   * Return a list of users in accumulo
-   *
-   * @return a set of user names
-   * @throws AccumuloException
-   *           if a general error occurs
-   * @throws AccumuloSecurityException
-   *           if the user does not have permission to query users
    * @since 1.5.0
    */
   Set<String> listLocalUsers() throws AccumuloException, AccumuloSecurityException;
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 0eff1ed..06a5141 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
@@ -83,44 +83,6 @@ public interface TableOperations {
   /**
    * @param tableName
    *          the name of the table
-   * @param limitVersion
-   *          Enables/disables the versioning iterator, which will limit the number of Key versions
-   *          kept.
-   * @throws AccumuloException
-   *           if a general error occurs
-   * @throws AccumuloSecurityException
-   *           if the user does not have permission
-   * @throws TableExistsException
-   *           if the table already exists
-   * @deprecated since 1.7.0; use {@link #create(String, NewTableConfiguration)} instead.
-   */
-  @Deprecated
-  void create(String tableName, boolean limitVersion)
-      throws AccumuloException, AccumuloSecurityException, TableExistsException;
-
-  /**
-   * @param tableName
-   *          the name of the table
-   * @param versioningIter
-   *          Enables/disables the versioning iterator, which will limit the number of Key versions
-   *          kept.
-   * @param timeType
-   *          specifies logical or real-time based time recording for entries in 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
-   * @deprecated since 1.7.0; use {@link #create(String, NewTableConfiguration)} instead.
-   */
-  @Deprecated
-  void create(String tableName, boolean versioningIter, TimeType timeType)
-      throws AccumuloException, AccumuloSecurityException, TableExistsException;
-
-  /**
-   * @param tableName
-   *          the name of the table
    * @param ntc
    *          specifies the new table's configuration variable, which are: 1. enable/disable the
    *          versioning iterator, which will limit the number of Key versions kept; 2. specifies
@@ -206,17 +168,6 @@ public interface TableOperations {
    * @return the split points (end-row names) for the table's current split profile
    * @throws TableNotFoundException
    *           if the table does not exist
-   * @deprecated since 1.5.0; use {@link #listSplits(String)} instead.
-   */
-  @Deprecated
-  Collection<Text> getSplits(String tableName) throws TableNotFoundException;
-
-  /**
-   * @param tableName
-   *          the name of the table
-   * @return the split points (end-row names) for the table's current split profile
-   * @throws TableNotFoundException
-   *           if the table does not exist
    * @throws AccumuloException
    *           if a general error occurs
    * @throws AccumuloSecurityException
@@ -231,18 +182,6 @@ public interface TableOperations {
    *          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
-   * @deprecated since 1.5.0; use {@link #listSplits(String, int)} instead.
-   */
-  @Deprecated
-  Collection<Text> getSplits(String tableName, int maxSplits) throws TableNotFoundException;
-
-  /**
-   * @param tableName
-   *          the name of the table
-   * @param maxSplits
-   *          specifies the maximum number of splits to return
    * @throws AccumuloException
    *           if a general error occurs
    * @throws AccumuloSecurityException
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/AccumuloClientImpl.java b/core/src/main/java/org/apache/accumulo/core/client/impl/AccumuloClientImpl.java
index 4987cbd..a6b8e54 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/impl/AccumuloClientImpl.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/AccumuloClientImpl.java
@@ -53,11 +53,10 @@ import org.apache.accumulo.core.trace.Tracer;
 
 import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
 
-public class AccumuloClientImpl extends org.apache.accumulo.core.client.Connector
-    implements AccumuloClient {
+public class AccumuloClientImpl implements AccumuloClient {
   private static final String SYSTEM_TOKEN_NAME = "org.apache.accumulo.server.security."
       + "SystemCredentials$SystemToken";
-  private final ClientContext context;
+  final ClientContext context;
   private final String instanceID;
   private SecurityOperations secops = null;
   private TableOperationsImpl tableops = null;
@@ -92,7 +91,7 @@ public class AccumuloClientImpl extends org.apache.accumulo.core.client.Connecto
     this.namespaceops = new NamespaceOperationsImpl(context, tableops);
   }
 
-  private Table.ID getTableId(String tableName) throws TableNotFoundException {
+  Table.ID getTableId(String tableName) throws TableNotFoundException {
     Table.ID tableId = Tables.getTableId(context, tableName);
     if (Tables.getTableState(context, tableId) == TableState.OFFLINE)
       throw new TableOfflineException(Tables.getTableOfflineMsg(context, tableId));
@@ -100,12 +99,6 @@ public class AccumuloClientImpl extends org.apache.accumulo.core.client.Connecto
   }
 
   @Override
-  @Deprecated
-  public org.apache.accumulo.core.client.Instance getInstance() {
-    return context.getDeprecatedInstance();
-  }
-
-  @Override
   public BatchScanner createBatchScanner(String tableName, Authorizations authorizations,
       int numQueryThreads) throws TableNotFoundException {
     checkArgument(tableName != null, "tableName is null");
@@ -123,18 +116,6 @@ public class AccumuloClientImpl extends org.apache.accumulo.core.client.Connecto
     return createBatchScanner(tableName, authorizations, numQueryThreads);
   }
 
-  @Deprecated
-  @Override
-  public BatchDeleter createBatchDeleter(String tableName, Authorizations authorizations,
-      int numQueryThreads, long maxMemory, long maxLatency, int maxWriteThreads)
-      throws TableNotFoundException {
-    checkArgument(tableName != null, "tableName is null");
-    checkArgument(authorizations != null, "authorizations is null");
-    return new TabletServerBatchDeleter(context, getTableId(tableName), authorizations,
-        numQueryThreads, new BatchWriterConfig().setMaxMemory(maxMemory)
-            .setMaxLatency(maxLatency, TimeUnit.MILLISECONDS).setMaxWriteThreads(maxWriteThreads));
-  }
-
   @Override
   public BatchDeleter createBatchDeleter(String tableName, Authorizations authorizations,
       int numQueryThreads, BatchWriterConfig config) throws TableNotFoundException {
@@ -150,16 +131,6 @@ public class AccumuloClientImpl extends org.apache.accumulo.core.client.Connecto
     return createBatchDeleter(tableName, authorizations, numQueryThreads, new BatchWriterConfig());
   }
 
-  @Deprecated
-  @Override
-  public BatchWriter createBatchWriter(String tableName, long maxMemory, long maxLatency,
-      int maxWriteThreads) throws TableNotFoundException {
-    checkArgument(tableName != null, "tableName is null");
-    return new BatchWriterImpl(context, getTableId(tableName),
-        new BatchWriterConfig().setMaxMemory(maxMemory)
-            .setMaxLatency(maxLatency, TimeUnit.MILLISECONDS).setMaxWriteThreads(maxWriteThreads));
-  }
-
   @Override
   public BatchWriter createBatchWriter(String tableName, BatchWriterConfig config)
       throws TableNotFoundException {
@@ -177,14 +148,6 @@ public class AccumuloClientImpl extends org.apache.accumulo.core.client.Connecto
     return createBatchWriter(tableName, new BatchWriterConfig());
   }
 
-  @Deprecated
-  @Override
-  public MultiTableBatchWriter createMultiTableBatchWriter(long maxMemory, long maxLatency,
-      int maxWriteThreads) {
-    return new MultiTableBatchWriterImpl(context, new BatchWriterConfig().setMaxMemory(maxMemory)
-        .setMaxLatency(maxLatency, TimeUnit.MILLISECONDS).setMaxWriteThreads(maxWriteThreads));
-  }
-
   @Override
   public MultiTableBatchWriter createMultiTableBatchWriter(BatchWriterConfig config) {
     return new MultiTableBatchWriterImpl(context, config.merge(context.getBatchWriterConfig()));
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
new file mode 100644
index 0000000..c48eab6
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/ConnectorImpl.java
@@ -0,0 +1,156 @@
+/*
+ * 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.util.concurrent.TimeUnit;
+
+import org.apache.accumulo.core.client.BatchDeleter;
+import org.apache.accumulo.core.client.BatchScanner;
+import org.apache.accumulo.core.client.BatchWriter;
+import org.apache.accumulo.core.client.BatchWriterConfig;
+import org.apache.accumulo.core.client.ConditionalWriter;
+import org.apache.accumulo.core.client.ConditionalWriterConfig;
+import org.apache.accumulo.core.client.Connector;
+import org.apache.accumulo.core.client.MultiTableBatchWriter;
+import org.apache.accumulo.core.client.Scanner;
+import org.apache.accumulo.core.client.TableNotFoundException;
+import org.apache.accumulo.core.client.admin.InstanceOperations;
+import org.apache.accumulo.core.client.admin.NamespaceOperations;
+import org.apache.accumulo.core.client.admin.ReplicationOperations;
+import org.apache.accumulo.core.client.admin.SecurityOperations;
+import org.apache.accumulo.core.client.admin.TableOperations;
+import org.apache.accumulo.core.security.Authorizations;
+
+/**
+ * This class now delegates to {@link AccumuloClientImpl}, except for the methods which were not
+ * copied over to that.
+ */
+@Deprecated
+public class ConnectorImpl extends Connector {
+
+  private final AccumuloClientImpl impl;
+
+  public ConnectorImpl(AccumuloClientImpl impl) {
+    this.impl = impl;
+  }
+
+  public AccumuloClientImpl getAccumuloClient() {
+    return impl;
+  }
+
+  @Override
+  @Deprecated
+  public org.apache.accumulo.core.client.Instance getInstance() {
+    return impl.context.getDeprecatedInstance();
+  }
+
+  @Override
+  public BatchScanner createBatchScanner(String tableName, Authorizations authorizations,
+      int numQueryThreads) throws TableNotFoundException {
+    return impl.createBatchScanner(tableName, authorizations, numQueryThreads);
+  }
+
+  @Override
+  public BatchDeleter createBatchDeleter(String tableName, Authorizations authorizations,
+      int numQueryThreads, long maxMemory, long maxLatency, int maxWriteThreads)
+      throws TableNotFoundException {
+    checkArgument(tableName != null, "tableName is null");
+    checkArgument(authorizations != null, "authorizations is null");
+    return new TabletServerBatchDeleter(impl.context, impl.getTableId(tableName), authorizations,
+        numQueryThreads, new BatchWriterConfig().setMaxMemory(maxMemory)
+            .setMaxLatency(maxLatency, TimeUnit.MILLISECONDS).setMaxWriteThreads(maxWriteThreads));
+  }
+
+  @Override
+  public BatchDeleter createBatchDeleter(String tableName, Authorizations authorizations,
+      int numQueryThreads, BatchWriterConfig config) throws TableNotFoundException {
+    return impl.createBatchDeleter(tableName, authorizations, numQueryThreads, config);
+  }
+
+  @Override
+  public BatchWriter createBatchWriter(String tableName, long maxMemory, long maxLatency,
+      int maxWriteThreads) throws TableNotFoundException {
+    checkArgument(tableName != null, "tableName is null");
+    return new BatchWriterImpl(impl.context, impl.getTableId(tableName),
+        new BatchWriterConfig().setMaxMemory(maxMemory)
+            .setMaxLatency(maxLatency, TimeUnit.MILLISECONDS).setMaxWriteThreads(maxWriteThreads));
+  }
+
+  @Override
+  public BatchWriter createBatchWriter(String tableName, BatchWriterConfig config)
+      throws TableNotFoundException {
+    return impl.createBatchWriter(tableName, config);
+  }
+
+  @Override
+  public MultiTableBatchWriter createMultiTableBatchWriter(long maxMemory, long maxLatency,
+      int maxWriteThreads) {
+    return new MultiTableBatchWriterImpl(impl.context,
+        new BatchWriterConfig().setMaxMemory(maxMemory)
+            .setMaxLatency(maxLatency, TimeUnit.MILLISECONDS).setMaxWriteThreads(maxWriteThreads));
+  }
+
+  @Override
+  public MultiTableBatchWriter createMultiTableBatchWriter(BatchWriterConfig config) {
+    return impl.createMultiTableBatchWriter(config);
+  }
+
+  @Override
+  public ConditionalWriter createConditionalWriter(String tableName, ConditionalWriterConfig config)
+      throws TableNotFoundException {
+    return impl.createConditionalWriter(tableName, config);
+  }
+
+  @Override
+  public Scanner createScanner(String tableName, Authorizations authorizations)
+      throws TableNotFoundException {
+    return impl.createScanner(tableName, authorizations);
+  }
+
+  @Override
+  public String whoami() {
+    return impl.whoami();
+  }
+
+  @Override
+  public TableOperations tableOperations() {
+    return impl.tableOperations();
+  }
+
+  @Override
+  public NamespaceOperations namespaceOperations() {
+    return impl.namespaceOperations();
+  }
+
+  @Override
+  public SecurityOperations securityOperations() {
+    return impl.securityOperations();
+  }
+
+  @Override
+  public InstanceOperations instanceOperations() {
+    return impl.instanceOperations();
+  }
+
+  @Override
+  public ReplicationOperations replicationOperations() {
+    return impl.replicationOperations();
+  }
+
+}
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/OfflineScanner.java b/core/src/main/java/org/apache/accumulo/core/client/impl/OfflineScanner.java
index d3229d2..79e08f6 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/impl/OfflineScanner.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/OfflineScanner.java
@@ -32,7 +32,6 @@ import org.apache.hadoop.io.Text;
 public class OfflineScanner extends ScannerOptions implements Scanner {
 
   private int batchSize;
-  private int timeOut;
   private Range range;
 
   private ClientContext context;
@@ -48,19 +47,6 @@ public class OfflineScanner extends ScannerOptions implements Scanner {
     this.range = new Range((Key) null, (Key) null);
     this.authorizations = authorizations;
     this.batchSize = Constants.SCAN_BATCH_SIZE;
-    this.timeOut = Integer.MAX_VALUE;
-  }
-
-  @Deprecated
-  @Override
-  public void setTimeOut(int timeOut) {
-    this.timeOut = timeOut;
-  }
-
-  @Deprecated
-  @Override
-  public int getTimeOut() {
-    return timeOut;
   }
 
   @Override
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/ScannerImpl.java b/core/src/main/java/org/apache/accumulo/core/client/impl/ScannerImpl.java
index 1809260..dd9b19d 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/impl/ScannerImpl.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/ScannerImpl.java
@@ -93,8 +93,8 @@ public class ScannerImpl extends ScannerOptions implements Scanner {
 
   @Override
   public synchronized Iterator<Entry<Key,Value>> iterator() {
-    return new ScannerIterator(context, tableId, authorizations, range, size, getTimeOut(), this,
-        isolated, readaheadThreshold);
+    return new ScannerIterator(context, tableId, authorizations, range, size,
+        getTimeout(TimeUnit.SECONDS), this, isolated, readaheadThreshold);
   }
 
   @Override
@@ -112,24 +112,6 @@ public class ScannerImpl extends ScannerOptions implements Scanner {
     this.isolated = false;
   }
 
-  @Deprecated
-  @Override
-  public void setTimeOut(int timeOut) {
-    if (timeOut == Integer.MAX_VALUE)
-      setTimeout(Long.MAX_VALUE, TimeUnit.MILLISECONDS);
-    else
-      setTimeout(timeOut, TimeUnit.SECONDS);
-  }
-
-  @Deprecated
-  @Override
-  public int getTimeOut() {
-    long timeout = getTimeout(TimeUnit.SECONDS);
-    if (timeout >= Integer.MAX_VALUE)
-      return Integer.MAX_VALUE;
-    return (int) timeout;
-  }
-
   @Override
   public synchronized void setReadaheadThreshold(long batches) {
     if (0 > batches) {
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/ScannerIterator.java b/core/src/main/java/org/apache/accumulo/core/client/impl/ScannerIterator.java
index bba3c12..2b9bf4d 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/impl/ScannerIterator.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/ScannerIterator.java
@@ -48,7 +48,7 @@ public class ScannerIterator implements Iterator<Entry<Key,Value>> {
   private static final Logger log = LoggerFactory.getLogger(ScannerIterator.class);
 
   // scanner options
-  private int timeOut;
+  private long timeOut;
 
   // scanner state
   private Iterator<KeyValue> iter;
@@ -107,7 +107,7 @@ public class ScannerIterator implements Iterator<Entry<Key,Value>> {
   }
 
   ScannerIterator(ClientContext context, Table.ID tableId, Authorizations authorizations,
-      Range range, int size, int timeOut, ScannerOptions options, boolean isolated,
+      Range range, int size, long timeOut, ScannerOptions options, boolean isolated,
       long readaheadThreshold) {
     this.timeOut = timeOut;
     this.readaheadThreshold = readaheadThreshold;
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 b2c1ed7..d808506 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
@@ -95,14 +95,6 @@ public class SecurityOperationsImpl implements SecurityOperations {
     this.context = context;
   }
 
-  @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 {
@@ -121,12 +113,6 @@ public class SecurityOperationsImpl implements SecurityOperations {
     });
   }
 
-  @Deprecated
-  @Override
-  public void dropUser(final String user) throws AccumuloException, AccumuloSecurityException {
-    dropLocalUser(user);
-  }
-
   @Override
   public void dropLocalUser(final String principal)
       throws AccumuloException, AccumuloSecurityException {
@@ -134,13 +120,6 @@ public class SecurityOperationsImpl implements SecurityOperations {
     executeVoid(client -> client.dropLocalUser(Tracer.traceInfo(), context.rpcCreds(), 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 {
@@ -152,13 +131,6 @@ public class SecurityOperationsImpl implements SecurityOperations {
   }
 
   @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");
@@ -296,12 +268,6 @@ public class SecurityOperationsImpl implements SecurityOperations {
         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(client -> client.listLocalUsers(Tracer.traceInfo(), context.rpcCreds()));
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 26eb070..f0b1506 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
@@ -79,7 +79,6 @@ import org.apache.accumulo.core.client.admin.Locations;
 import org.apache.accumulo.core.client.admin.NewTableConfiguration;
 import org.apache.accumulo.core.client.admin.SummaryRetriever;
 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.Client;
 import org.apache.accumulo.core.client.impl.thrift.TDiskUsage;
@@ -208,28 +207,6 @@ public class TableOperationsImpl extends TableOperationsHelper {
   }
 
   @Override
-  @Deprecated
-  public void create(String tableName, boolean limitVersion)
-      throws AccumuloException, AccumuloSecurityException, TableExistsException {
-    create(tableName, limitVersion, TimeType.MILLIS);
-  }
-
-  @Override
-  @Deprecated
-  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");
-
-    NewTableConfiguration ntc = new NewTableConfiguration().setTimeType(timeType);
-
-    if (limitVersion)
-      create(tableName, ntc);
-    else
-      create(tableName, ntc.withoutDefaultIterators());
-  }
-
-  @Override
   public void create(String tableName, NewTableConfiguration ntc)
       throws AccumuloException, AccumuloSecurityException, TableExistsException {
     checkArgument(tableName != null, "tableName is null");
@@ -688,16 +665,6 @@ public class TableOperationsImpl extends TableOperationsHelper {
     return endRows;
   }
 
-  @Deprecated
-  @Override
-  public Collection<Text> getSplits(String tableName) throws TableNotFoundException {
-    try {
-      return listSplits(tableName);
-    } catch (AccumuloSecurityException e) {
-      throw new RuntimeException(e);
-    }
-  }
-
   @Override
   public Collection<Text> listSplits(String tableName, int maxSplits)
       throws TableNotFoundException, AccumuloSecurityException {
@@ -724,16 +691,6 @@ public class TableOperationsImpl extends TableOperationsHelper {
     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);
-    }
-  }
-
   @Override
   public void delete(String tableName)
       throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
@@ -801,7 +758,6 @@ public class TableOperationsImpl extends TableOperationsHelper {
   }
 
   @Override
-  @Deprecated
   public void flush(String tableName) throws AccumuloException, AccumuloSecurityException {
     try {
       flush(tableName, null, null, false);
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/ThriftScanner.java b/core/src/main/java/org/apache/accumulo/core/client/impl/ThriftScanner.java
index 0a57a34..6a911c7 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/impl/ThriftScanner.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/ThriftScanner.java
@@ -231,7 +231,7 @@ public class ThriftScanner {
     return (long) (Math.min(millis * 2, maxSleep) * (.9 + secureRandom.nextDouble() / 5));
   }
 
-  public static List<KeyValue> scan(ClientContext context, ScanState scanState, int timeOut)
+  public static List<KeyValue> scan(ClientContext context, ScanState scanState, long timeOut)
       throws ScanTimedOutException, AccumuloException, AccumuloSecurityException,
       TableNotFoundException {
     TabletLocation loc = null;
diff --git a/core/src/main/java/org/apache/accumulo/core/client/rfile/RFileScanner.java b/core/src/main/java/org/apache/accumulo/core/client/rfile/RFileScanner.java
index e9c06cc..7045441 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/rfile/RFileScanner.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/rfile/RFileScanner.java
@@ -27,7 +27,6 @@ import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Set;
 import java.util.SortedSet;
-import java.util.concurrent.TimeUnit;
 import java.util.function.Supplier;
 
 import org.apache.accumulo.core.client.IteratorSetting;
@@ -247,24 +246,6 @@ class RFileScanner extends ScannerOptions implements Scanner {
     throw new UnsupportedOperationException();
   }
 
-  @Deprecated
-  @Override
-  public void setTimeOut(int timeOut) {
-    if (timeOut == Integer.MAX_VALUE)
-      setTimeout(Long.MAX_VALUE, TimeUnit.MILLISECONDS);
-    else
-      setTimeout(timeOut, TimeUnit.SECONDS);
-  }
-
-  @Deprecated
-  @Override
-  public int getTimeOut() {
-    long timeout = getTimeout(TimeUnit.SECONDS);
-    if (timeout >= Integer.MAX_VALUE)
-      return Integer.MAX_VALUE;
-    return (int) timeout;
-  }
-
   @Override
   public void setRange(Range range) {
     this.range = range;
diff --git a/core/src/main/java/org/apache/accumulo/core/client/security/tokens/KerberosToken.java b/core/src/main/java/org/apache/accumulo/core/client/security/tokens/KerberosToken.java
index 8a8bae8..94a178a 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/security/tokens/KerberosToken.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/security/tokens/KerberosToken.java
@@ -86,30 +86,9 @@ public class KerberosToken implements AuthenticationToken {
    *          A keytab file containing the principal's credentials.
    */
   public KerberosToken(String principal, File keytab) throws IOException {
-    this(principal, keytab, false);
-  }
-
-  /**
-   * Creates a token and logs in via {@link UserGroupInformation} using the provided principal and
-   * keytab. A key for the principal must exist in the keytab, otherwise login will fail.
-   *
-   * @param principal
-   *          The Kerberos principal
-   * @param keytab
-   *          A keytab file
-   * @param replaceCurrentUser
-   *          Should the current Hadoop user be replaced with this user
-   * @deprecated since 1.8.0, @see #KerberosToken(String, File)
-   */
-  @Deprecated
-  public KerberosToken(String principal, File keytab, boolean replaceCurrentUser)
-      throws IOException {
     this.principal = requireNonNull(principal, "Principal was null");
     this.keytab = requireNonNull(keytab, "Keytab was null");
     checkArgument(keytab.exists() && keytab.isFile(), "Keytab was not a normal file");
-    if (replaceCurrentUser) {
-      UserGroupInformation.loginUserFromKeytab(principal, keytab.getAbsolutePath());
-    }
   }
 
   /**
diff --git a/core/src/main/java/org/apache/accumulo/core/constraints/Constraint.java b/core/src/main/java/org/apache/accumulo/core/constraints/Constraint.java
index 8109e76..dc38234 100644
--- a/core/src/main/java/org/apache/accumulo/core/constraints/Constraint.java
+++ b/core/src/main/java/org/apache/accumulo/core/constraints/Constraint.java
@@ -21,7 +21,6 @@ import java.util.List;
 import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.data.impl.KeyExtent;
 import org.apache.accumulo.core.security.AuthorizationContainer;
-import org.apache.accumulo.core.security.Authorizations;
 
 /**
  * Constraint objects are used to determine if mutations will be applied to a table.
@@ -65,15 +64,6 @@ public interface Constraint {
      * Gets the authorizations in the environment.
      *
      * @return authorizations
-     * @deprecated Use {@link #getAuthorizationsContainer()} instead.
-     */
-    @Deprecated
-    Authorizations getAuthorizations();
-
-    /**
-     * Gets the authorizations in the environment.
-     *
-     * @return authorizations
      */
     AuthorizationContainer getAuthorizationsContainer();
   }
diff --git a/core/src/main/java/org/apache/accumulo/core/data/PartialKey.java b/core/src/main/java/org/apache/accumulo/core/data/PartialKey.java
index ed0e91e..402ab06 100644
--- a/core/src/main/java/org/apache/accumulo/core/data/PartialKey.java
+++ b/core/src/main/java/org/apache/accumulo/core/data/PartialKey.java
@@ -35,24 +35,6 @@ public enum PartialKey {
   }
 
   /**
-   * Get a partial key specification by depth of the specification.
-   *
-   * @param depth
-   *          depth of scope (i.e., number of fields included)
-   * @return partial key
-   * @throws IllegalArgumentException
-   *           if no partial key has the given depth
-   * @deprecated since 1.7.0
-   */
-  @Deprecated
-  public static PartialKey getByDepth(int depth) {
-    for (PartialKey d : PartialKey.values())
-      if (depth == d.depth)
-        return d;
-    throw new IllegalArgumentException("Invalid legacy depth " + depth);
-  }
-
-  /**
    * Gets the depth of this partial key.
    *
    * @return depth
diff --git a/core/src/main/java/org/apache/accumulo/core/data/Value.java b/core/src/main/java/org/apache/accumulo/core/data/Value.java
index 51767ea..1e57987 100644
--- a/core/src/main/java/org/apache/accumulo/core/data/Value.java
+++ b/core/src/main/java/org/apache/accumulo/core/data/Value.java
@@ -25,7 +25,6 @@ import java.io.DataInput;
 import java.io.DataOutput;
 import java.io.IOException;
 import java.nio.ByteBuffer;
-import java.util.List;
 
 import org.apache.hadoop.io.BytesWritable;
 import org.apache.hadoop.io.Text;
@@ -97,22 +96,6 @@ public class Value implements WritableComparable<Object> {
   }
 
   /**
-   * @deprecated A copy of the bytes in the buffer is always made. Use {@link #Value(ByteBuffer)}
-   *             instead.
-   *
-   * @param bytes
-   *          bytes of value (may not be null)
-   * @param copy
-   *          false to use the backing array of the buffer directly as the backing array, true to
-   *          force a copy
-   */
-  @Deprecated
-  public Value(ByteBuffer bytes, boolean copy) {
-    /* TODO ACCUMULO-2509 right now this uses the entire backing array, which must be accessible. */
-    this(toBytes(bytes), false);
-  }
-
-  /**
    * Creates a Value using a byte array as the initial value.
    *
    * @param bytes
@@ -291,22 +274,4 @@ public class Value implements WritableComparable<Object> {
     WritableComparator.define(Value.class, new Comparator());
   }
 
-  /**
-   * Converts a list of byte arrays to a two-dimensional array.
-   *
-   * @param array
-   *          list of byte arrays
-   * @return two-dimensional byte array containing one given byte array per row
-   * @deprecated since 1.7.0; this utility method is not appropriate for the {@link Value} object
-   */
-  @Deprecated
-  public static byte[][] toArray(final List<byte[]> array) {
-    // List#toArray doesn't work on lists of byte [].
-    byte[][] results = new byte[array.size()][];
-    for (int i = 0; i < array.size(); i++) {
-      results[i] = array.get(i);
-    }
-    return results;
-  }
-
 }
diff --git a/core/src/main/java/org/apache/accumulo/core/data/impl/KeyExtent.java b/core/src/main/java/org/apache/accumulo/core/data/impl/KeyExtent.java
index f42d265..5a7edd1 100644
--- a/core/src/main/java/org/apache/accumulo/core/data/impl/KeyExtent.java
+++ b/core/src/main/java/org/apache/accumulo/core/data/impl/KeyExtent.java
@@ -24,9 +24,7 @@ import java.io.DataOutput;
 import java.io.DataOutputStream;
 import java.io.IOException;
 import java.nio.ByteBuffer;
-import java.util.ArrayList;
 import java.util.Arrays;
-import java.util.Collection;
 import java.util.Collections;
 import java.util.Comparator;
 import java.util.Map.Entry;
@@ -282,82 +280,6 @@ public class KeyExtent implements WritableComparable<KeyExtent> {
     return getPrevRowUpdateMutation(this);
   }
 
-  /**
-   * Empty start or end rows tell the method there are no start or end rows, and to use all the
-   * keyextents that are before the end row if no start row etc.
-   *
-   * @deprecated this method not intended for public use and is likely to be removed in a future
-   *             version.
-   * @return all the key extents that the rows cover
-   */
-  @Deprecated
-  public static Collection<KeyExtent> getKeyExtentsForRange(Text startRow, Text endRow,
-      Set<KeyExtent> kes) {
-    if (kes == null)
-      return Collections.emptyList();
-    if (startRow == null)
-      startRow = new Text();
-    if (endRow == null)
-      endRow = new Text();
-    Collection<KeyExtent> keys = new ArrayList<>();
-    for (KeyExtent ckes : kes) {
-      if (ckes.getPrevEndRow() == null) {
-        if (ckes.getEndRow() == null) {
-          // only tablet
-          keys.add(ckes);
-        } else {
-          // first tablet
-          // if start row = '' then we want everything up to the endRow which will always include
-          // the first tablet
-          if (startRow.getLength() == 0) {
-            keys.add(ckes);
-          } else if (ckes.getEndRow().compareTo(startRow) >= 0) {
-            keys.add(ckes);
-          }
-        }
-      } else {
-        if (ckes.getEndRow() == null) {
-          // last tablet
-          // if endRow = '' and we're at the last tablet, add it
-          if (endRow.getLength() == 0) {
-            keys.add(ckes);
-          }
-          if (ckes.getPrevEndRow().compareTo(endRow) < 0) {
-            keys.add(ckes);
-          }
-        } else {
-          // tablet in the middle
-          if (startRow.getLength() == 0) {
-            // no start row
-
-            if (endRow.getLength() == 0) {
-              // no start & end row
-              keys.add(ckes);
-            } else {
-              // just no start row
-              if (ckes.getPrevEndRow().compareTo(endRow) < 0) {
-                keys.add(ckes);
-              }
-            }
-          } else if (endRow.getLength() == 0) {
-            // no end row
-            if (ckes.getEndRow().compareTo(startRow) >= 0) {
-              keys.add(ckes);
-            }
-          } else {
-            // no null prevEnd or endRows and no empty string start or end rows
-            if (ckes.getPrevEndRow().compareTo(endRow) < 0
-                && ckes.getEndRow().compareTo(startRow) >= 0) {
-              keys.add(ckes);
-            }
-          }
-
-        }
-      }
-    }
-    return keys;
-  }
-
   public static Text decodePrevEndRow(Value ibw) {
     Text per = null;
 
diff --git a/core/src/main/java/org/apache/accumulo/core/iterators/AggregatingIterator.java b/core/src/main/java/org/apache/accumulo/core/iterators/AggregatingIterator.java
deleted file mode 100644
index 9316825..0000000
--- a/core/src/main/java/org/apache/accumulo/core/iterators/AggregatingIterator.java
+++ /dev/null
@@ -1,221 +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.iterators;
-
-import java.io.IOException;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.Map;
-import java.util.Map.Entry;
-
-import org.apache.accumulo.core.conf.Property;
-import org.apache.accumulo.core.data.ByteSequence;
-import org.apache.accumulo.core.data.Key;
-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.conf.ColumnToClassMapping;
-import org.apache.accumulo.start.classloader.vfs.AccumuloVFSClassLoader;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * This iterator wraps another iterator. It automatically aggregates.
- *
- * @deprecated since 1.4, replaced by {@link org.apache.accumulo.core.iterators.Combiner}
- */
-@Deprecated
-public class AggregatingIterator implements SortedKeyValueIterator<Key,Value>, OptionDescriber {
-
-  private SortedKeyValueIterator<Key,Value> iterator;
-  // @formatter:off
-  private ColumnToClassMapping<org.apache.accumulo.core.iterators.aggregation.Aggregator>
-    aggregators;
-  // @formatter:on
-
-  private Key workKey = new Key();
-
-  private Key aggrKey;
-  private Value aggrValue;
-  // private boolean propogateDeletes;
-  private static final Logger log = LoggerFactory.getLogger(AggregatingIterator.class);
-
-  @Override
-  public AggregatingIterator deepCopy(IteratorEnvironment env) {
-    return new AggregatingIterator(this, env);
-  }
-
-  private AggregatingIterator(AggregatingIterator other, IteratorEnvironment env) {
-    iterator = other.iterator.deepCopy(env);
-    aggregators = other.aggregators;
-  }
-
-  public AggregatingIterator() {}
-
-  private void aggregateRowColumn(org.apache.accumulo.core.iterators.aggregation.Aggregator aggr)
-      throws IOException {
-    // this function assumes that first value is not delete
-
-    if (iterator.getTopKey().isDeleted())
-      return;
-
-    workKey.set(iterator.getTopKey());
-
-    Key keyToAggregate = workKey;
-
-    aggr.reset();
-
-    aggr.collect(iterator.getTopValue());
-    iterator.next();
-
-    while (iterator.hasTop() && !iterator.getTopKey().isDeleted()
-        && iterator.getTopKey().equals(keyToAggregate, PartialKey.ROW_COLFAM_COLQUAL_COLVIS)) {
-      aggr.collect(iterator.getTopValue());
-      iterator.next();
-    }
-
-    aggrKey = workKey;
-    aggrValue = aggr.aggregate();
-
-  }
-
-  private void findTop() throws IOException {
-    // check if aggregation is needed
-    if (iterator.hasTop()) {
-      org.apache.accumulo.core.iterators.aggregation.Aggregator aggr = aggregators
-          .getObject(iterator.getTopKey());
-      if (aggr != null) {
-        aggregateRowColumn(aggr);
-      }
-    }
-  }
-
-  public AggregatingIterator(SortedKeyValueIterator<Key,Value> iterator,
-      ColumnToClassMapping<org.apache.accumulo.core.iterators.aggregation.Aggregator> aggregators)
-      throws IOException {
-    this.iterator = iterator;
-    this.aggregators = aggregators;
-  }
-
-  @Override
-  public Key getTopKey() {
-    if (aggrKey != null) {
-      return aggrKey;
-    }
-    return iterator.getTopKey();
-  }
-
-  @Override
-  public Value getTopValue() {
-    if (aggrKey != null) {
-      return aggrValue;
-    }
-    return iterator.getTopValue();
-  }
-
-  @Override
-  public boolean hasTop() {
-    return aggrKey != null || iterator.hasTop();
-  }
-
-  @Override
-  public void next() throws IOException {
-    if (aggrKey != null) {
-      aggrKey = null;
-      aggrValue = null;
-    } else {
-      iterator.next();
-    }
-
-    findTop();
-  }
-
-  @Override
-  public void seek(Range range, Collection<ByteSequence> columnFamilies, boolean inclusive)
-      throws IOException {
-    // do not want to seek to the middle of a value that should be
-    // aggregated...
-
-    Range seekRange = IteratorUtil.maximizeStartKeyTimeStamp(range);
-
-    iterator.seek(seekRange, columnFamilies, inclusive);
-    findTop();
-
-    if (range.getStartKey() != null) {
-      while (hasTop()
-          && getTopKey().equals(range.getStartKey(), PartialKey.ROW_COLFAM_COLQUAL_COLVIS)
-          && getTopKey().getTimestamp() > range.getStartKey().getTimestamp()) {
-        // the value has a more recent time stamp, so
-        // pass it up
-        // log.debug("skipping "+getTopKey());
-        next();
-      }
-
-      while (hasTop() && range.beforeStartKey(getTopKey())) {
-        next();
-      }
-    }
-
-  }
-
-  @Override
-  public void init(SortedKeyValueIterator<Key,Value> source, Map<String,String> options,
-      IteratorEnvironment env) throws IOException {
-
-    this.iterator = source;
-
-    try {
-      String context = null;
-      if (null != env)
-        context = env.getConfig().get(Property.TABLE_CLASSPATH);
-      this.aggregators = new ColumnToClassMapping<>(options,
-          org.apache.accumulo.core.iterators.aggregation.Aggregator.class, context);
-    } catch (ClassNotFoundException | IllegalAccessException | InstantiationException e) {
-      log.error(e.toString());
-      throw new IllegalArgumentException(e);
-    }
-  }
-
-  @Override
-  public IteratorOptions describeOptions() {
-    return new IteratorOptions("agg",
-        "Aggregators apply aggregating functions to values with identical keys", null,
-        Collections.singletonList("<columnName> <aggregatorClass>"));
-  }
-
-  @Override
-  public boolean validateOptions(Map<String,String> options) {
-    for (Entry<String,String> entry : options.entrySet()) {
-      String classname = entry.getValue();
-      if (classname == null)
-        throw new IllegalArgumentException("classname null");
-      Class<? extends org.apache.accumulo.core.iterators.aggregation.Aggregator> clazz;
-      try {
-        clazz = AccumuloVFSClassLoader.loadClass(classname,
-            org.apache.accumulo.core.iterators.aggregation.Aggregator.class);
-        clazz.newInstance();
-      } catch (ClassNotFoundException e) {
-        throw new IllegalArgumentException("class not found: " + classname);
-      } catch (InstantiationException e) {
-        throw new IllegalArgumentException("instantiation exception: " + classname);
-      } catch (IllegalAccessException e) {
-        throw new IllegalArgumentException("illegal access exception: " + classname);
-      }
-    }
-    return true;
-  }
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/iterators/FamilyIntersectingIterator.java b/core/src/main/java/org/apache/accumulo/core/iterators/FamilyIntersectingIterator.java
deleted file mode 100644
index ba9df4a..0000000
--- a/core/src/main/java/org/apache/accumulo/core/iterators/FamilyIntersectingIterator.java
+++ /dev/null
@@ -1,30 +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.iterators;
-
-import org.apache.accumulo.core.iterators.user.IndexedDocIterator;
-
-/**
- * This class remains here for backwards compatibility.
- *
- * @deprecated since 1.4, replaced by
- *             {@link org.apache.accumulo.core.iterators.user.IndexedDocIterator}
- */
-@Deprecated
-public class FamilyIntersectingIterator extends IndexedDocIterator {
-
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/iterators/aggregation/Aggregator.java b/core/src/main/java/org/apache/accumulo/core/iterators/aggregation/Aggregator.java
deleted file mode 100644
index f9183dc..0000000
--- a/core/src/main/java/org/apache/accumulo/core/iterators/aggregation/Aggregator.java
+++ /dev/null
@@ -1,31 +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.iterators.aggregation;
-
-import org.apache.accumulo.core.data.Value;
-
-/**
- * @deprecated since 1.4, replaced by {@link org.apache.accumulo.core.iterators.Combiner}
- */
-@Deprecated
-public interface Aggregator {
-  void reset();
-
-  void collect(Value value);
-
-  Value aggregate();
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/iterators/aggregation/LongSummation.java b/core/src/main/java/org/apache/accumulo/core/iterators/aggregation/LongSummation.java
deleted file mode 100644
index 3d8d187..0000000
--- a/core/src/main/java/org/apache/accumulo/core/iterators/aggregation/LongSummation.java
+++ /dev/null
@@ -1,81 +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.iterators.aggregation;
-
-import java.io.IOException;
-
-import org.apache.accumulo.core.data.Value;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * @deprecated since 1.4, replaced by
- *             {@link org.apache.accumulo.core.iterators.user.SummingCombiner} with
- *             {@link org.apache.accumulo.core.iterators.LongCombiner.Type#FIXEDLEN}
- */
-@Deprecated
-public class LongSummation implements Aggregator {
-  private static final Logger log = LoggerFactory.getLogger(LongSummation.class);
-  long sum = 0;
-
-  @Override
-  public Value aggregate() {
-    return new Value(longToBytes(sum));
-  }
-
-  @Override
-  public void collect(Value value) {
-    try {
-      sum += bytesToLong(value.get());
-    } catch (IOException e) {
-      log.error("{} trying to convert bytes to long, but byte array isn't length 8",
-          LongSummation.class.getSimpleName());
-    }
-  }
-
-  @Override
-  public void reset() {
-    sum = 0;
-  }
-
-  public static long bytesToLong(byte[] b) throws IOException {
-    return bytesToLong(b, 0);
-  }
-
-  public static long bytesToLong(byte[] b, int offset) throws IOException {
-    if (b.length < offset + 8)
-      throw new IOException("trying to convert to long, but byte array isn't long enough, wanted "
-          + (offset + 8) + " found " + b.length);
-    return (((long) b[offset + 0] << 56) + ((long) (b[offset + 1] & 255) << 48)
-        + ((long) (b[offset + 2] & 255) << 40) + ((long) (b[offset + 3] & 255) << 32)
-        + ((long) (b[offset + 4] & 255) << 24) + ((b[offset + 5] & 255) << 16)
-        + ((b[offset + 6] & 255) << 8) + ((b[offset + 7] & 255) << 0));
-  }
-
-  public static byte[] longToBytes(long l) {
-    byte[] b = new byte[8];
-    b[0] = (byte) (l >>> 56);
-    b[1] = (byte) (l >>> 48);
-    b[2] = (byte) (l >>> 40);
-    b[3] = (byte) (l >>> 32);
-    b[4] = (byte) (l >>> 24);
-    b[5] = (byte) (l >>> 16);
-    b[6] = (byte) (l >>> 8);
-    b[7] = (byte) (l >>> 0);
-    return b;
-  }
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/iterators/aggregation/NumArraySummation.java b/core/src/main/java/org/apache/accumulo/core/iterators/aggregation/NumArraySummation.java
deleted file mode 100644
index a1dd031..0000000
--- a/core/src/main/java/org/apache/accumulo/core/iterators/aggregation/NumArraySummation.java
+++ /dev/null
@@ -1,97 +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.iterators.aggregation;
-
-import java.io.ByteArrayInputStream;
-import java.io.ByteArrayOutputStream;
-import java.io.DataInputStream;
-import java.io.DataOutputStream;
-import java.io.IOException;
-
-import org.apache.accumulo.core.data.Value;
-import org.apache.hadoop.io.WritableUtils;
-
-/**
- * @deprecated since 1.4, replaced by
- *             {@link org.apache.accumulo.core.iterators.user.SummingArrayCombiner} with
- *             {@link org.apache.accumulo.core.iterators.user.SummingArrayCombiner.Type#VARLEN}
- */
-@Deprecated
-public class NumArraySummation implements Aggregator {
-  long[] sum = new long[0];
-
-  @Override
-  public Value aggregate() {
-    try {
-      return new Value(NumArraySummation.longArrayToBytes(sum));
-    } catch (IOException e) {
-      throw new RuntimeException(e);
-    }
-  }
-
-  @Override
-  public void collect(Value value) {
-    long[] la;
-    try {
-      la = NumArraySummation.bytesToLongArray(value.get());
-    } catch (IOException e) {
-      throw new RuntimeException(e);
-    }
-
-    if (la.length > sum.length) {
-      for (int i = 0; i < sum.length; i++) {
-        la[i] = NumSummation.safeAdd(la[i], sum[i]);
-      }
-      sum = la;
-    } else {
-      for (int i = 0; i < la.length; i++) {
-        sum[i] = NumSummation.safeAdd(sum[i], la[i]);
-      }
-    }
-  }
-
-  public static byte[] longArrayToBytes(long[] la) throws IOException {
-    ByteArrayOutputStream baos = new ByteArrayOutputStream();
-    DataOutputStream dos = new DataOutputStream(baos);
-
-    WritableUtils.writeVInt(dos, la.length);
-    for (int i = 0; i < la.length; i++) {
-      WritableUtils.writeVLong(dos, la[i]);
-    }
-
-    return baos.toByteArray();
-  }
-
-  public static long[] bytesToLongArray(byte[] b) throws IOException {
-    DataInputStream dis = new DataInputStream(new ByteArrayInputStream(b));
-    int len = WritableUtils.readVInt(dis);
-
-    long[] la = new long[len];
-
-    for (int i = 0; i < len; i++) {
-      la[i] = WritableUtils.readVLong(dis);
-    }
-
-    return la;
-  }
-
-  @Override
-  public void reset() {
-    sum = new long[0];
-  }
-
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/iterators/aggregation/NumSummation.java b/core/src/main/java/org/apache/accumulo/core/iterators/aggregation/NumSummation.java
deleted file mode 100644
index 2501a26..0000000
--- a/core/src/main/java/org/apache/accumulo/core/iterators/aggregation/NumSummation.java
+++ /dev/null
@@ -1,92 +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.iterators.aggregation;
-
-import java.io.ByteArrayInputStream;
-import java.io.ByteArrayOutputStream;
-import java.io.DataInputStream;
-import java.io.DataOutputStream;
-import java.io.IOException;
-
-import org.apache.accumulo.core.data.Value;
-import org.apache.hadoop.io.WritableUtils;
-
-/**
- * @deprecated since 1.4, replaced by
- *             {@link org.apache.accumulo.core.iterators.user.SummingCombiner} with
- *             {@link org.apache.accumulo.core.iterators.LongCombiner.Type#VARLEN}
- */
-@Deprecated
-public class NumSummation implements Aggregator {
-  long sum = 0L;
-
-  @Override
-  public Value aggregate() {
-    try {
-      return new Value(NumSummation.longToBytes(sum));
-    } catch (IOException e) {
-      throw new RuntimeException(e);
-    }
-  }
-
-  @Override
-  public void collect(Value value) {
-    long l;
-    try {
-      l = NumSummation.bytesToLong(value.get());
-    } catch (IOException e) {
-      throw new RuntimeException(e);
-    }
-
-    sum = NumSummation.safeAdd(sum, l);
-  }
-
-  public static byte[] longToBytes(long l) throws IOException {
-    ByteArrayOutputStream baos = new ByteArrayOutputStream();
-    DataOutputStream dos = new DataOutputStream(baos);
-
-    WritableUtils.writeVLong(dos, l);
-
-    return baos.toByteArray();
-  }
-
-  public static long bytesToLong(byte[] b) throws IOException {
-    DataInputStream dis = new DataInputStream(new ByteArrayInputStream(b));
-    return WritableUtils.readVLong(dis);
-  }
-
-  public static long safeAdd(long a, long b) {
-    long aSign = Long.signum(a);
-    long bSign = Long.signum(b);
-    if ((aSign != 0) && (bSign != 0) && (aSign == bSign)) {
-      if (aSign > 0) {
-        if (Long.MAX_VALUE - a < b)
-          return Long.MAX_VALUE;
-      } else {
-        if (Long.MIN_VALUE - a > b)
-          return Long.MIN_VALUE;
-      }
-    }
-    return a + b;
-  }
-
-  @Override
-  public void reset() {
-    sum = 0L;
-  }
-
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/iterators/aggregation/StringMax.java b/core/src/main/java/org/apache/accumulo/core/iterators/aggregation/StringMax.java
deleted file mode 100644
index b389f12..0000000
--- a/core/src/main/java/org/apache/accumulo/core/iterators/aggregation/StringMax.java
+++ /dev/null
@@ -1,48 +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.iterators.aggregation;
-
-import org.apache.accumulo.core.data.Value;
-
-/**
- * @deprecated since 1.4, replaced by {@link org.apache.accumulo.core.iterators.user.MaxCombiner}
- *             with {@link org.apache.accumulo.core.iterators.LongCombiner.Type#STRING}
- */
-@Deprecated
-public class StringMax implements Aggregator {
-
-  long max = Long.MIN_VALUE;
-
-  @Override
-  public Value aggregate() {
-    return new Value(Long.toString(max).getBytes());
-  }
-
-  @Override
-  public void collect(Value value) {
-    long l = Long.parseLong(new String(value.get()));
-    if (l > max) {
-      max = l;
-    }
-  }
-
-  @Override
-  public void reset() {
-    max = Long.MIN_VALUE;
-  }
-
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/iterators/aggregation/StringMin.java b/core/src/main/java/org/apache/accumulo/core/iterators/aggregation/StringMin.java
deleted file mode 100644
index 5b49094..0000000
--- a/core/src/main/java/org/apache/accumulo/core/iterators/aggregation/StringMin.java
+++ /dev/null
@@ -1,48 +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.iterators.aggregation;
-
-import org.apache.accumulo.core.data.Value;
-
-/**
- * @deprecated since 1.4, replaced by {@link org.apache.accumulo.core.iterators.user.MinCombiner}
- *             with {@link org.apache.accumulo.core.iterators.LongCombiner.Type#STRING}
- */
-@Deprecated
-public class StringMin implements Aggregator {
-
-  long min = Long.MAX_VALUE;
-
-  @Override
-  public Value aggregate() {
-    return new Value(Long.toString(min).getBytes());
-  }
-
-  @Override
-  public void collect(Value value) {
-    long l = Long.parseLong(new String(value.get()));
-    if (l < min) {
-      min = l;
-    }
-  }
-
-  @Override
-  public void reset() {
-    min = Long.MAX_VALUE;
-  }
-
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/iterators/aggregation/StringSummation.java b/core/src/main/java/org/apache/accumulo/core/iterators/aggregation/StringSummation.java
deleted file mode 100644
index cf857de..0000000
--- a/core/src/main/java/org/apache/accumulo/core/iterators/aggregation/StringSummation.java
+++ /dev/null
@@ -1,46 +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.iterators.aggregation;
-
-import org.apache.accumulo.core.data.Value;
-
-/**
- * @deprecated since 1.4, replaced by
- *             {@link org.apache.accumulo.core.iterators.user.SummingCombiner} with
- *             {@link org.apache.accumulo.core.iterators.LongCombiner.Type#STRING}
- */
-@Deprecated
-public class StringSummation implements Aggregator {
-
-  long sum = 0;
-
-  @Override
-  public Value aggregate() {
-    return new Value(Long.toString(sum).getBytes());
-  }
-
-  @Override
-  public void collect(Value value) {
-    sum += Long.parseLong(new String(value.get()));
-  }
-
-  @Override
-  public void reset() {
-    sum = 0;
-
-  }
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/iterators/aggregation/conf/AggregatorConfiguration.java b/core/src/main/java/org/apache/accumulo/core/iterators/aggregation/conf/AggregatorConfiguration.java
deleted file mode 100644
index 69bb84f..0000000
--- a/core/src/main/java/org/apache/accumulo/core/iterators/aggregation/conf/AggregatorConfiguration.java
+++ /dev/null
@@ -1,35 +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.iterators.aggregation.conf;
-
-import org.apache.hadoop.io.Text;
-
-/**
- * @deprecated since 1.4
- */
-@Deprecated
-public class AggregatorConfiguration
-    extends org.apache.accumulo.core.iterators.conf.PerColumnIteratorConfig {
-
-  public AggregatorConfiguration(Text columnFamily, String aggClassName) {
-    super(columnFamily, aggClassName);
-  }
-
-  public AggregatorConfiguration(Text columnFamily, Text columnQualifier, String aggClassName) {
-    super(columnFamily, columnQualifier, aggClassName);
-  }
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/iterators/aggregation/conf/AggregatorSet.java b/core/src/main/java/org/apache/accumulo/core/iterators/aggregation/conf/AggregatorSet.java
deleted file mode 100644
index db288b6..0000000
--- a/core/src/main/java/org/apache/accumulo/core/iterators/aggregation/conf/AggregatorSet.java
+++ /dev/null
@@ -1,43 +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.iterators.aggregation.conf;
-
-import java.io.IOException;
-import java.util.Map;
-
-import org.apache.accumulo.core.data.Key;
-import org.apache.accumulo.core.iterators.conf.ColumnToClassMapping;
-
-/**
- * @deprecated since 1.4
- */
-@Deprecated
-public class AggregatorSet
-    extends ColumnToClassMapping<org.apache.accumulo.core.iterators.aggregation.Aggregator> {
-  public AggregatorSet(Map<String,String> opts)
-      throws InstantiationException, IllegalAccessException, ClassNotFoundException, IOException {
-    super(opts, org.apache.accumulo.core.iterators.aggregation.Aggregator.class);
-  }
-
-  public AggregatorSet() {
-    super();
-  }
-
-  public org.apache.accumulo.core.iterators.aggregation.Aggregator getAggregator(Key k) {
-    return getObject(k);
-  }
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/iterators/conf/PerColumnIteratorConfig.java b/core/src/main/java/org/apache/accumulo/core/iterators/conf/PerColumnIteratorConfig.java
deleted file mode 100644
index ae2b43b..0000000
--- a/core/src/main/java/org/apache/accumulo/core/iterators/conf/PerColumnIteratorConfig.java
+++ /dev/null
@@ -1,83 +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.iterators.conf;
-
-import org.apache.hadoop.io.Text;
-
-/**
- * @deprecated since 1.4
- *
- * @see org.apache.accumulo.core.client.IteratorSetting.Column
- * @see org.apache.accumulo.core.iterators.Combiner#setColumns(org.apache.accumulo.core.client.IteratorSetting,
- *      java.util.List)
- */
-@Deprecated
-public class PerColumnIteratorConfig {
-
-  private String parameter;
-  private Text colq;
-  private Text colf;
-
-  public PerColumnIteratorConfig(Text columnFamily, String parameter) {
-    this.colf = columnFamily;
-    this.colq = null;
-    this.parameter = parameter;
-  }
-
-  public PerColumnIteratorConfig(Text columnFamily, Text columnQualifier, String parameter) {
-    this.colf = columnFamily;
-    this.colq = columnQualifier;
-    this.parameter = parameter;
-  }
-
-  public Text getColumnFamily() {
-    return colf;
-  }
-
-  public Text getColumnQualifier() {
-    return colq;
-  }
-
-  public String encodeColumns() {
-    return encodeColumns(this);
-  }
-
-  public String getClassName() {
-    return parameter;
-  }
-
-  private static String encodeColumns(PerColumnIteratorConfig pcic) {
-    return ColumnSet.encodeColumns(pcic.colf, pcic.colq);
-  }
-
-  public static String encodeColumns(Text columnFamily, Text columnQualifier) {
-    return ColumnSet.encodeColumns(columnFamily, columnQualifier);
-  }
-
-  public static PerColumnIteratorConfig decodeColumns(String columns, String className) {
-    String[] cols = columns.split(":");
-
-    if (cols.length == 1) {
-      return new PerColumnIteratorConfig(ColumnSet.decode(cols[0]), className);
-    } else if (cols.length == 2) {
-      return new PerColumnIteratorConfig(ColumnSet.decode(cols[0]), ColumnSet.decode(cols[1]),
-          className);
-    } else {
-      throw new IllegalArgumentException(columns);
-    }
-  }
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/iterators/user/IntersectingIterator.java b/core/src/main/java/org/apache/accumulo/core/iterators/user/IntersectingIterator.java
index f5017a7..7300b46 100644
--- a/core/src/main/java/org/apache/accumulo/core/iterators/user/IntersectingIterator.java
+++ b/core/src/main/java/org/apache/accumulo/core/iterators/user/IntersectingIterator.java
@@ -528,31 +528,6 @@ public class IntersectingIterator implements SortedKeyValueIterator<Key,Value> {
   }
 
   /**
-   * @deprecated since 1.6.0
-   */
-  @Deprecated
-  public void addSource(SortedKeyValueIterator<Key,Value> source, IteratorEnvironment env,
-      Text term, boolean notFlag) {
-    // Check if we have space for the added Source
-    if (sources == null) {
-      sources = new TermSource[1];
-    } else {
-      // allocate space for node, and copy current tree.
-      // TODO: Should we change this to an ArrayList so that we can just add() ? - ACCUMULO-1309
-      TermSource[] localSources = new TermSource[sources.length + 1];
-      int currSource = 0;
-      for (TermSource myTerm : sources) {
-        // TODO: Do I need to call new here? or can I just re-use the term? - ACCUMULO-1309
-        localSources[currSource] = new TermSource(myTerm);
-        currSource++;
-      }
-      sources = localSources;
-    }
-    sources[sourcesCount] = new TermSource(source.deepCopy(env), term, notFlag);
-    sourcesCount++;
-  }
-
-  /**
    * Encode the columns to be used when iterating.
    */
   public static void setColumnFamilies(IteratorSetting cfg, Text[] columns) {
diff --git a/core/src/main/java/org/apache/accumulo/core/trace/Trace.java b/core/src/main/java/org/apache/accumulo/core/trace/Trace.java
index e6aded7..586a950 100644
--- a/core/src/main/java/org/apache/accumulo/core/trace/Trace.java
+++ b/core/src/main/java/org/apache/accumulo/core/trace/Trace.java
@@ -55,14 +55,6 @@ public class Trace {
   }
 
   /**
-   * @deprecated since 1.7, use {@link #off()} instead
-   */
-  @Deprecated
-  public static void offNoFlush() {
-    off();
-  }
-
-  /**
    * Returns whether tracing is currently on.
    */
   public static boolean isTracing() {
@@ -70,17 +62,6 @@ public class Trace {
   }
 
   /**
-   * Return the current span.
-   *
-   * @deprecated since 1.7 -- it is better to save the span you create in a local variable and call
-   *             its methods, rather than retrieving the current span
-   */
-  @Deprecated
-  public static Span currentTrace() {
-    return new Span(org.apache.htrace.Trace.currentSpan());
-  }
-
-  /**
    * Get the trace id of the current span.
    */
   public static long currentTraceId() {
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
index 619d2dd..461ec63 100644
--- 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
@@ -44,7 +44,6 @@ import org.apache.accumulo.core.client.admin.DiskUsage;
 import org.apache.accumulo.core.client.admin.Locations;
 import org.apache.accumulo.core.client.admin.NewTableConfiguration;
 import org.apache.accumulo.core.client.admin.SummaryRetriever;
-import org.apache.accumulo.core.client.admin.TimeType;
 import org.apache.accumulo.core.client.sample.SamplerConfiguration;
 import org.apache.accumulo.core.client.summary.SummarizerConfiguration;
 import org.apache.accumulo.core.data.Range;
@@ -73,18 +72,6 @@ public class TableOperationsHelperTest {
         throws AccumuloException, AccumuloSecurityException, TableExistsException {}
 
     @Override
-    @Deprecated
-    public void create(String tableName, boolean limitVersion)
-        throws AccumuloException, AccumuloSecurityException, TableExistsException {
-      create(tableName, limitVersion, TimeType.MILLIS);
-    }
-
-    @Override
-    @Deprecated
-    public void create(String tableName, boolean versioningIter, TimeType timeType)
-        throws AccumuloException, AccumuloSecurityException, TableExistsException {}
-
-    @Override
     public void create(String tableName, NewTableConfiguration ntc)
         throws AccumuloException, AccumuloSecurityException, TableExistsException {}
 
@@ -92,19 +79,6 @@ public class TableOperationsHelperTest {
     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;
@@ -160,7 +134,6 @@ public class TableOperationsHelperTest {
     public void rename(String oldTableName, String newTableName) throws AccumuloSecurityException,
         TableNotFoundException, AccumuloException, TableExistsException {}
 
-    @Deprecated
     @Override
     public void flush(String tableName) throws AccumuloException, AccumuloSecurityException {}
 
diff --git a/core/src/test/java/org/apache/accumulo/core/data/KeyExtentTest.java b/core/src/test/java/org/apache/accumulo/core/data/KeyExtentTest.java
index f8c6891..5ecc63d 100644
--- a/core/src/test/java/org/apache/accumulo/core/data/KeyExtentTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/data/KeyExtentTest.java
@@ -16,13 +16,9 @@
  */
 package org.apache.accumulo.core.data;
 
-import static org.hamcrest.CoreMatchers.hasItem;
-import static org.hamcrest.CoreMatchers.hasItems;
-import static org.hamcrest.CoreMatchers.is;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNull;
-import static org.junit.Assert.assertThat;
 import static org.junit.Assert.assertTrue;
 
 import java.io.ByteArrayInputStream;
@@ -30,9 +26,6 @@ import java.io.ByteArrayOutputStream;
 import java.io.DataInputStream;
 import java.io.DataOutputStream;
 import java.io.IOException;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Collections;
 import java.util.SortedMap;
 import java.util.TreeMap;
 import java.util.TreeSet;
@@ -294,57 +287,6 @@ public class KeyExtentTest {
     return out;
   }
 
-  @SuppressWarnings("deprecation")
-  @Test
-  public void testKeyExtentsForSimpleRange() {
-    Collection<KeyExtent> results;
-
-    results = KeyExtent.getKeyExtentsForRange(null, null, null);
-    assertTrue("Non-empty set returned from no extents", results.isEmpty());
-
-    results = KeyExtent.getKeyExtentsForRange(null, null, Collections.emptySet());
-    assertTrue("Non-empty set returned from no extents", results.isEmpty());
-
-    KeyExtent t = nke("t", null, null);
-    results = KeyExtent.getKeyExtentsForRange(null, null, Collections.singleton(t));
-    assertEquals("Single tablet should always be returned", 1, results.size());
-    assertEquals(t, results.iterator().next());
-  }
-
-  @SuppressWarnings("deprecation")
-  @Test
-  public void testKeyExtentsForRange() {
-    KeyExtent b = nke("t", "b", null);
-    KeyExtent e = nke("t", "e", "b");
-    KeyExtent h = nke("t", "h", "e");
-    KeyExtent m = nke("t", "m", "h");
-    KeyExtent z = nke("t", null, "m");
-
-    set0.addAll(Arrays.asList(b, e, h, m, z));
-
-    Collection<KeyExtent> results;
-
-    results = KeyExtent.getKeyExtentsForRange(null, null, set0);
-    assertThat("infinite range should return full set", results.size(), is(5));
-    assertThat("infinite range should return full set", results, hasItems(b, e, h, m, z));
-
-    results = KeyExtent.getKeyExtentsForRange(new Text("a"), new Text("z"), set0);
-    assertThat("full overlap should return full set", results.size(), is(5));
-    assertThat("full overlap should return full set", results, hasItems(b, e, h, m, z));
-
-    results = KeyExtent.getKeyExtentsForRange(null, new Text("f"), set0);
-    assertThat("end row should return head set", results.size(), is(3));
-    assertThat("end row should return head set", results, hasItems(b, e, h));
-
-    results = KeyExtent.getKeyExtentsForRange(new Text("f"), null, set0);
-    assertThat("start row should return tail set", results.size(), is(3));
-    assertThat("start row should return tail set", results, hasItems(h, m, z));
-
-    results = KeyExtent.getKeyExtentsForRange(new Text("f"), new Text("g"), set0);
-    assertThat("slice should return correct subset", results.size(), is(1));
-    assertThat("slice should return correct subset", results, hasItem(h));
-  }
-
   @Test
   public void testDecodeEncode() {
     assertNull(KeyExtent.decodePrevEndRow(KeyExtent.encodePrevEndRow(null)));
diff --git a/core/src/test/java/org/apache/accumulo/core/data/ValueTest.java b/core/src/test/java/org/apache/accumulo/core/data/ValueTest.java
index 9cdfaad..3257530 100644
--- a/core/src/test/java/org/apache/accumulo/core/data/ValueTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/data/ValueTest.java
@@ -33,7 +33,6 @@ import java.io.ByteArrayOutputStream;
 import java.io.DataInputStream;
 import java.io.DataOutputStream;
 import java.nio.ByteBuffer;
-import java.util.List;
 
 import org.apache.hadoop.io.Text;
 import org.junit.Before;
@@ -103,13 +102,6 @@ public class ValueTest {
   }
 
   @Test
-  public void testByteBufferCopy() {
-    @SuppressWarnings("deprecation")
-    Value v = new Value(DATABUFF, true);
-    assertArrayEquals(DATA, v.get());
-  }
-
-  @Test
   public void testValueCopy() {
     Value ov = createMock(Value.class);
     expect(ov.get()).andReturn(DATA);
@@ -200,24 +192,6 @@ public class ValueTest {
   }
 
   @Test
-  @Deprecated
-  public void testToArray() {
-    List<byte[]> l = new java.util.ArrayList<>();
-    byte[] one = toBytes("one");
-    byte[] two = toBytes("two");
-    byte[] three = toBytes("three");
-    l.add(one);
-    l.add(two);
-    l.add(three);
-
-    byte[][] a = Value.toArray(l);
-    assertEquals(3, a.length);
-    assertArrayEquals(one, a[0]);
-    assertArrayEquals(two, a[1]);
-    assertArrayEquals(three, a[2]);
-  }
-
-  @Test
   public void testString() {
     Value v1 = new Value("abc");
     Value v2 = new Value("abc".getBytes(UTF_8));
diff --git a/core/src/test/java/org/apache/accumulo/core/iterators/AggregatingIteratorTest.java b/core/src/test/java/org/apache/accumulo/core/iterators/AggregatingIteratorTest.java
deleted file mode 100644
index a84e2c8..0000000
--- a/core/src/test/java/org/apache/accumulo/core/iterators/AggregatingIteratorTest.java
+++ /dev/null
@@ -1,474 +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.iterators;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.TreeMap;
-
-import org.apache.accumulo.core.data.ByteSequence;
-import org.apache.accumulo.core.data.Key;
-import org.apache.accumulo.core.data.Range;
-import org.apache.accumulo.core.data.Value;
-import org.apache.accumulo.core.iterators.system.MultiIterator;
-import org.apache.hadoop.io.Text;
-import org.junit.Test;
-
-public class AggregatingIteratorTest {
-
-  private static final Collection<ByteSequence> EMPTY_COL_FAMS = new ArrayList<>();
-
-  /**
-   * @deprecated since 1.4; visible only for testing
-   */
-  @Deprecated
-  public static class SummationAggregator
-      implements org.apache.accumulo.core.iterators.aggregation.Aggregator {
-
-    int sum;
-
-    @Override
-    public Value aggregate() {
-      return new Value((sum + "").getBytes());
-    }
-
-    @Override
-    public void collect(Value value) {
-      int val = Integer.parseInt(value.toString());
-
-      sum += val;
-    }
-
-    @Override
-    public void reset() {
-      sum = 0;
-
-    }
-
-  }
-
-  static Key newKey(int row, int colf, int colq, long ts, boolean deleted) {
-    Key k = newKey(row, colf, colq, ts);
-    k.setDeleted(true);
-    return k;
-  }
-
-  static Key newKey(int row, int colf, int colq, long ts) {
-    return new Key(newRow(row), new Text(String.format("cf%03d", colf)),
-        new Text(String.format("cq%03d", colq)), ts);
-  }
-
-  static Range newRow(int row, int colf, int colq, long ts, boolean inclusive) {
-    return new Range(newKey(row, colf, colq, ts), inclusive, null, true);
-  }
-
-  static Range newRow(int row, int colf, int colq, long ts) {
-    return newRow(row, colf, colq, ts, true);
-  }
-
-  static void newKeyValue(TreeMap<Key,Value> tm, int row, int colf, int colq, long ts,
-      boolean deleted, String val) {
-    Key k = newKey(row, colf, colq, ts);
-    k.setDeleted(deleted);
-    tm.put(k, new Value(val.getBytes()));
-  }
-
-  static Text newRow(int row) {
-    return new Text(String.format("r%03d", row));
-  }
-
-  @SuppressWarnings("deprecation")
-  @Test
-  public void test1() throws IOException {
-
-    TreeMap<Key,Value> tm1 = new TreeMap<>();
-
-    // keys that do not aggregate
-    newKeyValue(tm1, 1, 1, 1, 1, false, "2");
-    newKeyValue(tm1, 1, 1, 1, 2, false, "3");
-    newKeyValue(tm1, 1, 1, 1, 3, false, "4");
-
-    AggregatingIterator ai = new AggregatingIterator();
-
-    Map<String,String> emptyMap = Collections.emptyMap();
-    ai.init(new SortedMapIterator(tm1), emptyMap, null);
-    ai.seek(new Range(), EMPTY_COL_FAMS, false);
-
-    assertTrue(ai.hasTop());
-    assertEquals(newKey(1, 1, 1, 3), ai.getTopKey());
-    assertEquals("4", ai.getTopValue().toString());
-
-    ai.next();
-
-    assertTrue(ai.hasTop());
-    assertEquals(newKey(1, 1, 1, 2), ai.getTopKey());
-    assertEquals("3", ai.getTopValue().toString());
-
-    ai.next();
-
-    assertTrue(ai.hasTop());
-    assertEquals(newKey(1, 1, 1, 1), ai.getTopKey());
-    assertEquals("2", ai.getTopValue().toString());
-
-    ai.next();
-
-    assertFalse(ai.hasTop());
-
-    // try seeking
-
-    ai.seek(newRow(1, 1, 1, 2), EMPTY_COL_FAMS, false);
-
-    assertTrue(ai.hasTop());
-    assertEquals(newKey(1, 1, 1, 2), ai.getTopKey());
-    assertEquals("3", ai.getTopValue().toString());
-
-    ai.next();
-
-    assertTrue(ai.hasTop());
-    assertEquals(newKey(1, 1, 1, 1), ai.getTopKey());
-    assertEquals("2", ai.getTopValue().toString());
-
-    ai.next();
-
-    assertFalse(ai.hasTop());
-
-    // seek after everything
-    ai.seek(newRow(1, 1, 1, 0), EMPTY_COL_FAMS, false);
-
-    assertFalse(ai.hasTop());
-
-  }
-
-  @SuppressWarnings("deprecation")
-  @Test
-  public void test2() throws IOException {
-    TreeMap<Key,Value> tm1 = new TreeMap<>();
-
-    // keys that aggregate
-    newKeyValue(tm1, 1, 1, 1, 1, false, "2");
-    newKeyValue(tm1, 1, 1, 1, 2, false, "3");
-    newKeyValue(tm1, 1, 1, 1, 3, false, "4");
-
-    AggregatingIterator ai = new AggregatingIterator();
-
-    Map<String,String> opts = new HashMap<>();
-
-    opts.put("cf001", SummationAggregator.class.getName());
-
-    ai.init(new SortedMapIterator(tm1), opts, null);
-    ai.seek(new Range(), EMPTY_COL_FAMS, false);
-
-    assertTrue(ai.hasTop());
-    assertEquals(newKey(1, 1, 1, 3), ai.getTopKey());
-    assertEquals("9", ai.getTopValue().toString());
-
-    ai.next();
-
-    assertFalse(ai.hasTop());
-
-    // try seeking to the beginning of a key that aggregates
-
-    ai.seek(newRow(1, 1, 1, 3), EMPTY_COL_FAMS, false);
-
-    assertTrue(ai.hasTop());
-    assertEquals(newKey(1, 1, 1, 3), ai.getTopKey());
-    assertEquals("9", ai.getTopValue().toString());
-
-    ai.next();
-
-    assertFalse(ai.hasTop());
-
-    // try seeking the middle of a key the aggregates
-    ai.seek(newRow(1, 1, 1, 2), EMPTY_COL_FAMS, false);
-
-    assertFalse(ai.hasTop());
-
-    // try seeking to the end of a key the aggregates
-    ai.seek(newRow(1, 1, 1, 1), EMPTY_COL_FAMS, false);
-
-    assertFalse(ai.hasTop());
-
-    // try seeking before a key the aggregates
-    ai.seek(newRow(1, 1, 1, 4), EMPTY_COL_FAMS, false);
-
-    assertTrue(ai.hasTop());
-    assertEquals(newKey(1, 1, 1, 3), ai.getTopKey());
-    assertEquals("9", ai.getTopValue().toString());
-
-    ai.next();
-
-    assertFalse(ai.hasTop());
-  }
-
-  @SuppressWarnings("deprecation")
-  @Test
-  public void test3() throws IOException {
-
-    TreeMap<Key,Value> tm1 = new TreeMap<>();
-
-    // keys that aggregate
-    newKeyValue(tm1, 1, 1, 1, 1, false, "2");
-    newKeyValue(tm1, 1, 1, 1, 2, false, "3");
-    newKeyValue(tm1, 1, 1, 1, 3, false, "4");
-
-    // keys that do not aggregate
-    newKeyValue(tm1, 2, 2, 1, 1, false, "2");
-    newKeyValue(tm1, 2, 2, 1, 2, false, "3");
-
-    AggregatingIterator ai = new AggregatingIterator();
-
-    Map<String,String> opts = new HashMap<>();
-
-    opts.put("cf001", SummationAggregator.class.getName());
-
-    ai.init(new SortedMapIterator(tm1), opts, null);
-    ai.seek(new Range(), EMPTY_COL_FAMS, false);
-
-    assertTrue(ai.hasTop());
-    assertEquals(newKey(1, 1, 1, 3), ai.getTopKey());
-    assertEquals("9", ai.getTopValue().toString());
-
-    ai.next();
-
-    assertTrue(ai.hasTop());
-    assertEquals(newKey(2, 2, 1, 2), ai.getTopKey());
-    assertEquals("3", ai.getTopValue().toString());
-
-    ai.next();
-
-    assertTrue(ai.hasTop());
-    assertEquals(newKey(2, 2, 1, 1), ai.getTopKey());
-    assertEquals("2", ai.getTopValue().toString());
-
-    ai.next();
-
-    assertFalse(ai.hasTop());
-
-    // seek after key that aggregates
-    ai.seek(newRow(1, 1, 1, 2), EMPTY_COL_FAMS, false);
-
-    assertTrue(ai.hasTop());
-    assertEquals(newKey(2, 2, 1, 2), ai.getTopKey());
-    assertEquals("3", ai.getTopValue().toString());
-
-    // seek before key that aggregates
-    ai.seek(newRow(1, 1, 1, 4), EMPTY_COL_FAMS, false);
-
-    assertTrue(ai.hasTop());
-    assertEquals(newKey(1, 1, 1, 3), ai.getTopKey());
-    assertEquals("9", ai.getTopValue().toString());
-
-    ai.next();
-
-    assertTrue(ai.hasTop());
-    assertEquals(newKey(2, 2, 1, 2), ai.getTopKey());
-    assertEquals("3", ai.getTopValue().toString());
-
-  }
-
-  @SuppressWarnings("deprecation")
-  @Test
-  public void test4() throws IOException {
-
-    TreeMap<Key,Value> tm1 = new TreeMap<>();
-
-    // keys that do not aggregate
-    newKeyValue(tm1, 0, 0, 1, 1, false, "7");
-
-    // keys that aggregate
-    newKeyValue(tm1, 1, 1, 1, 1, false, "2");
-    newKeyValue(tm1, 1, 1, 1, 2, false, "3");
-    newKeyValue(tm1, 1, 1, 1, 3, false, "4");
-
-    // keys that do not aggregate
-    newKeyValue(tm1, 2, 2, 1, 1, false, "2");
-    newKeyValue(tm1, 2, 2, 1, 2, false, "3");
-
-    AggregatingIterator ai = new AggregatingIterator();
-
-    Map<String,String> opts = new HashMap<>();
-
-    opts.put("cf001", SummationAggregator.class.getName());
-
-    ai.init(new SortedMapIterator(tm1), opts, null);
-    ai.seek(new Range(), EMPTY_COL_FAMS, false);
-
-    assertTrue(ai.hasTop());
-    assertEquals(newKey(0, 0, 1, 1), ai.getTopKey());
-    assertEquals("7", ai.getTopValue().toString());
-
-    ai.next();
-
-    assertTrue(ai.hasTop());
-    assertEquals(newKey(1, 1, 1, 3), ai.getTopKey());
-    assertEquals("9", ai.getTopValue().toString());
-
-    ai.next();
-
-    assertTrue(ai.hasTop());
-    assertEquals(newKey(2, 2, 1, 2), ai.getTopKey());
-    assertEquals("3", ai.getTopValue().toString());
-
-    ai.next();
-
-    assertTrue(ai.hasTop());
-    assertEquals(newKey(2, 2, 1, 1), ai.getTopKey());
-    assertEquals("2", ai.getTopValue().toString());
-
-    ai.next();
-
-    assertFalse(ai.hasTop());
-
-    // seek test
-    ai.seek(newRow(0, 0, 1, 0), EMPTY_COL_FAMS, false);
-
-    assertTrue(ai.hasTop());
-    assertEquals(newKey(1, 1, 1, 3), ai.getTopKey());
-    assertEquals("9", ai.getTopValue().toString());
-
-    ai.next();
-
-    assertTrue(ai.hasTop());
-    assertEquals(newKey(2, 2, 1, 2), ai.getTopKey());
-    assertEquals("3", ai.getTopValue().toString());
-
-    // seek after key that aggregates
-    ai.seek(newRow(1, 1, 1, 2), EMPTY_COL_FAMS, false);
-
-    assertTrue(ai.hasTop());
-    assertEquals(newKey(2, 2, 1, 2), ai.getTopKey());
-    assertEquals("3", ai.getTopValue().toString());
-
-  }
-
-  @SuppressWarnings("deprecation")
-  @Test
-  public void test5() throws IOException {
-    // try aggregating across multiple data sets that contain
-    // the exact same keys w/ different values
-
-    TreeMap<Key,Value> tm1 = new TreeMap<>();
-    newKeyValue(tm1, 1, 1, 1, 1, false, "2");
-
-    TreeMap<Key,Value> tm2 = new TreeMap<>();
-    newKeyValue(tm2, 1, 1, 1, 1, false, "3");
-
-    TreeMap<Key,Value> tm3 = new TreeMap<>();
-    newKeyValue(tm3, 1, 1, 1, 1, false, "4");
-
-    AggregatingIterator ai = new AggregatingIterator();
-    Map<String,String> opts = new HashMap<>();
-    opts.put("cf001", SummationAggregator.class.getName());
-
-    List<SortedKeyValueIterator<Key,Value>> sources = new ArrayList<>(3);
-    sources.add(new SortedMapIterator(tm1));
-    sources.add(new SortedMapIterator(tm2));
-    sources.add(new SortedMapIterator(tm3));
-
-    MultiIterator mi = new MultiIterator(sources, true);
-    ai.init(mi, opts, null);
-    ai.seek(new Range(), EMPTY_COL_FAMS, false);
-
-    assertTrue(ai.hasTop());
-    assertEquals(newKey(1, 1, 1, 1), ai.getTopKey());
-    assertEquals("9", ai.getTopValue().toString());
-  }
-
-  @SuppressWarnings("deprecation")
-  @Test
-  public void test6() throws IOException {
-    TreeMap<Key,Value> tm1 = new TreeMap<>();
-
-    // keys that aggregate
-    newKeyValue(tm1, 1, 1, 1, 1, false, "2");
-    newKeyValue(tm1, 1, 1, 1, 2, false, "3");
-    newKeyValue(tm1, 1, 1, 1, 3, false, "4");
-
-    AggregatingIterator ai = new AggregatingIterator();
-
-    Map<String,String> opts = new HashMap<>();
-
-    opts.put("cf001", SummationAggregator.class.getName());
-
-    ai.init(new SortedMapIterator(tm1), opts, new DefaultIteratorEnvironment());
-
-    // try seeking to the beginning of a key that aggregates
-
-    ai.seek(newRow(1, 1, 1, 3, false), EMPTY_COL_FAMS, false);
-
-    assertFalse(ai.hasTop());
-
-  }
-
-  @SuppressWarnings("deprecation")
-  @Test
-  public void test7() throws IOException {
-    // test that delete is not aggregated
-
-    TreeMap<Key,Value> tm1 = new TreeMap<>();
-
-    newKeyValue(tm1, 1, 1, 1, 2, true, "");
-    newKeyValue(tm1, 1, 1, 1, 3, false, "4");
-    newKeyValue(tm1, 1, 1, 1, 4, false, "3");
-
-    AggregatingIterator ai = new AggregatingIterator();
-
-    Map<String,String> opts = new HashMap<>();
-
-    opts.put("cf001", SummationAggregator.class.getName());
-
-    ai.init(new SortedMapIterator(tm1), opts, new DefaultIteratorEnvironment());
-
-    ai.seek(newRow(1, 1, 1, 4, true), EMPTY_COL_FAMS, false);
-
-    assertTrue(ai.hasTop());
-    assertEquals(newKey(1, 1, 1, 4), ai.getTopKey());
-    assertEquals("7", ai.getTopValue().toString());
-
-    ai.next();
-    assertTrue(ai.hasTop());
-    assertEquals(newKey(1, 1, 1, 2, true), ai.getTopKey());
-    assertEquals("", ai.getTopValue().toString());
-
-    ai.next();
-    assertFalse(ai.hasTop());
-
-    tm1 = new TreeMap<>();
-    newKeyValue(tm1, 1, 1, 1, 2, true, "");
-    ai = new AggregatingIterator();
-    ai.init(new SortedMapIterator(tm1), opts, new DefaultIteratorEnvironment());
-
-    ai.seek(newRow(1, 1, 1, 4, true), EMPTY_COL_FAMS, false);
-
-    assertTrue(ai.hasTop());
-    assertEquals(newKey(1, 1, 1, 2, true), ai.getTopKey());
-    assertEquals("", ai.getTopValue().toString());
-
-    ai.next();
-    assertFalse(ai.hasTop());
-
-  }
-}
diff --git a/core/src/test/java/org/apache/accumulo/core/iterators/aggregation/NumSummationTest.java b/core/src/test/java/org/apache/accumulo/core/iterators/aggregation/NumSummationTest.java
deleted file mode 100644
index 680f385..0000000
--- a/core/src/test/java/org/apache/accumulo/core/iterators/aggregation/NumSummationTest.java
+++ /dev/null
@@ -1,149 +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.iterators.aggregation;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.fail;
-
-import java.io.IOException;
-
-import org.apache.accumulo.core.data.Value;
-import org.junit.Test;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * @deprecated since 1.4
- */
-@Deprecated
-public class NumSummationTest {
-
-  private static final Logger log = LoggerFactory.getLogger(NumSummationTest.class);
-
-  public byte[] init(int n) {
-    byte[] b = new byte[n];
-    for (int i = 0; i < b.length; i++)
-      b[i] = 0;
-    return b;
-  }
-
-  @Test
-  public void test1() {
-    try {
-      long[] la = {1L, 2L, 3L};
-      byte[] b = NumArraySummation.longArrayToBytes(la);
-      long[] la2 = NumArraySummation.bytesToLongArray(b);
-
-      assertEquals(la.length, la2.length);
-      for (int i = 0; i < la.length; i++) {
-        assertEquals(i + ": " + la[i] + " does not equal " + la2[i], la[i], la2[i]);
-      }
-    } catch (Exception e) {
-      fail();
-    }
-  }
-
-  @Test
-  public void test2() {
-    try {
-      NumArraySummation nas = new NumArraySummation();
-      long[] la = {1L, 2L, 3L};
-      nas.collect(new Value(NumArraySummation.longArrayToBytes(la)));
-      long[] la2 = {3L, 2L, 1L, 0L};
-      nas.collect(new Value(NumArraySummation.longArrayToBytes(la2)));
-      la = NumArraySummation.bytesToLongArray(nas.aggregate().get());
-      assertEquals(4, la.length);
-      for (int i = 0; i < la.length - 1; i++) {
-        assertEquals(4, la[i]);
-      }
-      assertEquals(0, la[la.length - 1]);
-      nas.reset();
-      la = NumArraySummation.bytesToLongArray(nas.aggregate().get());
-      assertEquals(0, la.length);
-    } catch (Exception e) {
-      log.error("{}", e.getMessage(), e);
-      fail();
-    }
-  }
-
-  @Test
-  public void test3() {
-    try {
-      NumArraySummation nas = new NumArraySummation();
-      long[] la = {Long.MAX_VALUE, Long.MIN_VALUE, 3L, -5L, 5L, 5L};
-      nas.collect(new Value(NumArraySummation.longArrayToBytes(la)));
-      long[] la2 = {1L, -3L, 2L, 10L};
-      nas.collect(new Value(NumArraySummation.longArrayToBytes(la2)));
-      la = NumArraySummation.bytesToLongArray(nas.aggregate().get());
-      assertEquals(6, la.length);
-      for (int i = 2; i < la.length; i++) {
-        assertEquals(5, la[i]);
-      }
-      assertEquals("max long plus one was " + la[0], la[0], Long.MAX_VALUE);
-      assertEquals("min long minus 3 was " + la[1], la[1], Long.MIN_VALUE);
-    } catch (Exception e) {
-      fail();
-    }
-  }
-
-  @Test
-  public void test4() {
-    try {
-      long l = 5L;
-      byte[] b = NumSummation.longToBytes(l);
-      long l2 = NumSummation.bytesToLong(b);
-
-      assertEquals(l, l2);
-    } catch (Exception e) {
-      fail();
-    }
-  }
-
-  @Test
-  public void test5() {
-    try {
-      NumSummation ns = new NumSummation();
-      for (long l = -5L; l < 8L; l++) {
-        ns.collect(new Value(NumSummation.longToBytes(l)));
-      }
-      long l = NumSummation.bytesToLong(ns.aggregate().get());
-      assertEquals("l was " + l, 13, l);
-
-      ns.collect(new Value(NumSummation.longToBytes(Long.MAX_VALUE)));
-      l = NumSummation.bytesToLong(ns.aggregate().get());
-      assertEquals("l was " + l, l, Long.MAX_VALUE);
-
-      ns.collect(new Value(NumSummation.longToBytes(Long.MIN_VALUE)));
-      l = NumSummation.bytesToLong(ns.aggregate().get());
-      assertEquals("l was " + l, l, -1);
-
-      ns.collect(new Value(NumSummation.longToBytes(Long.MIN_VALUE)));
-      l = NumSummation.bytesToLong(ns.aggregate().get());
-      assertEquals("l was " + l, l, Long.MIN_VALUE);
-
-      ns.collect(new Value(NumSummation.longToBytes(Long.MIN_VALUE)));
-      l = NumSummation.bytesToLong(ns.aggregate().get());
-      assertEquals("l was " + l, l, Long.MIN_VALUE);
-
-      ns.reset();
-      l = NumSummation.bytesToLong(ns.aggregate().get());
-      assertEquals("l was " + l, 0, l);
-    } catch (IOException | RuntimeException e) {
-      fail();
-    }
-  }
-}
diff --git a/core/src/test/java/org/apache/accumulo/core/iterators/aggregation/conf/AggregatorConfigurationTest.java b/core/src/test/java/org/apache/accumulo/core/iterators/aggregation/conf/AggregatorConfigurationTest.java
deleted file mode 100644
index 373fcb6..0000000
--- a/core/src/test/java/org/apache/accumulo/core/iterators/aggregation/conf/AggregatorConfigurationTest.java
+++ /dev/null
@@ -1,79 +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.iterators.aggregation.conf;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNull;
-
-import org.apache.hadoop.io.Text;
-import org.junit.Test;
-
-public class AggregatorConfigurationTest {
-
-  @Test
-  public void testBinary() {
-    Text colf = new Text();
-    Text colq = new Text();
-
-    for (int i = 0; i < 256; i++) {
-      colf.append(new byte[] {(byte) i}, 0, 1);
-      colq.append(new byte[] {(byte) (255 - i)}, 0, 1);
-    }
-
-    runTest(colf, colq);
-    runTest(colf);
-  }
-
-  @Test
-  public void testBasic() {
-    runTest(new Text("colf1"), new Text("cq2"));
-    runTest(new Text("colf1"));
-  }
-
-  @SuppressWarnings("deprecation")
-  private void runTest(Text colf) {
-    String encodedCols;
-    // @formatter:off
-    org.apache.accumulo.core.iterators.conf.PerColumnIteratorConfig ac3 =
-      new org.apache.accumulo.core.iterators.conf.PerColumnIteratorConfig(colf, "com.foo.SuperAgg");
-    encodedCols = ac3.encodeColumns();
-    org.apache.accumulo.core.iterators.conf.PerColumnIteratorConfig ac4 =
-      org.apache.accumulo.core.iterators.conf.PerColumnIteratorConfig
-        .decodeColumns(encodedCols, "com.foo.SuperAgg");
-    // @formatter:on
-
-    assertEquals(colf, ac4.getColumnFamily());
-    assertNull(ac4.getColumnQualifier());
-  }
-
-  @SuppressWarnings("deprecation")
-  private void runTest(Text colf, Text colq) {
-    // @formatter:off
-    org.apache.accumulo.core.iterators.conf.PerColumnIteratorConfig ac =
-      new org.apache.accumulo.core.iterators.conf.PerColumnIteratorConfig(colf, colq,
-        "com.foo.SuperAgg");
-    String encodedCols = ac.encodeColumns();
-    org.apache.accumulo.core.iterators.conf.PerColumnIteratorConfig ac2 =
-      org.apache.accumulo.core.iterators.conf.PerColumnIteratorConfig
-        .decodeColumns(encodedCols, "com.foo.SuperAgg");
-    // @formatter:on
-
-    assertEquals(colf, ac2.getColumnFamily());
-    assertEquals(colq, ac2.getColumnQualifier());
-  }
-
-}
diff --git a/proxy/src/main/java/org/apache/accumulo/proxy/ProxyServer.java b/proxy/src/main/java/org/apache/accumulo/proxy/ProxyServer.java
index 0bb9f55..fb1b53e 100644
--- a/proxy/src/main/java/org/apache/accumulo/proxy/ProxyServer.java
+++ b/proxy/src/main/java/org/apache/accumulo/proxy/ProxyServer.java
@@ -225,7 +225,8 @@ public class ProxyServer implements AccumuloProxy.Iface {
     String[] pair = ByteBufferUtil.toString(login).split(",", 2);
     if (instance.getInstanceID().equals(pair[0])) {
       Credentials creds = Credentials.deserialize(pair[1]);
-      return (AccumuloClient) instance.getConnector(creds.getPrincipal(), creds.getToken());
+      return ((org.apache.accumulo.core.client.impl.ConnectorImpl) instance
+          .getConnector(creds.getPrincipal(), creds.getToken())).getAccumuloClient();
     } else {
       throw new org.apache.accumulo.core.client.AccumuloSecurityException(pair[0],
           org.apache.accumulo.core.client.impl.thrift.SecurityErrorCode.INVALID_INSTANCEID);
diff --git a/server/tracer/src/test/java/org/apache/accumulo/tracer/TracerTest.java b/server/tracer/src/test/java/org/apache/accumulo/tracer/TracerTest.java
index cc1df86..ef08361 100644
--- a/server/tracer/src/test/java/org/apache/accumulo/tracer/TracerTest.java
+++ b/server/tracer/src/test/java/org/apache/accumulo/tracer/TracerTest.java
@@ -102,7 +102,6 @@ public class TracerTest {
     public void close() throws IOException {}
   }
 
-  @SuppressWarnings("deprecation")
   @Test
   public void testTrace() throws Exception {
     TestReceiver tracer = new TestReceiver();
@@ -117,8 +116,7 @@ public class TracerTest {
     assertEquals(1, tracer.traces.size());
     assertFalse(Trace.isTracing());
 
-    Span start = Trace.on("testing");
-    assertEquals(Trace.currentTrace().getSpan(), start.getScope().getSpan());
+    Trace.on("testing");
     assertTrue(Trace.isTracing());
 
     Span span = Trace.start("shortest trace ever");
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/TservConstraintEnv.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/TservConstraintEnv.java
index 7f068c9..7c5ec2f 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/TservConstraintEnv.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/TservConstraintEnv.java
@@ -23,7 +23,6 @@ import org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException;
 import org.apache.accumulo.core.data.ByteSequence;
 import org.apache.accumulo.core.data.impl.KeyExtent;
 import org.apache.accumulo.core.security.AuthorizationContainer;
-import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.security.thrift.TCredentials;
 import org.apache.accumulo.server.ServerContext;
 import org.apache.accumulo.server.constraints.SystemEnvironment;
@@ -34,7 +33,6 @@ public class TservConstraintEnv implements SystemEnvironment {
   private final ServerContext context;
   private final TCredentials credentials;
   private final SecurityOperation security;
-  private Authorizations auths;
   private KeyExtent ke;
 
   TservConstraintEnv(ServerContext context, SecurityOperation secOp, TCredentials credentials) {
@@ -58,18 +56,6 @@ public class TservConstraintEnv implements SystemEnvironment {
   }
 
   @Override
-  @Deprecated
-  public Authorizations getAuthorizations() {
-    if (auths == null)
-      try {
-        this.auths = security.getUserAuthorizations(credentials);
-      } catch (ThriftSecurityException e) {
-        throw new RuntimeException(e);
-      }
-    return auths;
-  }
-
-  @Override
   public AuthorizationContainer getAuthorizationsContainer() {
     return new AuthorizationContainer() {
       @Override
diff --git a/shell/src/main/java/org/apache/accumulo/shell/commands/SetIterCommand.java b/shell/src/main/java/org/apache/accumulo/shell/commands/SetIterCommand.java
index 33fa798..8848d14 100644
--- a/shell/src/main/java/org/apache/accumulo/shell/commands/SetIterCommand.java
+++ b/shell/src/main/java/org/apache/accumulo/shell/commands/SetIterCommand.java
@@ -54,7 +54,7 @@ public class SetIterCommand extends Command {
 
   private Option allScopeOpt, mincScopeOpt, majcScopeOpt, scanScopeOpt;
   Option profileOpt, priorityOpt, nameOpt;
-  Option aggTypeOpt, ageoffTypeOpt, regexTypeOpt, versionTypeOpt, reqvisTypeOpt, classnameTypeOpt;
+  Option ageoffTypeOpt, regexTypeOpt, versionTypeOpt, reqvisTypeOpt, classnameTypeOpt;
 
   @Override
   public int execute(final String fullCommand, final CommandLine cl, final Shell shellState)
@@ -69,13 +69,7 @@ public class SetIterCommand extends Command {
 
     final Map<String,String> options = new HashMap<>();
     String classname = cl.getOptionValue(classnameTypeOpt.getOpt());
-    if (cl.hasOption(aggTypeOpt.getOpt())) {
-      Shell.log.warn("aggregators are deprecated");
-      @SuppressWarnings("deprecation")
-      String deprecatedClassName = org.apache.accumulo.core.iterators.AggregatingIterator.class
-          .getName();
-      classname = deprecatedClassName;
-    } else if (cl.hasOption(regexTypeOpt.getOpt())) {
+    if (cl.hasOption(regexTypeOpt.getOpt())) {
       classname = RegExFilter.class.getName();
     } else if (cl.hasOption(ageoffTypeOpt.getOpt())) {
       classname = AgeOffFilter.class.getName();
@@ -149,19 +143,6 @@ public class SetIterCommand extends Command {
 
     ScanCommand.ensureTserversCanLoadIterator(shellState, tableName, classname);
 
-    final String aggregatorClass = options.get("aggregatorClass");
-    // @formatter:off
-    @SuppressWarnings("deprecation")
-    String deprecatedAggregatorClassName =
-      org.apache.accumulo.core.iterators.aggregation.Aggregator.class.getName();
-    // @formatter:on
-    if (aggregatorClass != null && !shellState.getAccumuloClient().tableOperations()
-        .testClassLoad(tableName, aggregatorClass, deprecatedAggregatorClassName)) {
-      throw new ShellCommandException(ErrorCode.INITIALIZATION_FAILURE,
-          "Servers are unable to load " + aggregatorClass + " as type "
-              + deprecatedAggregatorClassName);
-    }
-
     for (Iterator<Entry<String,String>> i = options.entrySet().iterator(); i.hasNext();) {
       final Entry<String,String> entry = i.next();
       if (entry.getValue() == null || entry.getValue().isEmpty()) {
@@ -200,19 +181,6 @@ public class SetIterCommand extends Command {
               + SortedKeyValueIterator.class.getName());
     }
 
-    final String aggregatorClass = options.get("aggregatorClass");
-    // @formatter:off
-    @SuppressWarnings("deprecation")
-    String deprecatedAggregatorClassName =
-      org.apache.accumulo.core.iterators.aggregation.Aggregator.class.getName();
-    // @formatter:on
-    if (aggregatorClass != null && !shellState.getAccumuloClient().namespaceOperations()
-        .testClassLoad(namespace, aggregatorClass, deprecatedAggregatorClassName)) {
-      throw new ShellCommandException(ErrorCode.INITIALIZATION_FAILURE,
-          "Servers are unable to load " + aggregatorClass + " as type "
-              + deprecatedAggregatorClassName);
-    }
-
     for (Iterator<Entry<String,String>> i = options.entrySet().iterator(); i.hasNext();) {
       final Entry<String,String> entry = i.next();
       if (entry.getValue() == null || entry.getValue().isEmpty()) {
@@ -441,7 +409,6 @@ public class SetIterCommand extends Command {
     classnameTypeOpt = new Option("class", "class-name", true,
         "a java class that implements SortedKeyValueIterator");
     classnameTypeOpt.setArgName("name");
-    aggTypeOpt = new Option("agg", "aggregator", false, "an aggregating type");
     regexTypeOpt = new Option("regex", "regular-expression", false, "a regex matching iterator");
     versionTypeOpt = new Option("vers", "version", false, "a versioning iterator");
     reqvisTypeOpt = new Option("reqvis", "require-visibility", false,
@@ -449,7 +416,6 @@ public class SetIterCommand extends Command {
     ageoffTypeOpt = new Option("ageoff", "ageoff", false, "an aging off iterator");
 
     typeGroup.addOption(classnameTypeOpt);
-    typeGroup.addOption(aggTypeOpt);
     typeGroup.addOption(regexTypeOpt);
     typeGroup.addOption(versionTypeOpt);
     typeGroup.addOption(reqvisTypeOpt);
diff --git a/shell/src/test/java/org/apache/accumulo/shell/commands/SetIterCommandTest.java b/shell/src/test/java/org/apache/accumulo/shell/commands/SetIterCommandTest.java
index b4fdc97..f120af2 100644
--- a/shell/src/test/java/org/apache/accumulo/shell/commands/SetIterCommandTest.java
+++ b/shell/src/test/java/org/apache/accumulo/shell/commands/SetIterCommandTest.java
@@ -67,7 +67,6 @@ public class SetIterCommandTest {
     EasyMock.expect(cli.getOptionValue("class"))
         .andReturn("org.apache.accumulo.core.iterators.user.ColumnAgeOffFilter");
     EasyMock.expect(cli.hasOption("ageoff")).andReturn(false);
-    EasyMock.expect(cli.hasOption("agg")).andReturn(false);
     EasyMock.expect(cli.hasOption("regex")).andReturn(false);
     EasyMock.expect(cli.hasOption("reqvis")).andReturn(false);
     EasyMock.expect(cli.hasOption("vers")).andReturn(false);
diff --git a/test/src/main/java/org/apache/accumulo/test/NewTableConfigurationIT.java b/test/src/main/java/org/apache/accumulo/test/NewTableConfigurationIT.java
index 9902c81..c2e092c 100644
--- a/test/src/main/java/org/apache/accumulo/test/NewTableConfigurationIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/NewTableConfigurationIT.java
@@ -18,8 +18,6 @@ package org.apache.accumulo.test;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
 
 import java.util.Collections;
 import java.util.EnumSet;
@@ -33,31 +31,20 @@ import org.apache.accumulo.core.client.AccumuloClient;
 import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.IteratorSetting;
-import org.apache.accumulo.core.client.Scanner;
 import org.apache.accumulo.core.client.TableExistsException;
 import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.client.admin.NewTableConfiguration;
-import org.apache.accumulo.core.client.admin.TimeType;
 import org.apache.accumulo.core.conf.Property;
-import org.apache.accumulo.core.data.Key;
-import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.iterators.IteratorUtil.IteratorScope;
-import org.apache.accumulo.core.metadata.MetadataTable;
-import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.ServerColumnFamily;
-import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.harness.SharedMiniClusterBase;
 import org.apache.hadoop.io.Text;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
 import com.google.common.collect.ImmutableSet;
-import com.google.common.collect.Iterators;
 
 public class NewTableConfigurationIT extends SharedMiniClusterBase {
-  private static final Logger log = LoggerFactory.getLogger(NewTableConfigurationIT.class);
 
   @Override
   protected int defaultTimeoutSeconds() {
@@ -678,150 +665,4 @@ public class NewTableConfigurationIT extends SharedMiniClusterBase {
     return propertyMap;
   }
 
-  public int numProperties(AccumuloClient accumuloClient, String tableName)
-      throws AccumuloException, TableNotFoundException {
-    return Iterators.size(accumuloClient.tableOperations().getProperties(tableName).iterator());
-  }
-
-  public int compareProperties(AccumuloClient accumuloClient, String tableNameOrig,
-      String tableName, String changedProp) throws AccumuloException, TableNotFoundException {
-    boolean inNew = false;
-    int countOrig = 0;
-    for (Entry<String,String> orig : accumuloClient.tableOperations()
-        .getProperties(tableNameOrig)) {
-      countOrig++;
-      for (Entry<String,String> entry : accumuloClient.tableOperations().getProperties(tableName)) {
-        if (entry.equals(orig)) {
-          inNew = true;
-          break;
-        } else if (entry.getKey().equals(orig.getKey()) && !entry.getKey().equals(changedProp))
-          fail("Property " + orig.getKey() + " has different value than deprecated method");
-      }
-      if (!inNew)
-        fail("Original property missing after using the new create method");
-    }
-    return countOrig;
-  }
-
-  public boolean checkTimeType(AccumuloClient accumuloClient, String tableName,
-      TimeType expectedTimeType) throws TableNotFoundException {
-    final Scanner scanner = accumuloClient.createScanner(MetadataTable.NAME, Authorizations.EMPTY);
-    String tableID = accumuloClient.tableOperations().tableIdMap().get(tableName) + "<";
-    for (Entry<Key,Value> entry : scanner) {
-      Key k = entry.getKey();
-
-      if (k.getRow().toString().equals(tableID) && k.getColumnQualifier().toString()
-          .equals(ServerColumnFamily.TIME_COLUMN.getColumnQualifier().toString())) {
-        if (expectedTimeType == TimeType.MILLIS && entry.getValue().toString().charAt(0) == 'M')
-          return true;
-        if (expectedTimeType == TimeType.LOGICAL && entry.getValue().toString().charAt(0) == 'L')
-          return true;
-      }
-    }
-    return false;
-  }
-
-  @SuppressWarnings("deprecation")
-  @Test
-  public void tableNameOnly() throws Exception {
-    log.info("Starting tableNameOnly");
-
-    // Create a table with the initial properties
-    AccumuloClient accumuloClient = getClient();
-    String tableName = getUniqueNames(2)[0];
-    accumuloClient.tableOperations().create(tableName, new NewTableConfiguration());
-
-    String tableNameOrig = "original";
-    accumuloClient.tableOperations().create(tableNameOrig, true);
-
-    int countNew = numProperties(accumuloClient, tableName);
-    int countOrig = compareProperties(accumuloClient, tableNameOrig, tableName, null);
-
-    assertEquals("Extra properties using the new create method", countOrig, countNew);
-    assertTrue("Wrong TimeType", checkTimeType(accumuloClient, tableName, TimeType.MILLIS));
-  }
-
-  @SuppressWarnings("deprecation")
-  @Test
-  public void tableNameAndLimitVersion() throws Exception {
-    log.info("Starting tableNameAndLimitVersion");
-
-    // Create a table with the initial properties
-    AccumuloClient accumuloClient = getClient();
-    String tableName = getUniqueNames(2)[0];
-    boolean limitVersion = false;
-    accumuloClient.tableOperations().create(tableName,
-        new NewTableConfiguration().withoutDefaultIterators());
-
-    String tableNameOrig = "originalWithLimitVersion";
-    accumuloClient.tableOperations().create(tableNameOrig, limitVersion);
-
-    int countNew = numProperties(accumuloClient, tableName);
-    int countOrig = compareProperties(accumuloClient, tableNameOrig, tableName, null);
-
-    assertEquals("Extra properties using the new create method", countOrig, countNew);
-    assertTrue("Wrong TimeType", checkTimeType(accumuloClient, tableName, TimeType.MILLIS));
-  }
-
-  @SuppressWarnings("deprecation")
-  @Test
-  public void tableNameLimitVersionAndTimeType() throws Exception {
-    log.info("Starting tableNameLimitVersionAndTimeType");
-
-    // Create a table with the initial properties
-    AccumuloClient accumuloClient = getClient();
-    String tableName = getUniqueNames(2)[0];
-    boolean limitVersion = false;
-    TimeType tt = TimeType.LOGICAL;
-    accumuloClient.tableOperations().create(tableName,
-        new NewTableConfiguration().withoutDefaultIterators().setTimeType(tt));
-
-    String tableNameOrig = "originalWithLimitVersionAndTimeType";
-    accumuloClient.tableOperations().create(tableNameOrig, limitVersion, tt);
-
-    int countNew = numProperties(accumuloClient, tableName);
-    int countOrig = compareProperties(accumuloClient, tableNameOrig, tableName, null);
-
-    assertEquals("Extra properties using the new create method", countOrig, countNew);
-    assertTrue("Wrong TimeType", checkTimeType(accumuloClient, tableName, tt));
-  }
-
-  @SuppressWarnings("deprecation")
-  @Test
-  public void addCustomPropAndChangeExisting() throws Exception {
-    log.info("Starting addCustomPropAndChangeExisting");
-
-    // Create and populate initial properties map for creating table 1
-    Map<String,String> properties = new HashMap<>();
-    String propertyName = Property.TABLE_SPLIT_THRESHOLD.getKey();
-    String volume = "10K";
-    properties.put(propertyName, volume);
-
-    String propertyName2 = "table.custom.testProp";
-    String volume2 = "Test property";
-    properties.put(propertyName2, volume2);
-
-    // Create a table with the initial properties
-    AccumuloClient accumuloClient = getClient();
-    String tableName = getUniqueNames(2)[0];
-    accumuloClient.tableOperations().create(tableName,
-        new NewTableConfiguration().setProperties(properties));
-
-    String tableNameOrig = "originalWithTableName";
-    accumuloClient.tableOperations().create(tableNameOrig, true);
-
-    int countNew = numProperties(accumuloClient, tableName);
-    int countOrig = compareProperties(accumuloClient, tableNameOrig, tableName, propertyName);
-
-    for (Entry<String,String> entry : accumuloClient.tableOperations().getProperties(tableName)) {
-      if (entry.getKey().equals(Property.TABLE_SPLIT_THRESHOLD.getKey()))
-        assertEquals("TABLE_SPLIT_THRESHOLD has been changed", "10K", entry.getValue());
-      if (entry.getKey().equals("table.custom.testProp"))
-        assertEquals("table.custom.testProp has been changed", "Test property", entry.getValue());
-    }
-
-    assertEquals("Extra properties using the new create method", countOrig + 1, countNew);
-    assertTrue("Wrong TimeType", checkTimeType(accumuloClient, tableName, TimeType.MILLIS));
-
-  }
 }
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/AccumuloClientIT.java b/test/src/main/java/org/apache/accumulo/test/functional/AccumuloClientIT.java
index 46b1f6b..a548ac2 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/AccumuloClientIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/AccumuloClientIT.java
@@ -31,6 +31,15 @@ import org.junit.Test;
 
 public class AccumuloClientIT extends AccumuloClusterHarness {
 
+  @SuppressWarnings("deprecation")
+  @Test
+  public void testGetConnectorFromAccumuloClient() {
+    AccumuloClient client = getAccumuloClient();
+    org.apache.accumulo.core.client.Connector c = org.apache.accumulo.core.client.Connector
+        .from(client);
+    assertEquals(client.whoami(), c.whoami());
+  }
+
   @Test
   public void testclientectorBuilder() throws Exception {
     AccumuloClient c = getAccumuloClient();