You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@accumulo.apache.org by kt...@apache.org on 2015/04/22 00:23:31 UTC

[5/8] accumulo git commit: ACCUMULO-3657 ACCUMULO-3548 Clarify Accumulo API

http://git-wip-us.apache.org/repos/asf/accumulo/blob/c6252914/core/src/main/java/org/apache/accumulo/core/client/mapred/AbstractInputFormat.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapred/AbstractInputFormat.java b/core/src/main/java/org/apache/accumulo/core/client/mapred/AbstractInputFormat.java
index df317e3..b97d4de 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mapred/AbstractInputFormat.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mapred/AbstractInputFormat.java
@@ -42,7 +42,11 @@ import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.client.TableOfflineException;
 import org.apache.accumulo.core.client.IteratorSetting;
 import org.apache.accumulo.core.client.admin.DelegationTokenConfig;
+import org.apache.accumulo.core.client.admin.SecurityOperations;
+import org.apache.accumulo.core.client.impl.AuthenticationTokenIdentifier;
 import org.apache.accumulo.core.client.impl.ClientContext;
+import org.apache.accumulo.core.client.impl.Credentials;
+import org.apache.accumulo.core.client.impl.DelegationTokenImpl;
 import org.apache.accumulo.core.client.impl.OfflineScanner;
 import org.apache.accumulo.core.client.impl.ScannerImpl;
 import org.apache.accumulo.core.client.impl.Tables;
@@ -58,13 +62,11 @@ import org.apache.accumulo.core.client.security.tokens.DelegationToken;
 import org.apache.accumulo.core.client.security.tokens.KerberosToken;
 import org.apache.accumulo.core.client.security.tokens.PasswordToken;
 import org.apache.accumulo.core.data.Key;
-import org.apache.accumulo.core.data.KeyExtent;
 import org.apache.accumulo.core.data.Range;
 import org.apache.accumulo.core.data.Value;
+import org.apache.accumulo.core.data.impl.KeyExtent;
 import org.apache.accumulo.core.master.state.tables.TableState;
-import org.apache.accumulo.core.security.AuthenticationTokenIdentifier;
 import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.core.security.Credentials;
 import org.apache.accumulo.core.util.Pair;
 import org.apache.accumulo.core.util.UtilWaitThread;
 import org.apache.hadoop.io.Text;
@@ -90,7 +92,8 @@ public abstract class AbstractInputFormat<K,V> implements InputFormat<K,V> {
    * <p>
    * <b>WARNING:</b> Some tokens, when serialized, divulge sensitive information in the configuration as a means to pass the token to MapReduce tasks. This
    * information is BASE64 encoded to provide a charset safe conversion to a string, but this conversion is not intended to be secure. {@link PasswordToken} is
-   * one example that is insecure in this way; however {@link DelegationToken}s, acquired using a {@link KerberosToken}, is not subject to this concern.
+   * one example that is insecure in this way; however {@link DelegationToken}s, acquired using
+   * {@link SecurityOperations#getDelegationToken(DelegationTokenConfig)}, is not subject to this concern.
    *
    * @param job
    *          the Hadoop job instance to be configured
@@ -112,8 +115,8 @@ public abstract class AbstractInputFormat<K,V> implements InputFormat<K,V> {
       }
     }
     // DelegationTokens can be passed securely from user to task without serializing insecurely in the configuration
-    if (token instanceof DelegationToken) {
-      DelegationToken delegationToken = (DelegationToken) token;
+    if (token instanceof DelegationTokenImpl) {
+      DelegationTokenImpl delegationToken = (DelegationTokenImpl) token;
 
       // Convert it into a Hadoop Token
       AuthenticationTokenIdentifier identifier = delegationToken.getIdentifier();
@@ -304,7 +307,10 @@ public abstract class AbstractInputFormat<K,V> implements InputFormat<K,V> {
    * @throws org.apache.accumulo.core.client.TableNotFoundException
    *           if the table name set on the configuration doesn't exist
    * @since 1.6.0
+   * @deprecated since 1.7.0 This method returns a type that is not part of the public API and is not guaranteed to be stable. The method was deprecated to
+   *             discourage its use.
    */
+  @Deprecated
   protected static TabletLocator getTabletLocator(JobConf job, String tableId) throws TableNotFoundException {
     return InputConfigurator.getTabletLocator(CLASS, job, tableId);
   }
@@ -669,7 +675,7 @@ public abstract class AbstractInputFormat<K,V> implements InputFormat<K,V> {
             binnedRanges = binOfflineTable(job, tableId, ranges);
           }
         } else {
-          tl = getTabletLocator(job, tableId);
+          tl = InputConfigurator.getTabletLocator(CLASS, job, tableId);
           // its possible that the cache could contain complete, but old information about a tables tablets... so clear it
           tl.invalidateCache();
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/c6252914/core/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloFileOutputFormat.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloFileOutputFormat.java b/core/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloFileOutputFormat.java
index 2f2b4b2..0eb304f 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloFileOutputFormat.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloFileOutputFormat.java
@@ -62,7 +62,10 @@ public class AccumuloFileOutputFormat extends FileOutputFormat<Key,Value> {
    * @param job
    *          the Hadoop context for the configured job
    * @since 1.5.0
+   * @deprecated since 1.7.0 This method returns a type that is not part of the public API and is not guaranteed to be stable. The method was deprecated to
+   *             discourage its use.
    */
+  @Deprecated
   protected static AccumuloConfiguration getAccumuloConfiguration(JobConf job) {
     return FileOutputConfigurator.getAccumuloConfiguration(CLASS, job);
   }
@@ -141,7 +144,7 @@ public class AccumuloFileOutputFormat extends FileOutputFormat<Key,Value> {
   public RecordWriter<Key,Value> getRecordWriter(FileSystem ignored, JobConf job, String name, Progressable progress) throws IOException {
     // get the path of the temporary output file
     final Configuration conf = job;
-    final AccumuloConfiguration acuConf = getAccumuloConfiguration(job);
+    final AccumuloConfiguration acuConf = FileOutputConfigurator.getAccumuloConfiguration(CLASS, job);
 
     final String extension = acuConf.get(Property.TABLE_FILE_TYPE);
     final Path file = new Path(getWorkOutputPath(job), getUniqueName(job, "part") + "." + extension);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/c6252914/core/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloOutputFormat.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloOutputFormat.java b/core/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloOutputFormat.java
index e0ac2c0..62ca05c 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloOutputFormat.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloOutputFormat.java
@@ -35,6 +35,9 @@ import org.apache.accumulo.core.client.TableExistsException;
 import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.client.ZooKeeperInstance;
 import org.apache.accumulo.core.client.admin.DelegationTokenConfig;
+import org.apache.accumulo.core.client.admin.SecurityOperations;
+import org.apache.accumulo.core.client.impl.AuthenticationTokenIdentifier;
+import org.apache.accumulo.core.client.impl.DelegationTokenImpl;
 import org.apache.accumulo.core.client.mapreduce.lib.impl.ConfiguratorBase;
 import org.apache.accumulo.core.client.mapreduce.lib.impl.OutputConfigurator;
 import org.apache.accumulo.core.client.mock.MockInstance;
@@ -43,10 +46,10 @@ import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
 import org.apache.accumulo.core.client.security.tokens.AuthenticationToken.AuthenticationTokenSerializer;
 import org.apache.accumulo.core.client.security.tokens.DelegationToken;
 import org.apache.accumulo.core.client.security.tokens.KerberosToken;
+import org.apache.accumulo.core.client.security.tokens.PasswordToken;
 import org.apache.accumulo.core.data.ColumnUpdate;
-import org.apache.accumulo.core.data.KeyExtent;
 import org.apache.accumulo.core.data.Mutation;
-import org.apache.accumulo.core.security.AuthenticationTokenIdentifier;
+import org.apache.accumulo.core.data.TabletID;
 import org.apache.accumulo.core.security.ColumnVisibility;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.io.Text;
@@ -80,10 +83,12 @@ public class AccumuloOutputFormat implements OutputFormat<Text,Mutation> {
 
   /**
    * Sets the connector information needed to communicate with Accumulo in this job.
-   *
+
    * <p>
-   * <b>WARNING:</b> The serialized token is stored in the configuration and shared with all MapReduce tasks. It is BASE64 encoded to provide a charset safe
-   * conversion to a string, and is not intended to be secure.
+   * <b>WARNING:</b> Some tokens, when serialized, divulge sensitive information in the configuration as a means to pass the token to MapReduce tasks. This
+   * information is BASE64 encoded to provide a charset safe conversion to a string, but this conversion is not intended to be secure. {@link PasswordToken} is
+   * one example that is insecure in this way; however {@link DelegationToken}s, acquired using
+   * {@link SecurityOperations#getDelegationToken(DelegationTokenConfig)}, is not subject to this concern.
    *
    * @param job
    *          the Hadoop job instance to be configured
@@ -105,8 +110,8 @@ public class AccumuloOutputFormat implements OutputFormat<Text,Mutation> {
       }
     }
     // DelegationTokens can be passed securely from user to task without serializing insecurely in the configuration
-    if (token instanceof DelegationToken) {
-      DelegationToken delegationToken = (DelegationToken) token;
+    if (token instanceof DelegationTokenImpl) {
+      DelegationTokenImpl delegationToken = (DelegationTokenImpl) token;
 
       // Convert it into a Hadoop Token
       AuthenticationTokenIdentifier identifier = delegationToken.getIdentifier();
@@ -537,9 +542,9 @@ public class AccumuloOutputFormat implements OutputFormat<Text,Mutation> {
       try {
         mtbw.close();
       } catch (MutationsRejectedException e) {
-        if (e.getAuthorizationFailuresMap().size() >= 0) {
+        if (e.getSecurityErrorCodes().size() >= 0) {
           HashMap<String,Set<SecurityErrorCode>> tables = new HashMap<String,Set<SecurityErrorCode>>();
-          for (Entry<KeyExtent,Set<SecurityErrorCode>> ke : e.getAuthorizationFailuresMap().entrySet()) {
+          for (Entry<TabletID,Set<SecurityErrorCode>> ke : e.getSecurityErrorCodes().entrySet()) {
             Set<SecurityErrorCode> secCodes = tables.get(ke.getKey().getTableId().toString());
             if (secCodes == null) {
               secCodes = new HashSet<SecurityErrorCode>();

http://git-wip-us.apache.org/repos/asf/accumulo/blob/c6252914/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AbstractInputFormat.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AbstractInputFormat.java b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AbstractInputFormat.java
index e3c3c42..c7a304c 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AbstractInputFormat.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AbstractInputFormat.java
@@ -41,7 +41,11 @@ import org.apache.accumulo.core.client.TableDeletedException;
 import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.client.TableOfflineException;
 import org.apache.accumulo.core.client.admin.DelegationTokenConfig;
+import org.apache.accumulo.core.client.admin.SecurityOperations;
+import org.apache.accumulo.core.client.impl.AuthenticationTokenIdentifier;
 import org.apache.accumulo.core.client.impl.ClientContext;
+import org.apache.accumulo.core.client.impl.Credentials;
+import org.apache.accumulo.core.client.impl.DelegationTokenImpl;
 import org.apache.accumulo.core.client.impl.OfflineScanner;
 import org.apache.accumulo.core.client.impl.ScannerImpl;
 import org.apache.accumulo.core.client.impl.Tables;
@@ -56,13 +60,11 @@ import org.apache.accumulo.core.client.security.tokens.DelegationToken;
 import org.apache.accumulo.core.client.security.tokens.KerberosToken;
 import org.apache.accumulo.core.client.security.tokens.PasswordToken;
 import org.apache.accumulo.core.data.Key;
-import org.apache.accumulo.core.data.KeyExtent;
 import org.apache.accumulo.core.data.Range;
 import org.apache.accumulo.core.data.Value;
+import org.apache.accumulo.core.data.impl.KeyExtent;
 import org.apache.accumulo.core.master.state.tables.TableState;
-import org.apache.accumulo.core.security.AuthenticationTokenIdentifier;
 import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.core.security.Credentials;
 import org.apache.accumulo.core.util.Pair;
 import org.apache.accumulo.core.util.UtilWaitThread;
 import org.apache.accumulo.core.client.IteratorSetting;
@@ -91,9 +93,10 @@ public abstract class AbstractInputFormat<K,V> extends InputFormat<K,V> {
    * Sets the connector information needed to communicate with Accumulo in this job.
    *
    * <p>
-   * <b>WARNING:</b> For {@link PasswordToken}, the serialized token is stored in the configuration and shared with all MapReduce tasks. It is BASE64 encoded to
-   * provide a charset safe conversion to a string, and is not intended to be secure. This is not the case for {@link KerberosToken} and the corresponding
-   * {@link DelegationToken} acquired using the KerberosToken.
+   * <b>WARNING:</b> Some tokens, when serialized, divulge sensitive information in the configuration as a means to pass the token to MapReduce tasks. This
+   * information is BASE64 encoded to provide a charset safe conversion to a string, but this conversion is not intended to be secure. {@link PasswordToken} is
+   * one example that is insecure in this way; however {@link DelegationToken}s, acquired using
+   * {@link SecurityOperations#getDelegationToken(DelegationTokenConfig)}, is not subject to this concern.
    *
    * @param job
    *          the Hadoop job instance to be configured
@@ -115,8 +118,8 @@ public abstract class AbstractInputFormat<K,V> extends InputFormat<K,V> {
       }
     }
     // DelegationTokens can be passed securely from user to task without serializing insecurely in the configuration
-    if (token instanceof DelegationToken) {
-      DelegationToken delegationToken = (DelegationToken) token;
+    if (token instanceof DelegationTokenImpl) {
+      DelegationTokenImpl delegationToken = (DelegationTokenImpl) token;
 
       // Convert it into a Hadoop Token
       AuthenticationTokenIdentifier identifier = delegationToken.getIdentifier();
@@ -360,7 +363,10 @@ public abstract class AbstractInputFormat<K,V> extends InputFormat<K,V> {
    * @throws org.apache.accumulo.core.client.TableNotFoundException
    *           if the table name set on the configuration doesn't exist
    * @since 1.6.0
+   * @deprecated since 1.7.0 This method returns a type that is not part of the public API and is not guaranteed to be stable. The method was deprecated to
+   *              discourage its use.
    */
+  @Deprecated
   protected static TabletLocator getTabletLocator(JobContext context, String table) throws TableNotFoundException {
     return InputConfigurator.getTabletLocator(CLASS, context.getConfiguration(), table);
   }
@@ -714,7 +720,7 @@ public abstract class AbstractInputFormat<K,V> extends InputFormat<K,V> {
 
           }
         } else {
-          tl = getTabletLocator(context, tableId);
+          tl = InputConfigurator.getTabletLocator(CLASS, context.getConfiguration(), tableId);
           // its possible that the cache could contain complete, but old information about a tables tablets... so clear it
           tl.invalidateCache();
 
@@ -802,4 +808,4 @@ public abstract class AbstractInputFormat<K,V> extends InputFormat<K,V> {
     }
     return splits;
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/c6252914/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloFileOutputFormat.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloFileOutputFormat.java b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloFileOutputFormat.java
index db7b689..abd96b6 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloFileOutputFormat.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloFileOutputFormat.java
@@ -60,7 +60,10 @@ public class AccumuloFileOutputFormat extends FileOutputFormat<Key,Value> {
    * @param context
    *          the Hadoop context for the configured job
    * @since 1.5.0
+   * @deprecated since 1.7.0 This method returns a type that is not part of the public API and is not guaranteed to be stable. The method was deprecated to
+   *             discourage its use.
    */
+  @Deprecated
   protected static AccumuloConfiguration getAccumuloConfiguration(JobContext context) {
     return FileOutputConfigurator.getAccumuloConfiguration(CLASS, context.getConfiguration());
   }
@@ -139,7 +142,7 @@ public class AccumuloFileOutputFormat extends FileOutputFormat<Key,Value> {
   public RecordWriter<Key,Value> getRecordWriter(TaskAttemptContext context) throws IOException {
     // get the path of the temporary output file
     final Configuration conf = context.getConfiguration();
-    final AccumuloConfiguration acuConf = getAccumuloConfiguration(context);
+    final AccumuloConfiguration acuConf = FileOutputConfigurator.getAccumuloConfiguration(CLASS, context.getConfiguration());
 
     final String extension = acuConf.get(Property.TABLE_FILE_TYPE);
     final Path file = this.getDefaultWorkFile(context, "." + extension);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/c6252914/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloOutputFormat.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloOutputFormat.java b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloOutputFormat.java
index 29257d9..76fe523 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloOutputFormat.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloOutputFormat.java
@@ -35,6 +35,9 @@ import org.apache.accumulo.core.client.TableExistsException;
 import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.client.ZooKeeperInstance;
 import org.apache.accumulo.core.client.admin.DelegationTokenConfig;
+import org.apache.accumulo.core.client.admin.SecurityOperations;
+import org.apache.accumulo.core.client.impl.AuthenticationTokenIdentifier;
+import org.apache.accumulo.core.client.impl.DelegationTokenImpl;
 import org.apache.accumulo.core.client.mapreduce.lib.impl.ConfiguratorBase;
 import org.apache.accumulo.core.client.mapreduce.lib.impl.OutputConfigurator;
 import org.apache.accumulo.core.client.mock.MockInstance;
@@ -43,10 +46,10 @@ import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
 import org.apache.accumulo.core.client.security.tokens.AuthenticationToken.AuthenticationTokenSerializer;
 import org.apache.accumulo.core.client.security.tokens.DelegationToken;
 import org.apache.accumulo.core.client.security.tokens.KerberosToken;
+import org.apache.accumulo.core.client.security.tokens.PasswordToken;
 import org.apache.accumulo.core.data.ColumnUpdate;
-import org.apache.accumulo.core.data.KeyExtent;
 import org.apache.accumulo.core.data.Mutation;
-import org.apache.accumulo.core.security.AuthenticationTokenIdentifier;
+import org.apache.accumulo.core.data.TabletID;
 import org.apache.accumulo.core.security.ColumnVisibility;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.mapreduce.Job;
@@ -83,8 +86,10 @@ public class AccumuloOutputFormat extends OutputFormat<Text,Mutation> {
    * Sets the connector information needed to communicate with Accumulo in this job.
    *
    * <p>
-   * <b>WARNING:</b> The serialized token is stored in the configuration and shared with all MapReduce tasks. It is BASE64 encoded to provide a charset safe
-   * conversion to a string, and is not intended to be secure.
+   * <b>WARNING:</b> Some tokens, when serialized, divulge sensitive information in the configuration as a means to pass the token to MapReduce tasks. This
+   * information is BASE64 encoded to provide a charset safe conversion to a string, but this conversion is not intended to be secure. {@link PasswordToken} is
+   * one example that is insecure in this way; however {@link DelegationToken}s, acquired using
+   * {@link SecurityOperations#getDelegationToken(DelegationTokenConfig)}, is not subject to this concern.
    *
    * @param job
    *          the Hadoop job instance to be configured
@@ -106,8 +111,8 @@ public class AccumuloOutputFormat extends OutputFormat<Text,Mutation> {
       }
     }
     // DelegationTokens can be passed securely from user to task without serializing insecurely in the configuration
-    if (token instanceof DelegationToken) {
-      DelegationToken delegationToken = (DelegationToken) token;
+    if (token instanceof DelegationTokenImpl) {
+      DelegationTokenImpl delegationToken = (DelegationTokenImpl) token;
 
       // Convert it into a Hadoop Token
       AuthenticationTokenIdentifier identifier = delegationToken.getIdentifier();
@@ -537,9 +542,9 @@ public class AccumuloOutputFormat extends OutputFormat<Text,Mutation> {
       try {
         mtbw.close();
       } catch (MutationsRejectedException e) {
-        if (e.getAuthorizationFailuresMap().size() >= 0) {
+        if (e.getSecurityErrorCodes().size() >= 0) {
           HashMap<String,Set<SecurityErrorCode>> tables = new HashMap<String,Set<SecurityErrorCode>>();
-          for (Entry<KeyExtent,Set<SecurityErrorCode>> ke : e.getAuthorizationFailuresMap().entrySet()) {
+          for (Entry<TabletID,Set<SecurityErrorCode>> ke : e.getSecurityErrorCodes().entrySet()) {
             Set<SecurityErrorCode> secCodes = tables.get(ke.getKey().getTableId().toString());
             if (secCodes == null) {
               secCodes = new HashSet<SecurityErrorCode>();

http://git-wip-us.apache.org/repos/asf/accumulo/blob/c6252914/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/impl/ConfiguratorBase.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/impl/ConfiguratorBase.java b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/impl/ConfiguratorBase.java
index b32d1b3..5364e4a 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/impl/ConfiguratorBase.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/impl/ConfiguratorBase.java
@@ -31,13 +31,13 @@ import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.ClientConfiguration;
 import org.apache.accumulo.core.client.Instance;
 import org.apache.accumulo.core.client.ZooKeeperInstance;
+import org.apache.accumulo.core.client.impl.AuthenticationTokenIdentifier;
+import org.apache.accumulo.core.client.impl.Credentials;
+import org.apache.accumulo.core.client.impl.DelegationTokenImpl;
 import org.apache.accumulo.core.client.mapreduce.impl.DelegationTokenStub;
 import org.apache.accumulo.core.client.mock.MockInstance;
 import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
 import org.apache.accumulo.core.client.security.tokens.AuthenticationToken.AuthenticationTokenSerializer;
-import org.apache.accumulo.core.client.security.tokens.DelegationToken;
-import org.apache.accumulo.core.security.AuthenticationTokenIdentifier;
-import org.apache.accumulo.core.security.Credentials;
 import org.apache.accumulo.core.util.Base64;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataInputStream;
@@ -149,9 +149,9 @@ public class ConfiguratorBase {
     checkArgument(token != null, "token is null");
     conf.setBoolean(enumToConfKey(implementingClass, ConnectorInfo.IS_CONFIGURED), true);
     conf.set(enumToConfKey(implementingClass, ConnectorInfo.PRINCIPAL), principal);
-    if (token instanceof DelegationToken) {
+    if (token instanceof DelegationTokenImpl) {
       // Avoid serializing the DelegationToken secret in the configuration -- the Job will do that work for us securely
-      DelegationToken delToken = (DelegationToken) token;
+      DelegationTokenImpl delToken = (DelegationTokenImpl) token;
       conf.set(enumToConfKey(implementingClass, ConnectorInfo.TOKEN), TokenSource.JOB.prefix() + token.getClass().getName() + ":"
           + delToken.getServiceName().toString());
     } else {
@@ -252,7 +252,7 @@ public class ConfiguratorBase {
       String[] args = token.substring(TokenSource.JOB.prefix().length()).split(":", 2);
       if (args.length == 2) {
         String className = args[0], serviceName = args[1];
-        if (DelegationToken.class.getName().equals(className)) {
+        if (DelegationTokenImpl.class.getName().equals(className)) {
           return new DelegationTokenStub(serviceName);
         }
       }
@@ -466,7 +466,7 @@ public class ConfiguratorBase {
       AuthenticationTokenIdentifier identifier = new AuthenticationTokenIdentifier();
       try {
         identifier.readFields(new DataInputStream(new ByteArrayInputStream(hadoopToken.getIdentifier())));
-        return new DelegationToken(hadoopToken.getPassword(), identifier);
+        return new DelegationTokenImpl(hadoopToken.getPassword(), identifier);
       } catch (IOException e) {
         throw new RuntimeException("Could not construct DelegationToken from JobConf Credentials", e);
       }
@@ -491,7 +491,7 @@ public class ConfiguratorBase {
       AuthenticationTokenIdentifier identifier = new AuthenticationTokenIdentifier();
       try {
         identifier.readFields(new DataInputStream(new ByteArrayInputStream(hadoopToken.getIdentifier())));
-        return new DelegationToken(hadoopToken.getPassword(), identifier);
+        return new DelegationTokenImpl(hadoopToken.getPassword(), identifier);
       } catch (IOException e) {
         throw new RuntimeException("Could not construct DelegationToken from JobConf Credentials", e);
       }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/c6252914/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/impl/InputConfigurator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/impl/InputConfigurator.java b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/impl/InputConfigurator.java
index 238539b..0e640b4 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/impl/InputConfigurator.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/impl/InputConfigurator.java
@@ -47,23 +47,23 @@ import org.apache.accumulo.core.client.RowIterator;
 import org.apache.accumulo.core.client.Scanner;
 import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.client.impl.ClientContext;
+import org.apache.accumulo.core.client.impl.Credentials;
+import org.apache.accumulo.core.client.impl.DelegationTokenImpl;
 import org.apache.accumulo.core.client.impl.Tables;
 import org.apache.accumulo.core.client.impl.TabletLocator;
 import org.apache.accumulo.core.client.mapreduce.InputTableConfig;
 import org.apache.accumulo.core.client.mock.impl.MockTabletLocator;
 import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
-import org.apache.accumulo.core.client.security.tokens.DelegationToken;
 import org.apache.accumulo.core.data.Key;
-import org.apache.accumulo.core.data.KeyExtent;
 import org.apache.accumulo.core.data.PartialKey;
 import org.apache.accumulo.core.data.Range;
 import org.apache.accumulo.core.data.Value;
+import org.apache.accumulo.core.data.impl.KeyExtent;
 import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
 import org.apache.accumulo.core.master.state.tables.TableState;
 import org.apache.accumulo.core.metadata.MetadataTable;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema;
 import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.core.security.Credentials;
 import org.apache.accumulo.core.security.TablePermission;
 import org.apache.accumulo.core.util.Base64;
 import org.apache.accumulo.core.util.Pair;
@@ -718,7 +718,7 @@ public class InputConfigurator extends ConfiguratorBase {
    *
    * <p>
    * The implementation (JobContext or JobConf which created the Configuration) needs to be used to extract the proper {@link AuthenticationToken} for
-   * {@link DelegationToken} support.
+   * {@link DelegationTokenImpl} support.
    *
    * @param implementingClass
    *          the class whose name will be used as a prefix for the property configuration key

http://git-wip-us.apache.org/repos/asf/accumulo/blob/c6252914/core/src/main/java/org/apache/accumulo/core/client/mock/MockConnector.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mock/MockConnector.java b/core/src/main/java/org/apache/accumulo/core/client/mock/MockConnector.java
index 4d32093..d348400 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mock/MockConnector.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mock/MockConnector.java
@@ -35,10 +35,10 @@ 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.Credentials;
 import org.apache.accumulo.core.client.impl.thrift.SecurityErrorCode;
 import org.apache.accumulo.core.client.security.tokens.NullToken;
 import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.core.security.Credentials;
 
 public class MockConnector extends Connector {
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/c6252914/core/src/main/java/org/apache/accumulo/core/client/mock/MockInstance.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mock/MockInstance.java b/core/src/main/java/org/apache/accumulo/core/client/mock/MockInstance.java
index 01dc91b..57cd5ee 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mock/MockInstance.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mock/MockInstance.java
@@ -27,12 +27,12 @@ import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.Connector;
 import org.apache.accumulo.core.client.Instance;
+import org.apache.accumulo.core.client.impl.Credentials;
 import org.apache.accumulo.core.client.impl.thrift.SecurityErrorCode;
 import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
 import org.apache.accumulo.core.client.security.tokens.PasswordToken;
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
 import org.apache.accumulo.core.conf.DefaultConfiguration;
-import org.apache.accumulo.core.security.Credentials;
 import org.apache.accumulo.core.util.ByteBufferUtil;
 import org.apache.accumulo.core.util.CachedConfiguration;
 import org.apache.accumulo.core.util.TextUtil;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/c6252914/core/src/main/java/org/apache/accumulo/core/client/mock/impl/MockTabletLocator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mock/impl/MockTabletLocator.java b/core/src/main/java/org/apache/accumulo/core/client/mock/impl/MockTabletLocator.java
index 8927495..1c0c6a9 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mock/impl/MockTabletLocator.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mock/impl/MockTabletLocator.java
@@ -27,9 +27,9 @@ import org.apache.accumulo.core.client.Instance;
 import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.client.impl.ClientContext;
 import org.apache.accumulo.core.client.impl.TabletLocator;
-import org.apache.accumulo.core.data.KeyExtent;
 import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.data.Range;
+import org.apache.accumulo.core.data.impl.KeyExtent;
 import org.apache.hadoop.io.Text;
 
 public class MockTabletLocator extends TabletLocator {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/c6252914/core/src/main/java/org/apache/accumulo/core/client/security/tokens/DelegationToken.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/security/tokens/DelegationToken.java b/core/src/main/java/org/apache/accumulo/core/client/security/tokens/DelegationToken.java
index 1c52bcf..8256d66 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/security/tokens/DelegationToken.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/security/tokens/DelegationToken.java
@@ -14,138 +14,22 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.accumulo.core.client.security.tokens;
-
-import static com.google.common.base.Preconditions.checkNotNull;
 
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.util.Collections;
-import java.util.Set;
+package org.apache.accumulo.core.client.security.tokens;
 
-import org.apache.accumulo.core.client.Instance;
-import org.apache.accumulo.core.security.AuthenticationTokenIdentifier;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.security.Credentials;
-import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hadoop.security.token.Token;
-import org.apache.hadoop.security.token.TokenIdentifier;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
+import org.apache.accumulo.core.client.admin.SecurityOperations;
 
 /**
- * An {@link AuthenticationToken} that wraps a "Hadoop style" delegation token created by Accumulo. The only intended scope of this implementation is when a
- * KerberosToken cannot be used instead. The most common reason for this is within YARN jobs. The Kerberos credentials of the user are not passed over the wire
- * to the job itself. The delegation token serves as a mechanism to obtain a shared secret with Accumulo using a {@link KerberosToken} and then run some task
- * authenticating with that shared secret, this {@link DelegationToken}.
+ * An {@link AuthenticationToken} that wraps a "Hadoop style" delegation token created by Accumulo. The intended scope of this token is when a KerberosToken
+ * cannot be used instead. The most common reason for this is within YARN jobs. The Kerberos credentials of the user are not passed over the wire to the job
+ * itself. The delegation token serves as a mechanism to obtain a transient shared secret with Accumulo using a {@link KerberosToken} and then run some task
+ * authenticating with that shared secret.
+ *
+ * <p>Obtain a delegation token by calling {@link SecurityOperations#getDelegationToken(org.apache.accumulo.core.client.admin.DelegationTokenConfig)}
  *
  * @since 1.7.0
  */
-public class DelegationToken extends PasswordToken {
-  private static final Logger log = LoggerFactory.getLogger(DelegationToken.class);
-
-  public static final String SERVICE_NAME = "AccumuloDelegationToken";
-
-  private AuthenticationTokenIdentifier identifier;
-
-  public DelegationToken() {
-    super();
-  }
-
-  public DelegationToken(byte[] delegationTokenPassword, AuthenticationTokenIdentifier identifier) {
-    checkNotNull(delegationTokenPassword);
-    checkNotNull(identifier);
-    setPassword(delegationTokenPassword);
-    this.identifier = identifier;
-  }
-
-  public DelegationToken(Instance instance, UserGroupInformation user, AuthenticationTokenIdentifier identifier) {
-    checkNotNull(instance);
-    checkNotNull(user);
-    checkNotNull(identifier);
-
-    Credentials creds = user.getCredentials();
-    Token<? extends TokenIdentifier> token = creds.getToken(new Text(SERVICE_NAME + "-" + instance.getInstanceID()));
-    if (null == token) {
-      throw new IllegalArgumentException("Did not find Accumulo delegation token in provided UserGroupInformation");
-    }
-    setPasswordFromToken(token, identifier);
-  }
-
-  public DelegationToken(Token<? extends TokenIdentifier> token, AuthenticationTokenIdentifier identifier) {
-    checkNotNull(token);
-    checkNotNull(identifier);
-    setPasswordFromToken(token, identifier);
-  }
-
-  private void setPasswordFromToken(Token<? extends TokenIdentifier> token, AuthenticationTokenIdentifier identifier) {
-    if (!AuthenticationTokenIdentifier.TOKEN_KIND.equals(token.getKind())) {
-      String msg = "Expected an AuthenticationTokenIdentifier but got a " + token.getKind();
-      log.error(msg);
-      throw new IllegalArgumentException(msg);
-    }
-
-    setPassword(token.getPassword());
-    this.identifier = identifier;
-  }
-
-  /**
-   * The identifier for this token, may be null.
-   */
-  public AuthenticationTokenIdentifier getIdentifier() {
-    return identifier;
-  }
-
-  /**
-   * The service name used to identify the {@link Token}
-   */
-  public Text getServiceName() {
-    checkNotNull(identifier);
-    return new Text(SERVICE_NAME + "-" + identifier.getInstanceId());
-  }
-
-  @Override
-  public void init(Properties properties) {
-    // Encourage use of UserGroupInformation as entry point
-  }
-
-  @Override
-  public Set<TokenProperty> getProperties() {
-    // Encourage use of UserGroupInformation as entry point
-    return Collections.emptySet();
-  }
-
-  @Override
-  public void write(DataOutput out) throws IOException {
-    super.write(out);
-    identifier.write(out);
-  }
-
-  @Override
-  public void readFields(DataInput in) throws IOException {
-    super.readFields(in);
-    identifier = new AuthenticationTokenIdentifier();
-    identifier.readFields(in);
-  }
-
-  @Override
-  public DelegationToken clone() {
-    DelegationToken copy = (DelegationToken) super.clone();
-    copy.setPassword(getPassword());
-    copy.identifier = new AuthenticationTokenIdentifier(identifier);
-    return copy;
-  }
-
-  @Override
-  public int hashCode() {
-    return super.hashCode() ^ identifier.hashCode();
-  }
 
-  @Override
-  public boolean equals(Object obj) {
-    // We assume we can cast obj to DelegationToken because the super.equals(obj) check ensures obj is of the same type as this
-    return super.equals(obj) && identifier.equals(((DelegationToken) obj).identifier);
-  }
+public interface DelegationToken extends AuthenticationToken {
 
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/c6252914/core/src/main/java/org/apache/accumulo/core/constraints/Constraint.java
----------------------------------------------------------------------
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 e95088a..78ec0ac 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
@@ -18,8 +18,8 @@ package org.apache.accumulo.core.constraints;
 
 import java.util.List;
 
-import org.apache.accumulo.core.data.KeyExtent;
 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;
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/c6252914/core/src/main/java/org/apache/accumulo/core/constraints/VisibilityConstraint.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/constraints/VisibilityConstraint.java b/core/src/main/java/org/apache/accumulo/core/constraints/VisibilityConstraint.java
new file mode 100644
index 0000000..91bc22f
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/constraints/VisibilityConstraint.java
@@ -0,0 +1,93 @@
+/*
+ * 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.constraints;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+
+import org.apache.accumulo.core.data.ColumnUpdate;
+import org.apache.accumulo.core.data.Mutation;
+import org.apache.accumulo.core.security.ColumnVisibility;
+import org.apache.accumulo.core.security.VisibilityEvaluator;
+import org.apache.accumulo.core.security.VisibilityParseException;
+import org.apache.accumulo.core.util.BadArgumentException;
+
+/**
+ * A constraint that checks the visibility of columns against the actor's authorizations. Violation codes:
+ * <p>
+ * <ul>
+ * <li>1 = failure to parse visibility expression</li>
+ * <li>2 = insufficient authorization</li>
+ * </ul>
+ */
+public class VisibilityConstraint implements Constraint {
+
+  @Override
+  public String getViolationDescription(short violationCode) {
+    switch (violationCode) {
+      case 1:
+        return "Malformed column visibility";
+      case 2:
+        return "User does not have authorization on column visibility";
+    }
+
+    return null;
+  }
+
+  @Override
+  public List<Short> check(Environment env, Mutation mutation) {
+    List<ColumnUpdate> updates = mutation.getUpdates();
+
+    HashSet<String> ok = null;
+    if (updates.size() > 1)
+      ok = new HashSet<String>();
+
+    VisibilityEvaluator ve = null;
+
+    for (ColumnUpdate update : updates) {
+
+      byte[] cv = update.getColumnVisibility();
+      if (cv.length > 0) {
+        String key = null;
+        if (ok != null && ok.contains(key = new String(cv, UTF_8)))
+          continue;
+
+        try {
+
+          if (ve == null)
+            ve = new VisibilityEvaluator(env.getAuthorizationsContainer());
+
+          if (!ve.evaluate(new ColumnVisibility(cv)))
+            return Collections.singletonList(Short.valueOf((short) 2));
+
+        } catch (BadArgumentException bae) {
+          return Collections.singletonList(new Short((short) 1));
+        } catch (VisibilityParseException e) {
+          return Collections.singletonList(new Short((short) 1));
+        }
+
+        if (ok != null)
+          ok.add(key);
+      }
+    }
+
+    return null;
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/c6252914/core/src/main/java/org/apache/accumulo/core/data/ComparableBytes.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/data/ComparableBytes.java b/core/src/main/java/org/apache/accumulo/core/data/ComparableBytes.java
index 11aaf2b..78c0e56 100644
--- a/core/src/main/java/org/apache/accumulo/core/data/ComparableBytes.java
+++ b/core/src/main/java/org/apache/accumulo/core/data/ComparableBytes.java
@@ -20,7 +20,12 @@ import org.apache.hadoop.io.BinaryComparable;
 
 /**
  * An array of bytes wrapped so as to extend Hadoop's <code>BinaryComparable</code> class.
+ *
+ * @deprecated since 1.7.0 In an attempt to clean up types in the data package that were not intended to be in public API this type was deprecated. Technically
+ *             this method was not considered part of the public API in 1.6.0 and earlier, therefore it could have been deleted. However a decision was made to
+ *             deprecate in order to be cautious and avoid confusion between 1.6.0 and 1.7.0.
  */
+@Deprecated
 public class ComparableBytes extends BinaryComparable {
 
   public byte[] data;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/c6252914/core/src/main/java/org/apache/accumulo/core/data/KeyExtent.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/data/KeyExtent.java b/core/src/main/java/org/apache/accumulo/core/data/KeyExtent.java
index fe7f776..f730cf6 100644
--- a/core/src/main/java/org/apache/accumulo/core/data/KeyExtent.java
+++ b/core/src/main/java/org/apache/accumulo/core/data/KeyExtent.java
@@ -16,750 +16,243 @@
  */
 package org.apache.accumulo.core.data;
 
-/**
- * keeps track of information needed to identify a tablet
- * apparently, we only need the endKey and not the start as well
- *
- */
-
-import java.io.ByteArrayOutputStream;
 import java.io.DataInput;
 import java.io.DataOutput;
-import java.io.DataOutputStream;
 import java.io.IOException;
-import java.lang.ref.WeakReference;
-import java.util.ArrayList;
 import java.util.Collection;
-import java.util.Collections;
 import java.util.Map.Entry;
 import java.util.Set;
 import java.util.SortedMap;
 import java.util.SortedSet;
+import java.util.TreeMap;
 import java.util.TreeSet;
 import java.util.UUID;
-import java.util.WeakHashMap;
 
 import org.apache.accumulo.core.data.thrift.TKeyExtent;
-import org.apache.accumulo.core.metadata.MetadataTable;
-import org.apache.accumulo.core.metadata.RootTable;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema;
-import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection;
-import org.apache.accumulo.core.util.ByteBufferUtil;
-import org.apache.accumulo.core.util.TextUtil;
 import org.apache.hadoop.io.BinaryComparable;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.WritableComparable;
 
+/**
+ * keeps track of information needed to identify a tablet
+ *
+ * @deprecated since 1.7.0 use {@link TabletID}
+ */
+@Deprecated
 public class KeyExtent implements WritableComparable<KeyExtent> {
 
-  private static final WeakHashMap<Text,WeakReference<Text>> tableIds = new WeakHashMap<Text,WeakReference<Text>>();
-
-  private static Text dedupeTableId(Text tableId) {
-    synchronized (tableIds) {
-      WeakReference<Text> etir = tableIds.get(tableId);
-      if (etir != null) {
-        Text eti = etir.get();
-        if (eti != null) {
-          return eti;
-        }
-      }
-
-      tableId = new Text(tableId);
-      tableIds.put(tableId, new WeakReference<Text>(tableId));
-      return tableId;
-    }
-  }
-
-  private Text textTableId;
-  private Text textEndRow;
-  private Text textPrevEndRow;
-
-  private void check() {
-
-    if (getTableId() == null)
-      throw new IllegalArgumentException("null table id not allowed");
+  // Wrapping impl.KeyExtent to resuse code. Did not want to extend impl.KeyExtent because any changes to impl.KeyExtent would be reflected in this class.
+  // Wrapping impl.KeyExtent allows the API of this deprecated class to be frozen.
+  private org.apache.accumulo.core.data.impl.KeyExtent wrapped;
 
-    if (getEndRow() == null || getPrevEndRow() == null)
-      return;
-
-    if (getPrevEndRow().compareTo(getEndRow()) >= 0) {
-      throw new IllegalArgumentException("prevEndRow (" + getPrevEndRow() + ") >= endRow (" + getEndRow() + ")");
-    }
-  }
-
-  /**
-   * Default constructor
-   *
-   */
   public KeyExtent() {
-    this.setTableId(new Text());
-    this.setEndRow(new Text(), false, false);
-    this.setPrevEndRow(new Text(), false, false);
+    this.wrapped = new org.apache.accumulo.core.data.impl.KeyExtent();
   }
 
   public KeyExtent(Text table, Text endRow, Text prevEndRow) {
-    this.setTableId(table);
-    this.setEndRow(endRow, false, true);
-    this.setPrevEndRow(prevEndRow, false, true);
-
-    check();
+    this.wrapped = new org.apache.accumulo.core.data.impl.KeyExtent(table, endRow, prevEndRow);
   }
 
   public KeyExtent(KeyExtent extent) {
-    // extent has already deduped table id, so there is no need to do it again
-    this.textTableId = extent.textTableId;
-    this.setEndRow(extent.getEndRow(), false, true);
-    this.setPrevEndRow(extent.getPrevEndRow(), false, true);
-
-    check();
+    this.wrapped = new org.apache.accumulo.core.data.impl.KeyExtent(extent.getTableId(), extent.getEndRow(), extent.getPrevEndRow());
   }
 
   public KeyExtent(TKeyExtent tke) {
-    this.setTableId(new Text(ByteBufferUtil.toBytes(tke.table)));
-    this.setEndRow(tke.endRow == null ? null : new Text(ByteBufferUtil.toBytes(tke.endRow)), false, false);
-    this.setPrevEndRow(tke.prevEndRow == null ? null : new Text(ByteBufferUtil.toBytes(tke.prevEndRow)), false, false);
-
-    check();
-  }
-
-  /**
-   * Returns a String representing this extent's entry in the Metadata table
-   *
-   */
-  public Text getMetadataEntry() {
-    return getMetadataEntry(getTableId(), getEndRow());
-  }
-
-  public static Text getMetadataEntry(Text tableId, Text endRow) {
-    return MetadataSchema.TabletsSection.getRow(tableId, endRow);
+    this.wrapped = new org.apache.accumulo.core.data.impl.KeyExtent(tke);
   }
 
   // constructor for loading extents from metadata rows
   public KeyExtent(Text flattenedExtent, Value prevEndRow) {
-    decodeMetadataRow(flattenedExtent);
-
-    // decode the prev row
-    this.setPrevEndRow(decodePrevEndRow(prevEndRow), false, true);
-
-    check();
+    this.wrapped = new org.apache.accumulo.core.data.impl.KeyExtent(flattenedExtent, prevEndRow);
   }
 
   // recreates an encoded extent from a string representation
   // this encoding is what is stored as the row id of the metadata table
   public KeyExtent(Text flattenedExtent, Text prevEndRow) {
+    this.wrapped = new org.apache.accumulo.core.data.impl.KeyExtent(flattenedExtent, prevEndRow);
+  }
 
-    decodeMetadataRow(flattenedExtent);
-
-    this.setPrevEndRow(null, false, false);
-    if (prevEndRow != null)
-      this.setPrevEndRow(prevEndRow, false, true);
-
-    check();
+  public Text getMetadataEntry() {
+    return wrapped.getMetadataEntry();
   }
 
-  /**
-   * Sets the extents table id
-   *
-   */
   public void setTableId(Text tId) {
-
-    if (tId == null)
-      throw new IllegalArgumentException("null table name not allowed");
-
-    this.textTableId = dedupeTableId(tId);
-
-    hashCode = 0;
+    wrapped.setTableId(tId);
   }
 
-  /**
-   * Returns the extent's table id
-   *
-   */
   public Text getTableId() {
-    return textTableId;
+    return wrapped.getTableId();
   }
 
-  private void setEndRow(Text endRow, boolean check, boolean copy) {
-    if (endRow != null)
-      if (copy)
-        this.textEndRow = new Text(endRow);
-      else
-        this.textEndRow = endRow;
-    else
-      this.textEndRow = null;
-
-    hashCode = 0;
-    if (check)
-      check();
-  }
-
-  /**
-   * Sets this extent's end row
-   *
-   */
   public void setEndRow(Text endRow) {
-    setEndRow(endRow, true, true);
+    wrapped.setEndRow(endRow);
   }
 
-  /**
-   * Returns this extent's end row
-   *
-   */
   public Text getEndRow() {
-    return textEndRow;
+    return wrapped.getEndRow();
   }
 
-  /**
-   * Return the previous extent's end row
-   *
-   */
   public Text getPrevEndRow() {
-    return textPrevEndRow;
-  }
-
-  private void setPrevEndRow(Text prevEndRow, boolean check, boolean copy) {
-    if (prevEndRow != null)
-      if (copy)
-        this.textPrevEndRow = new Text(prevEndRow);
-      else
-        this.textPrevEndRow = prevEndRow;
-    else
-      this.textPrevEndRow = null;
-
-    hashCode = 0;
-    if (check)
-      check();
+    return wrapped.getPrevEndRow();
   }
 
-  /**
-   * Sets the previous extent's end row
-   *
-   */
   public void setPrevEndRow(Text prevEndRow) {
-    setPrevEndRow(prevEndRow, true, true);
+    wrapped.setPrevEndRow(prevEndRow);
   }
 
   @Override
   public void readFields(DataInput in) throws IOException {
-    Text tid = new Text();
-    tid.readFields(in);
-    setTableId(tid);
-    boolean hasRow = in.readBoolean();
-    if (hasRow) {
-      Text er = new Text();
-      er.readFields(in);
-      setEndRow(er, false, false);
-    } else {
-      setEndRow(null, false, false);
-    }
-    boolean hasPrevRow = in.readBoolean();
-    if (hasPrevRow) {
-      Text per = new Text();
-      per.readFields(in);
-      setPrevEndRow(per, false, true);
-    } else {
-      setPrevEndRow((Text) null);
-    }
-
-    hashCode = 0;
-    check();
+    wrapped.readFields(in);
   }
 
   @Override
   public void write(DataOutput out) throws IOException {
-    getTableId().write(out);
-    if (getEndRow() != null) {
-      out.writeBoolean(true);
-      getEndRow().write(out);
-    } else {
-      out.writeBoolean(false);
-    }
-    if (getPrevEndRow() != null) {
-      out.writeBoolean(true);
-      getPrevEndRow().write(out);
-    } else {
-      out.writeBoolean(false);
-    }
+    wrapped.write(out);
   }
 
-  /**
-   * Returns a String representing the previous extent's entry in the Metadata table
-   *
-   */
   public Mutation getPrevRowUpdateMutation() {
-    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<KeyExtent>();
-    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;
-
-    if (ibw.get()[0] != 0) {
-      per = new Text();
-      per.set(ibw.get(), 1, ibw.get().length - 1);
-    }
-
-    return per;
-  }
-
-  public static Value encodePrevEndRow(Text per) {
-    if (per == null)
-      return new Value(new byte[] {0});
-    byte[] b = new byte[per.getLength() + 1];
-    b[0] = 1;
-    System.arraycopy(per.getBytes(), 0, b, 1, per.getLength());
-    return new Value(b);
-  }
-
-  public static Mutation getPrevRowUpdateMutation(KeyExtent ke) {
-    Mutation m = new Mutation(ke.getMetadataEntry());
-    TabletsSection.TabletColumnFamily.PREV_ROW_COLUMN.put(m, encodePrevEndRow(ke.getPrevEndRow()));
-    return m;
+    return wrapped.getPrevRowUpdateMutation();
   }
 
   @Override
   public int compareTo(KeyExtent other) {
-
-    int result = getTableId().compareTo(other.getTableId());
-    if (result != 0)
-      return result;
-
-    if (this.getEndRow() == null) {
-      if (other.getEndRow() != null)
-        return 1;
-    } else {
-      if (other.getEndRow() == null)
-        return -1;
-
-      result = getEndRow().compareTo(other.getEndRow());
-      if (result != 0)
-        return result;
-    }
-    if (this.getPrevEndRow() == null) {
-      if (other.getPrevEndRow() == null)
-        return 0;
-      return -1;
-    }
-    if (other.getPrevEndRow() == null)
-      return 1;
-    return this.getPrevEndRow().compareTo(other.getPrevEndRow());
+    return wrapped.compareTo(other.wrapped);
   }
 
-  private int hashCode = 0;
-
   @Override
   public int hashCode() {
-    if (hashCode != 0)
-      return hashCode;
-
-    int prevEndRowHash = 0;
-    int endRowHash = 0;
-    if (this.getEndRow() != null) {
-      endRowHash = this.getEndRow().hashCode();
-    }
-
-    if (this.getPrevEndRow() != null) {
-      prevEndRowHash = this.getPrevEndRow().hashCode();
-    }
-
-    hashCode = getTableId().hashCode() + endRowHash + prevEndRowHash;
-    return hashCode;
-  }
-
-  private boolean equals(Text t1, Text t2) {
-    if (t1 == null || t2 == null)
-      return t1 == t2;
-
-    return t1.equals(t2);
+    return wrapped.hashCode();
   }
 
   @Override
   public boolean equals(Object o) {
-    if (o == this)
-      return true;
-    if (!(o instanceof KeyExtent))
-      return false;
-    KeyExtent oke = (KeyExtent) o;
-    return textTableId.equals(oke.textTableId) && equals(textEndRow, oke.textEndRow) && equals(textPrevEndRow, oke.textPrevEndRow);
+    if(o instanceof KeyExtent){
+      return wrapped.equals(((KeyExtent)o).wrapped);
+    }
+
+    return false;
   }
 
   @Override
   public String toString() {
-    String endRowString;
-    String prevEndRowString;
-    String tableIdString = getTableId().toString().replaceAll(";", "\\\\;").replaceAll("\\\\", "\\\\\\\\");
-
-    if (getEndRow() == null)
-      endRowString = "<";
-    else
-      endRowString = ";" + TextUtil.truncate(getEndRow()).toString().replaceAll(";", "\\\\;").replaceAll("\\\\", "\\\\\\\\");
-
-    if (getPrevEndRow() == null)
-      prevEndRowString = "<";
-    else
-      prevEndRowString = ";" + TextUtil.truncate(getPrevEndRow()).toString().replaceAll(";", "\\\\;").replaceAll("\\\\", "\\\\\\\\");
-
-    return tableIdString + endRowString + prevEndRowString;
+    return wrapped.toString();
   }
 
   public UUID getUUID() {
-    try {
-
-      ByteArrayOutputStream baos = new ByteArrayOutputStream();
-      DataOutputStream dos = new DataOutputStream(baos);
-
-      // to get a unique hash it is important to encode the data
-      // like it is being serialized
-
-      this.write(dos);
-
-      dos.close();
-
-      return UUID.nameUUIDFromBytes(baos.toByteArray());
-
-    } catch (IOException e) {
-      // should not happen since we are writing to memory
-      throw new RuntimeException(e);
-    }
-  }
-
-  // note: this is only the encoding of the table id and the last row, not the prev row
-  /**
-   * Populates the extent's fields based on a flatted extent
-   *
-   */
-  private void decodeMetadataRow(Text flattenedExtent) {
-    int semiPos = -1;
-    int ltPos = -1;
-
-    for (int i = 0; i < flattenedExtent.getLength(); i++) {
-      if (flattenedExtent.getBytes()[i] == ';' && semiPos < 0) {
-        // want the position of the first semicolon
-        semiPos = i;
-      }
-
-      if (flattenedExtent.getBytes()[i] == '<') {
-        ltPos = i;
-      }
-    }
-
-    if (semiPos < 0 && ltPos < 0) {
-      throw new IllegalArgumentException("Metadata row does not contain ; or <  " + flattenedExtent);
-    }
-
-    if (semiPos < 0) {
-
-      if (ltPos != flattenedExtent.getLength() - 1) {
-        throw new IllegalArgumentException("< must come at end of Metadata row  " + flattenedExtent);
-      }
-
-      Text tableId = new Text();
-      tableId.set(flattenedExtent.getBytes(), 0, flattenedExtent.getLength() - 1);
-      this.setTableId(tableId);
-      this.setEndRow(null, false, false);
-    } else {
-
-      Text tableId = new Text();
-      tableId.set(flattenedExtent.getBytes(), 0, semiPos);
-
-      Text endRow = new Text();
-      endRow.set(flattenedExtent.getBytes(), semiPos + 1, flattenedExtent.getLength() - (semiPos + 1));
-
-      this.setTableId(tableId);
-
-      this.setEndRow(endRow, false, false);
-    }
-  }
-
-  public static byte[] tableOfMetadataRow(Text row) {
-    KeyExtent ke = new KeyExtent();
-    ke.decodeMetadataRow(row);
-    return TextUtil.getBytes(ke.getTableId());
+    return wrapped.getUUID();
   }
 
-  public boolean contains(final ByteSequence bsrow) {
-    if (bsrow == null) {
-      throw new IllegalArgumentException("Passing null to contains is ambiguous, could be in first or last extent of table");
-    }
-
-    BinaryComparable row = new BinaryComparable() {
-
-      @Override
-      public int getLength() {
-        return bsrow.length();
-      }
-
-      @Override
-      public byte[] getBytes() {
-        if (bsrow.isBackedByArray() && bsrow.offset() == 0)
-          return bsrow.getBackingArray();
-
-        return bsrow.toArray();
-      }
-    };
-
-    if ((this.getPrevEndRow() == null || this.getPrevEndRow().compareTo(row) < 0) && (this.getEndRow() == null || this.getEndRow().compareTo(row) >= 0)) {
-      return true;
-    }
-    return false;
+  public boolean contains(ByteSequence bsrow) {
+    return wrapped.contains(bsrow);
   }
 
   public boolean contains(BinaryComparable row) {
-    if (row == null) {
-      throw new IllegalArgumentException("Passing null to contains is ambiguous, could be in first or last extent of table");
-    }
-
-    if ((this.getPrevEndRow() == null || this.getPrevEndRow().compareTo(row) < 0) && (this.getEndRow() == null || this.getEndRow().compareTo(row) >= 0)) {
-      return true;
-    }
-    return false;
+    return wrapped.contains(row);
   }
 
   public Range toDataRange() {
-    return new Range(getPrevEndRow(), false, getEndRow(), true);
+    return wrapped.toDataRange();
   }
 
   public Range toMetadataRange() {
-    Text metadataPrevRow = new Text(getTableId());
-    metadataPrevRow.append(new byte[] {';'}, 0, 1);
-    if (getPrevEndRow() != null) {
-      metadataPrevRow.append(getPrevEndRow().getBytes(), 0, getPrevEndRow().getLength());
-    }
-
-    Range range = new Range(metadataPrevRow, getPrevEndRow() == null, getMetadataEntry(), true);
-    return range;
+    return wrapped.toMetadataRange();
   }
 
-  public static SortedSet<KeyExtent> findChildren(KeyExtent ke, SortedSet<KeyExtent> tablets) {
-
-    SortedSet<KeyExtent> children = null;
-
-    for (KeyExtent tabletKe : tablets) {
-
-      if (ke.getPrevEndRow() == tabletKe.getPrevEndRow() || ke.getPrevEndRow() != null && tabletKe.getPrevEndRow() != null
-          && tabletKe.getPrevEndRow().compareTo(ke.getPrevEndRow()) == 0) {
-        children = new TreeSet<KeyExtent>();
-      }
-
-      if (children != null) {
-        children.add(tabletKe);
-      }
-
-      if (ke.getEndRow() == tabletKe.getEndRow() || ke.getEndRow() != null && tabletKe.getEndRow() != null
-          && tabletKe.getEndRow().compareTo(ke.getEndRow()) == 0) {
-        return children;
-      }
-    }
-
-    return new TreeSet<KeyExtent>();
+  public boolean overlaps(KeyExtent other) {
+    return wrapped.overlaps(other.wrapped);
   }
 
-  public static KeyExtent findContainingExtent(KeyExtent extent, SortedSet<KeyExtent> extents) {
-
-    KeyExtent lookupExtent = new KeyExtent(extent);
-    lookupExtent.setPrevEndRow((Text) null);
-
-    SortedSet<KeyExtent> tailSet = extents.tailSet(lookupExtent);
-
-    if (tailSet.isEmpty()) {
-      return null;
-    }
-
-    KeyExtent first = tailSet.first();
-
-    if (first.getTableId().compareTo(extent.getTableId()) != 0) {
-      return null;
-    }
-
-    if (first.getPrevEndRow() == null) {
-      return first;
-    }
-
-    if (extent.getPrevEndRow() == null) {
-      return null;
-    }
+  public TKeyExtent toThrift() {
+    return wrapped.toThrift();
+  }
 
-    if (extent.getPrevEndRow().compareTo(first.getPrevEndRow()) >= 0)
-      return first;
-    return null;
+  public boolean isPreviousExtent(KeyExtent prevExtent) {
+    return wrapped.isPreviousExtent(prevExtent.wrapped);
   }
 
-  private static boolean startsAfter(KeyExtent nke, KeyExtent ke) {
+  public boolean isMeta() {
+    return wrapped.isMeta();
+  }
 
-    int tiCmp = ke.getTableId().compareTo(nke.getTableId());
+  public boolean isRootTablet() {
+    return wrapped.isRootTablet();
+  }
 
-    if (tiCmp > 0) {
-      return true;
+  private static SortedSet<org.apache.accumulo.core.data.impl.KeyExtent> unwrap(Set<KeyExtent> tablets){
+    SortedSet<org.apache.accumulo.core.data.impl.KeyExtent> trans = new TreeSet<>();
+    for (KeyExtent wrapper : tablets) {
+      trans.add(wrapper.wrapped);
     }
 
-    return ke.getPrevEndRow() != null && nke.getEndRow() != null && ke.getPrevEndRow().compareTo(nke.getEndRow()) >= 0;
+    return trans;
   }
 
-  private static Text rowAfterPrevRow(KeyExtent nke) {
-    Text row = new Text(nke.getPrevEndRow());
-    row.append(new byte[] {0}, 0, 1);
-    return row;
+  private static KeyExtent wrap(org.apache.accumulo.core.data.impl.KeyExtent ke){
+    return new KeyExtent(ke.getTableId(), ke.getEndRow(), ke.getPrevEndRow());
   }
 
-  // Some duplication with TabletLocatorImpl
-  public static Set<KeyExtent> findOverlapping(KeyExtent nke, SortedSet<KeyExtent> extents) {
-    if (nke == null || extents == null || extents.isEmpty())
-      return Collections.emptySet();
-
-    SortedSet<KeyExtent> start;
-
-    if (nke.getPrevEndRow() != null) {
-      Text row = rowAfterPrevRow(nke);
-      KeyExtent lookupKey = new KeyExtent(nke.getTableId(), row, null);
-      start = extents.tailSet(lookupKey);
-    } else {
-      KeyExtent lookupKey = new KeyExtent(nke.getTableId(), new Text(), null);
-      start = extents.tailSet(lookupKey);
+  private static SortedSet<KeyExtent> wrap(Collection<org.apache.accumulo.core.data.impl.KeyExtent> unwrapped){
+    SortedSet<KeyExtent> wrapped = new TreeSet<>();
+    for (org.apache.accumulo.core.data.impl.KeyExtent wrappee : unwrapped) {
+      wrapped.add(wrap(wrappee));
     }
 
-    TreeSet<KeyExtent> result = new TreeSet<KeyExtent>();
-    for (KeyExtent ke : start) {
-      if (startsAfter(nke, ke)) {
-        break;
-      }
-      result.add(ke);
-    }
-    return result;
+    return wrapped;
   }
 
-  public boolean overlaps(KeyExtent other) {
-    SortedSet<KeyExtent> set = new TreeSet<KeyExtent>();
-    set.add(other);
-    return !findOverlapping(this, set).isEmpty();
+  public static Text getMetadataEntry(Text tableId, Text endRow) {
+    return MetadataSchema.TabletsSection.getRow(tableId, endRow);
   }
 
-  // Specialization of findOverlapping(KeyExtent, SortedSet<KeyExtent> to work with SortedMap
-  public static Set<KeyExtent> findOverlapping(KeyExtent nke, SortedMap<KeyExtent,?> extents) {
-    if (nke == null || extents == null || extents.isEmpty())
-      return Collections.emptySet();
-
-    SortedMap<KeyExtent,?> start;
-
-    if (nke.getPrevEndRow() != null) {
-      Text row = rowAfterPrevRow(nke);
-      KeyExtent lookupKey = new KeyExtent(nke.getTableId(), row, null);
-      start = extents.tailMap(lookupKey);
-    } else {
-      KeyExtent lookupKey = new KeyExtent(nke.getTableId(), new Text(), null);
-      start = extents.tailMap(lookupKey);
-    }
-
-    TreeSet<KeyExtent> result = new TreeSet<KeyExtent>();
-    for (Entry<KeyExtent,?> entry : start.entrySet()) {
-      KeyExtent ke = entry.getKey();
-      if (startsAfter(nke, ke)) {
-        break;
-      }
-      result.add(ke);
-    }
-    return result;
+  /**
+   * 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) {
+    return wrap(org.apache.accumulo.core.data.impl.KeyExtent.getKeyExtentsForRange(startRow, endRow, unwrap(kes)));
   }
 
-  public static Text getMetadataEntry(KeyExtent extent) {
-    return getMetadataEntry(extent.getTableId(), extent.getEndRow());
+  public static Text decodePrevEndRow(Value ibw) {
+    return org.apache.accumulo.core.data.impl.KeyExtent.decodePrevEndRow(ibw);
   }
 
-  public TKeyExtent toThrift() {
-    return new TKeyExtent(TextUtil.getByteBuffer(textTableId), textEndRow == null ? null : TextUtil.getByteBuffer(textEndRow), textPrevEndRow == null ? null
-        : TextUtil.getByteBuffer(textPrevEndRow));
+  public static Value encodePrevEndRow(Text per) {
+    return org.apache.accumulo.core.data.impl.KeyExtent.encodePrevEndRow(per);
   }
 
-  public boolean isPreviousExtent(KeyExtent prevExtent) {
-    if (prevExtent == null)
-      return getPrevEndRow() == null;
-
-    if (!prevExtent.getTableId().equals(getTableId()))
-      throw new IllegalArgumentException("Cannot compare accross tables " + prevExtent + " " + this);
+  public static Mutation getPrevRowUpdateMutation(KeyExtent ke) {
+   return org.apache.accumulo.core.data.impl.KeyExtent.getPrevRowUpdateMutation(ke.wrapped);
+  }
 
-    if (prevExtent.getEndRow() == null)
-      return false;
+  public static byte[] tableOfMetadataRow(Text row) {
+    return org.apache.accumulo.core.data.impl.KeyExtent.tableOfMetadataRow(row);
+  }
 
-    if (getPrevEndRow() == null)
-      return false;
+  public static SortedSet<KeyExtent> findChildren(KeyExtent ke, SortedSet<KeyExtent> tablets) {
+    return wrap(org.apache.accumulo.core.data.impl.KeyExtent.findChildren(ke.wrapped, unwrap(tablets)));
+  }
 
-    return prevExtent.getEndRow().equals(getPrevEndRow());
+  public static KeyExtent findContainingExtent(KeyExtent extent, SortedSet<KeyExtent> extents) {
+    return wrap(org.apache.accumulo.core.data.impl.KeyExtent.findContainingExtent(extent.wrapped, unwrap(extents)));
   }
 
-  public boolean isMeta() {
-    return getTableId().toString().equals(MetadataTable.ID) || isRootTablet();
+  public static Set<KeyExtent> findOverlapping(KeyExtent nke, SortedSet<KeyExtent> extents) {
+    return wrap(org.apache.accumulo.core.data.impl.KeyExtent.findOverlapping(nke.wrapped, unwrap(extents)));
   }
 
-  public boolean isRootTablet() {
-    return getTableId().toString().equals(RootTable.ID);
+  public static Set<KeyExtent> findOverlapping(KeyExtent nke, SortedMap<KeyExtent,?> extents) {
+    SortedMap<org.apache.accumulo.core.data.impl.KeyExtent,Object> trans = new TreeMap<>();
+    for(Entry<KeyExtent, ?> entry : extents.entrySet()){
+      trans.put(entry.getKey().wrapped, entry.getValue());
+    }
+
+    return wrap(org.apache.accumulo.core.data.impl.KeyExtent.findOverlapping(nke.wrapped, trans));
+  }
+  public static Text getMetadataEntry(KeyExtent extent) {
+    return org.apache.accumulo.core.data.impl.KeyExtent.getMetadataEntry(extent.wrapped);
   }
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/c6252914/core/src/main/java/org/apache/accumulo/core/data/TabletID.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/data/TabletID.java b/core/src/main/java/org/apache/accumulo/core/data/TabletID.java
new file mode 100644
index 0000000..bfcb742
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/data/TabletID.java
@@ -0,0 +1,29 @@
+/*
+ * 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.data;
+
+import org.apache.hadoop.io.Text;
+
+/**
+ * @since 1.7.0
+ */
+public interface TabletID extends Comparable<TabletID> {
+  public Text getTableId();
+  public Text getEndRow();
+  public Text getPrevEndRow();
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/c6252914/core/src/main/java/org/apache/accumulo/core/data/impl/ComparableBytes.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/data/impl/ComparableBytes.java b/core/src/main/java/org/apache/accumulo/core/data/impl/ComparableBytes.java
new file mode 100644
index 0000000..3bf4357
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/data/impl/ComparableBytes.java
@@ -0,0 +1,53 @@
+/*
+ * 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.data.impl;
+
+import org.apache.hadoop.io.BinaryComparable;
+
+/**
+ * An array of bytes wrapped so as to extend Hadoop's <code>BinaryComparable</code> class.
+ */
+public class ComparableBytes extends BinaryComparable {
+
+  public byte[] data;
+
+  /**
+   * Creates a new byte wrapper. The given byte array is used directly as a backing array, so later changes made to the array reflect into the new object.
+   *
+   * @param b
+   *          bytes to wrap
+   */
+  public ComparableBytes(byte[] b) {
+    this.data = b;
+  }
+
+  /**
+   * Gets the wrapped bytes in this object.
+   *
+   * @return bytes
+   */
+  @Override
+  public byte[] getBytes() {
+    return data;
+  }
+
+  @Override
+  public int getLength() {
+    return data.length;
+  }
+
+}