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:45 UTC

[4/9] 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/data/impl/KeyExtent.java
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..d2ae00b
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/data/impl/KeyExtent.java
@@ -0,0 +1,768 @@
+/*
+ * 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 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.TreeSet;
+import java.util.UUID;
+import java.util.WeakHashMap;
+
+import org.apache.accumulo.core.data.ByteSequence;
+import org.apache.accumulo.core.data.Mutation;
+import org.apache.accumulo.core.data.Range;
+import org.apache.accumulo.core.data.Value;
+import org.apache.accumulo.core.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
+ *
+ */
+
+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");
+
+    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);
+  }
+
+  public KeyExtent(Text table, Text endRow, Text prevEndRow) {
+    this.setTableId(table);
+    this.setEndRow(endRow, false, true);
+    this.setPrevEndRow(prevEndRow, false, true);
+
+    check();
+  }
+
+  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();
+  }
+
+  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);
+  }
+
+  // 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();
+  }
+
+  // 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) {
+
+    decodeMetadataRow(flattenedExtent);
+
+    this.setPrevEndRow(null, false, false);
+    if (prevEndRow != null)
+      this.setPrevEndRow(prevEndRow, false, true);
+
+    check();
+  }
+
+  /**
+   * 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;
+  }
+
+  /**
+   * Returns the extent's table id
+   *
+   */
+  public Text getTableId() {
+    return textTableId;
+  }
+
+  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);
+  }
+
+  /**
+   * Returns this extent's end row
+   *
+   */
+  public Text getEndRow() {
+    return textEndRow;
+  }
+
+  /**
+   * 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();
+  }
+
+  /**
+   * Sets the previous extent's end row
+   *
+   */
+  public void setPrevEndRow(Text prevEndRow) {
+    setPrevEndRow(prevEndRow, true, true);
+  }
+
+  @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();
+  }
+
+  @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);
+    }
+  }
+
+  /**
+   * 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;
+  }
+
+  @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());
+  }
+
+  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);
+  }
+
+  @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);
+  }
+
+  @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;
+  }
+
+  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());
+  }
+
+  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(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;
+  }
+
+  public Range toDataRange() {
+    return new Range(getPrevEndRow(), false, getEndRow(), true);
+  }
+
+  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;
+  }
+
+  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 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;
+    }
+
+    if (extent.getPrevEndRow().compareTo(first.getPrevEndRow()) >= 0)
+      return first;
+    return null;
+  }
+
+  private static boolean startsAfter(KeyExtent nke, KeyExtent ke) {
+
+    int tiCmp = ke.getTableId().compareTo(nke.getTableId());
+
+    if (tiCmp > 0) {
+      return true;
+    }
+
+    return ke.getPrevEndRow() != null && nke.getEndRow() != null && ke.getPrevEndRow().compareTo(nke.getEndRow()) >= 0;
+  }
+
+  private static Text rowAfterPrevRow(KeyExtent nke) {
+    Text row = new Text(nke.getPrevEndRow());
+    row.append(new byte[] {0}, 0, 1);
+    return row;
+  }
+
+  // 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);
+    }
+
+    TreeSet<KeyExtent> result = new TreeSet<KeyExtent>();
+    for (KeyExtent ke : start) {
+      if (startsAfter(nke, ke)) {
+        break;
+      }
+      result.add(ke);
+    }
+    return result;
+  }
+
+  public boolean overlaps(KeyExtent other) {
+    SortedSet<KeyExtent> set = new TreeSet<KeyExtent>();
+    set.add(other);
+    return !findOverlapping(this, set).isEmpty();
+  }
+
+  // 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;
+  }
+
+  public static Text getMetadataEntry(KeyExtent extent) {
+    return getMetadataEntry(extent.getTableId(), extent.getEndRow());
+  }
+
+  public TKeyExtent toThrift() {
+    return new TKeyExtent(TextUtil.getByteBuffer(textTableId), textEndRow == null ? null : TextUtil.getByteBuffer(textEndRow), textPrevEndRow == null ? null
+        : TextUtil.getByteBuffer(textPrevEndRow));
+  }
+
+  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);
+
+    if (prevExtent.getEndRow() == null)
+      return false;
+
+    if (getPrevEndRow() == null)
+      return false;
+
+    return prevExtent.getEndRow().equals(getPrevEndRow());
+  }
+
+  public boolean isMeta() {
+    return getTableId().toString().equals(MetadataTable.ID) || isRootTablet();
+  }
+
+  public boolean isRootTablet() {
+    return getTableId().toString().equals(RootTable.ID);
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/c6252914/core/src/main/java/org/apache/accumulo/core/data/impl/TabletIDImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/data/impl/TabletIDImpl.java b/core/src/main/java/org/apache/accumulo/core/data/impl/TabletIDImpl.java
new file mode 100644
index 0000000..dd90e0d
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/data/impl/TabletIDImpl.java
@@ -0,0 +1,100 @@
+/*
+ * 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.accumulo.core.data.TabletID;
+import org.apache.hadoop.io.Text;
+
+import com.google.common.base.Function;
+
+public class TabletIDImpl implements TabletID {
+
+  private KeyExtent ke;
+
+  @SuppressWarnings("deprecation")
+  public static final Function<org.apache.accumulo.core.data.KeyExtent,TabletID> KE_2_TID_OLD = new Function<org.apache.accumulo.core.data.KeyExtent,TabletID>() {
+    @Override
+    public TabletID apply(org.apache.accumulo.core.data.KeyExtent input) {
+      //the following if null check is to appease findbugs... grumble grumble spent a good part of my morning looking into this
+      // http://sourceforge.net/p/findbugs/bugs/1139/
+      // https://code.google.com/p/guava-libraries/issues/detail?id=920
+      if(input == null)
+        return null;
+      return new TabletIDImpl(input);
+    }
+  };
+
+  @SuppressWarnings("deprecation")
+  public static final Function<TabletID, org.apache.accumulo.core.data.KeyExtent> TID_2_KE_OLD = new Function<TabletID, org.apache.accumulo.core.data.KeyExtent>() {
+    @Override
+    public org.apache.accumulo.core.data.KeyExtent apply(TabletID input) {
+      if(input == null)
+        return null;
+      return new org.apache.accumulo.core.data.KeyExtent(input.getTableId(), input.getEndRow(), input.getPrevEndRow());
+    }
+
+  };
+
+  @Deprecated
+  public TabletIDImpl(org.apache.accumulo.core.data.KeyExtent ke) {
+    this.ke = new KeyExtent(ke.getTableId(), ke.getEndRow(), ke.getPrevEndRow());
+  }
+
+  public TabletIDImpl(KeyExtent ke) {
+    this.ke = ke;
+  }
+
+  @Override
+  public int compareTo(TabletID o) {
+    return ke.compareTo(((TabletIDImpl) o).ke);
+  }
+
+  @Override
+  public Text getTableId() {
+    return ke.getTableId();
+  }
+
+  @Override
+  public Text getEndRow() {
+    return ke.getEndRow();
+  }
+
+  @Override
+  public Text getPrevEndRow() {
+    return ke.getPrevEndRow();
+  }
+
+  @Override
+  public int hashCode() {
+    return ke.hashCode();
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (o instanceof TabletIDImpl) {
+      return ke.equals(((TabletIDImpl) o).ke);
+    }
+
+    return false;
+  }
+
+  @Override
+  public String toString(){
+    return ke.toString();
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/c6252914/core/src/main/java/org/apache/accumulo/core/iterators/IteratorUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/iterators/IteratorUtil.java b/core/src/main/java/org/apache/accumulo/core/iterators/IteratorUtil.java
index 587fc33..5efa2c2 100644
--- a/core/src/main/java/org/apache/accumulo/core/iterators/IteratorUtil.java
+++ b/core/src/main/java/org/apache/accumulo/core/iterators/IteratorUtil.java
@@ -33,8 +33,8 @@ import org.apache.accumulo.core.conf.AccumuloConfiguration;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.constraints.DefaultKeySizeConstraint;
 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.impl.KeyExtent;
 import org.apache.accumulo.core.data.thrift.IterInfo;
 import org.apache.accumulo.core.iterators.system.SynchronizedIterator;
 import org.apache.accumulo.core.iterators.user.VersioningIterator;
@@ -56,24 +56,21 @@ public class IteratorUtil {
 
   private static final Logger log = LoggerFactory.getLogger(IteratorUtil.class);
 
+  /**
+   * Even though this type is not in a public API package, its used by methods in the public API. Therefore it should be treated as public API and should not
+   * reference any non public API types. Also this type can not be moved.
+   */
   public static enum IteratorScope {
     majc, minc, scan;
 
     /**
      * Fetch the correct configuration key prefix for the given scope. Throws an IllegalArgumentException if no property exists for the given scope.
+     *
+     * @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
     public static Property getProperty(IteratorScope scope) {
-      Preconditions.checkNotNull(scope);
-      switch (scope) {
-        case scan:
-          return Property.TABLE_ITERATOR_SCAN_PREFIX;
-        case minc:
-          return Property.TABLE_ITERATOR_MINC_PREFIX;
-        case majc:
-          return Property.TABLE_ITERATOR_MAJC_PREFIX;
-        default:
-          throw new IllegalStateException("Could not find configuration property for IteratorScope");
-      }
+      return IteratorUtil.getProperty(scope);
     }
   }
 
@@ -88,6 +85,23 @@ public class IteratorUtil {
   }
 
   /**
+   * Fetch the correct configuration key prefix for the given scope. Throws an IllegalArgumentException if no property exists for the given scope.
+   */
+  static Property getProperty(IteratorScope scope) {
+    Preconditions.checkNotNull(scope);
+    switch (scope) {
+      case scan:
+        return Property.TABLE_ITERATOR_SCAN_PREFIX;
+      case minc:
+        return Property.TABLE_ITERATOR_MINC_PREFIX;
+      case majc:
+        return Property.TABLE_ITERATOR_MAJC_PREFIX;
+      default:
+        throw new IllegalStateException("Could not find configuration property for IteratorScope");
+    }
+  }
+
+  /**
    * Generate the initial (default) properties for a table
    *
    * @param limitVersion
@@ -124,7 +138,7 @@ public class IteratorUtil {
   }
 
   protected static void parseIterConf(IteratorScope scope, List<IterInfo> iters, Map<String,Map<String,String>> allOptions, AccumuloConfiguration conf) {
-    final Property scopeProperty = IteratorScope.getProperty(scope);
+    final Property scopeProperty = getProperty(scope);
     final String scopePropertyKey = scopeProperty.getKey();
 
     for (Entry<String,String> entry : conf.getAllPropertiesWithPrefix(scopeProperty).entrySet()) {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/c6252914/core/src/main/java/org/apache/accumulo/core/iterators/system/MultiIterator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/iterators/system/MultiIterator.java b/core/src/main/java/org/apache/accumulo/core/iterators/system/MultiIterator.java
index 426ae5f..aef6aeb 100644
--- a/core/src/main/java/org/apache/accumulo/core/iterators/system/MultiIterator.java
+++ b/core/src/main/java/org/apache/accumulo/core/iterators/system/MultiIterator.java
@@ -24,9 +24,9 @@ import java.util.Map;
 
 import org.apache.accumulo.core.data.ByteSequence;
 import org.apache.accumulo.core.data.Key;
-import org.apache.accumulo.core.data.KeyExtent;
 import org.apache.accumulo.core.data.Range;
 import org.apache.accumulo.core.data.Value;
+import org.apache.accumulo.core.data.impl.KeyExtent;
 import org.apache.accumulo.core.iterators.IteratorEnvironment;
 import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/c6252914/core/src/main/java/org/apache/accumulo/core/metadata/MetadataLocationObtainer.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/metadata/MetadataLocationObtainer.java b/core/src/main/java/org/apache/accumulo/core/metadata/MetadataLocationObtainer.java
index 20f2130..c8c61aa 100644
--- a/core/src/main/java/org/apache/accumulo/core/metadata/MetadataLocationObtainer.java
+++ b/core/src/main/java/org/apache/accumulo/core/metadata/MetadataLocationObtainer.java
@@ -44,10 +44,10 @@ import org.apache.accumulo.core.client.impl.TabletServerBatchReaderIterator.Resu
 import org.apache.accumulo.core.client.impl.ThriftScanner;
 import org.apache.accumulo.core.data.Column;
 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.data.thrift.IterInfo;
 import org.apache.accumulo.core.iterators.user.WholeRowIterator;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/c6252914/core/src/main/java/org/apache/accumulo/core/metadata/MetadataServicer.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/metadata/MetadataServicer.java b/core/src/main/java/org/apache/accumulo/core/metadata/MetadataServicer.java
index 3b443f4..c6cf61e 100644
--- a/core/src/main/java/org/apache/accumulo/core/metadata/MetadataServicer.java
+++ b/core/src/main/java/org/apache/accumulo/core/metadata/MetadataServicer.java
@@ -24,7 +24,7 @@ import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.client.impl.ClientContext;
-import org.apache.accumulo.core.data.KeyExtent;
+import org.apache.accumulo.core.data.impl.KeyExtent;
 
 /**
  * Provides a consolidated API for handling table metadata

http://git-wip-us.apache.org/repos/asf/accumulo/blob/c6252914/core/src/main/java/org/apache/accumulo/core/metadata/RootTable.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/metadata/RootTable.java b/core/src/main/java/org/apache/accumulo/core/metadata/RootTable.java
index 24148b1..292ba3b 100644
--- a/core/src/main/java/org/apache/accumulo/core/metadata/RootTable.java
+++ b/core/src/main/java/org/apache/accumulo/core/metadata/RootTable.java
@@ -17,7 +17,7 @@
 package org.apache.accumulo.core.metadata;
 
 import org.apache.accumulo.core.client.impl.Namespaces;
-import org.apache.accumulo.core.data.KeyExtent;
+import org.apache.accumulo.core.data.impl.KeyExtent;
 import org.apache.hadoop.io.Text;
 
 /**

http://git-wip-us.apache.org/repos/asf/accumulo/blob/c6252914/core/src/main/java/org/apache/accumulo/core/metadata/ServicerForRootTable.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/metadata/ServicerForRootTable.java b/core/src/main/java/org/apache/accumulo/core/metadata/ServicerForRootTable.java
index d02242c..73a943d 100644
--- a/core/src/main/java/org/apache/accumulo/core/metadata/ServicerForRootTable.java
+++ b/core/src/main/java/org/apache/accumulo/core/metadata/ServicerForRootTable.java
@@ -23,7 +23,7 @@ import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.Instance;
 import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.client.impl.ClientContext;
-import org.apache.accumulo.core.data.KeyExtent;
+import org.apache.accumulo.core.data.impl.KeyExtent;
 
 /**
  * A metadata servicer for the root table.<br />

http://git-wip-us.apache.org/repos/asf/accumulo/blob/c6252914/core/src/main/java/org/apache/accumulo/core/metadata/TableMetadataServicer.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/metadata/TableMetadataServicer.java b/core/src/main/java/org/apache/accumulo/core/metadata/TableMetadataServicer.java
index 7e2ae0a..fcd74a3 100644
--- a/core/src/main/java/org/apache/accumulo/core/metadata/TableMetadataServicer.java
+++ b/core/src/main/java/org/apache/accumulo/core/metadata/TableMetadataServicer.java
@@ -27,8 +27,8 @@ 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.data.Key;
-import org.apache.accumulo.core.data.KeyExtent;
 import org.apache.accumulo.core.data.Value;
+import org.apache.accumulo.core.data.impl.KeyExtent;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.hadoop.io.Text;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/c6252914/core/src/main/java/org/apache/accumulo/core/replication/ReplicationConfigurationUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/replication/ReplicationConfigurationUtil.java b/core/src/main/java/org/apache/accumulo/core/replication/ReplicationConfigurationUtil.java
index 0817d3b..f711868 100644
--- a/core/src/main/java/org/apache/accumulo/core/replication/ReplicationConfigurationUtil.java
+++ b/core/src/main/java/org/apache/accumulo/core/replication/ReplicationConfigurationUtil.java
@@ -18,7 +18,7 @@ package org.apache.accumulo.core.replication;
 
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
 import org.apache.accumulo.core.conf.Property;
-import org.apache.accumulo.core.data.KeyExtent;
+import org.apache.accumulo.core.data.impl.KeyExtent;
 
 /**
  * Encapsulates configuration semantics around replication

http://git-wip-us.apache.org/repos/asf/accumulo/blob/c6252914/core/src/main/java/org/apache/accumulo/core/rpc/SaslClientDigestCallbackHandler.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/rpc/SaslClientDigestCallbackHandler.java b/core/src/main/java/org/apache/accumulo/core/rpc/SaslClientDigestCallbackHandler.java
index 18dd7e1..6504741 100644
--- a/core/src/main/java/org/apache/accumulo/core/rpc/SaslClientDigestCallbackHandler.java
+++ b/core/src/main/java/org/apache/accumulo/core/rpc/SaslClientDigestCallbackHandler.java
@@ -27,7 +27,7 @@ import javax.security.auth.callback.UnsupportedCallbackException;
 import javax.security.sasl.RealmCallback;
 import javax.security.sasl.RealmChoiceCallback;
 
-import org.apache.accumulo.core.client.security.tokens.DelegationToken;
+import org.apache.accumulo.core.client.impl.DelegationTokenImpl;
 import org.apache.commons.lang.builder.HashCodeBuilder;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -43,7 +43,7 @@ public class SaslClientDigestCallbackHandler extends SaslDigestCallbackHandler {
   private final String userName;
   private final char[] userPassword;
 
-  public SaslClientDigestCallbackHandler(DelegationToken token) {
+  public SaslClientDigestCallbackHandler(DelegationTokenImpl token) {
     checkNotNull(token);
     this.userName = encodeIdentifier(token.getIdentifier().getBytes());
     this.userPassword = encodePassword(token.getPassword());

http://git-wip-us.apache.org/repos/asf/accumulo/blob/c6252914/core/src/main/java/org/apache/accumulo/core/rpc/SaslConnectionParams.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/rpc/SaslConnectionParams.java b/core/src/main/java/org/apache/accumulo/core/rpc/SaslConnectionParams.java
index 33392b9..0e88967 100644
--- a/core/src/main/java/org/apache/accumulo/core/rpc/SaslConnectionParams.java
+++ b/core/src/main/java/org/apache/accumulo/core/rpc/SaslConnectionParams.java
@@ -30,8 +30,8 @@ import javax.security.sasl.Sasl;
 
 import org.apache.accumulo.core.client.ClientConfiguration;
 import org.apache.accumulo.core.client.ClientConfiguration.ClientProperty;
+import org.apache.accumulo.core.client.impl.DelegationTokenImpl;
 import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
-import org.apache.accumulo.core.client.security.tokens.DelegationToken;
 import org.apache.accumulo.core.client.security.tokens.KerberosToken;
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
 import org.apache.accumulo.core.conf.Property;
@@ -150,9 +150,9 @@ public class SaslConnectionParams {
       mechanism = SaslMechanism.GSSAPI;
       // No callbackhandlers necessary for GSSAPI
       callbackHandler = null;
-    } else if (token instanceof DelegationToken) {
+    } else if (token instanceof DelegationTokenImpl) {
       mechanism = SaslMechanism.DIGEST_MD5;
-      callbackHandler = new SaslClientDigestCallbackHandler((DelegationToken) token);
+      callbackHandler = new SaslClientDigestCallbackHandler((DelegationTokenImpl) token);
     } else {
       throw new IllegalArgumentException("Cannot determine SASL mechanism for token class: " + token.getClass());
     }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/c6252914/core/src/main/java/org/apache/accumulo/core/security/AuthenticationTokenIdentifier.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/security/AuthenticationTokenIdentifier.java b/core/src/main/java/org/apache/accumulo/core/security/AuthenticationTokenIdentifier.java
deleted file mode 100644
index 0b671d8..0000000
--- a/core/src/main/java/org/apache/accumulo/core/security/AuthenticationTokenIdentifier.java
+++ /dev/null
@@ -1,210 +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.security;
-
-import static com.google.common.base.Preconditions.checkNotNull;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.nio.ByteBuffer;
-
-import org.apache.accumulo.core.client.admin.DelegationTokenConfig;
-import org.apache.accumulo.core.security.thrift.TAuthenticationTokenIdentifier;
-import org.apache.accumulo.core.util.ThriftMessageUtil;
-import org.apache.commons.lang.builder.HashCodeBuilder;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hadoop.security.token.Token;
-import org.apache.hadoop.security.token.TokenIdentifier;
-
-/**
- * Implementation that identifies the underlying {@link Token} for Accumulo.
- */
-public class AuthenticationTokenIdentifier extends TokenIdentifier {
-  public static final Text TOKEN_KIND = new Text("ACCUMULO_AUTH_TOKEN");
-
-  private TAuthenticationTokenIdentifier impl = null;
-  private DelegationTokenConfig cfg = null;
-
-  public AuthenticationTokenIdentifier() {
-    // noop for Writable
-  }
-
-  public AuthenticationTokenIdentifier(String principal) {
-    this(principal, null);
-  }
-
-  public AuthenticationTokenIdentifier(String principal, DelegationTokenConfig cfg) {
-    checkNotNull(principal);
-    impl = new TAuthenticationTokenIdentifier(principal);
-    this.cfg = cfg;
-  }
-
-  public AuthenticationTokenIdentifier(String principal, int keyId, long issueDate, long expirationDate, String instanceId) {
-    checkNotNull(principal);
-    impl = new TAuthenticationTokenIdentifier(principal);
-    impl.setKeyId(keyId);
-    impl.setIssueDate(issueDate);
-    impl.setExpirationDate(expirationDate);
-    impl.setInstanceId(instanceId);
-  }
-
-  public AuthenticationTokenIdentifier(AuthenticationTokenIdentifier identifier) {
-    checkNotNull(identifier);
-    impl = new TAuthenticationTokenIdentifier(identifier.getThriftIdentifier());
-  }
-
-  public AuthenticationTokenIdentifier(TAuthenticationTokenIdentifier identifier) {
-    checkNotNull(identifier);
-    impl = new TAuthenticationTokenIdentifier(identifier);
-  }
-
-  public void setKeyId(int keyId) {
-    impl.setKeyId(keyId);
-  }
-
-  public int getKeyId() {
-    checkNotNull(impl, "Identifier not initialized");
-    return impl.getKeyId();
-  }
-
-  public void setIssueDate(long issueDate) {
-    checkNotNull(impl, "Identifier not initialized");
-    impl.setIssueDate(issueDate);
-  }
-
-  public long getIssueDate() {
-    checkNotNull(impl, "Identifier not initialized");
-    return impl.getIssueDate();
-  }
-
-  public void setExpirationDate(long expirationDate) {
-    checkNotNull(impl, "Identifier not initialized");
-    impl.setExpirationDate(expirationDate);
-  }
-
-  public long getExpirationDate() {
-    checkNotNull(impl, "Identifier not initialized");
-    return impl.getExpirationDate();
-  }
-
-  public void setInstanceId(String instanceId) {
-    checkNotNull(impl, "Identifier not initialized");
-    impl.setInstanceId(instanceId);
-  }
-
-  public String getInstanceId() {
-    checkNotNull(impl, "Identifier not initialized");
-    return impl.getInstanceId();
-  }
-
-  public TAuthenticationTokenIdentifier getThriftIdentifier() {
-    checkNotNull(impl);
-    return impl;
-  }
-
-  /**
-   * A configuration from the requesting user, may be null.
-   */
-  public DelegationTokenConfig getConfig() {
-    return cfg;
-  }
-
-  @Override
-  public void write(DataOutput out) throws IOException {
-    if (null != impl) {
-      ThriftMessageUtil msgUtil = new ThriftMessageUtil();
-      ByteBuffer serialized = msgUtil.serialize(impl);
-      out.writeInt(serialized.limit());
-      out.write(serialized.array(), serialized.arrayOffset(), serialized.limit());
-    } else {
-      out.writeInt(0);
-    }
-  }
-
-  @Override
-  public void readFields(DataInput in) throws IOException {
-    int length = in.readInt();
-    if (length > 0) {
-      ThriftMessageUtil msgUtil = new ThriftMessageUtil();
-      byte[] serialized = new byte[length];
-      in.readFully(serialized);
-      impl = new TAuthenticationTokenIdentifier();
-      msgUtil.deserialize(serialized, impl);
-    }
-  }
-
-  @Override
-  public Text getKind() {
-    return TOKEN_KIND;
-  }
-
-  @Override
-  public UserGroupInformation getUser() {
-    if (null != impl && impl.isSetPrincipal()) {
-      return UserGroupInformation.createRemoteUser(impl.getPrincipal());
-    }
-    return null;
-  }
-
-  @Override
-  public int hashCode() {
-    if (null == impl) {
-      return 0;
-    }
-    HashCodeBuilder hcb = new HashCodeBuilder(7, 11);
-    if (impl.isSetPrincipal()) {
-      hcb.append(impl.getPrincipal());
-    }
-    if (impl.isSetKeyId()) {
-      hcb.append(impl.getKeyId());
-    }
-    if (impl.isSetIssueDate()) {
-      hcb.append(impl.getIssueDate());
-    }
-    if (impl.isSetExpirationDate()) {
-      hcb.append(impl.getExpirationDate());
-    }
-    if (impl.isSetInstanceId()) {
-      hcb.append(impl.getInstanceId());
-    }
-    return hcb.toHashCode();
-  }
-
-  @Override
-  public String toString() {
-    StringBuilder sb = new StringBuilder(128);
-    sb.append("AuthenticationTokenIdentifier(").append(impl).append(")");
-    return sb.toString();
-  }
-
-  @Override
-  public boolean equals(Object o) {
-    if (null == o) {
-      return false;
-    }
-    if (o instanceof AuthenticationTokenIdentifier) {
-      AuthenticationTokenIdentifier other = (AuthenticationTokenIdentifier) o;
-      if (null == impl) {
-        return null == other.impl;
-      }
-      return impl.equals(other.impl);
-    }
-    return false;
-  }
-}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/c6252914/core/src/main/java/org/apache/accumulo/core/security/Credentials.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/security/Credentials.java b/core/src/main/java/org/apache/accumulo/core/security/Credentials.java
deleted file mode 100644
index ed7789a..0000000
--- a/core/src/main/java/org/apache/accumulo/core/security/Credentials.java
+++ /dev/null
@@ -1,157 +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.security;
-
-import static java.nio.charset.StandardCharsets.UTF_8;
-
-import java.nio.ByteBuffer;
-
-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.thrift.SecurityErrorCode;
-import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
-import org.apache.accumulo.core.client.security.tokens.AuthenticationToken.AuthenticationTokenSerializer;
-import org.apache.accumulo.core.security.thrift.TCredentials;
-import org.apache.accumulo.core.util.Base64;
-
-/**
- * A wrapper for internal use. This class carries the instance, principal, and authentication token for use in the public API, in a non-serialized form. This is
- * important, so that the authentication token carried in a {@link Connector} can be destroyed, invalidating future RPC operations from that {@link Connector}.
- * <p>
- * See ACCUMULO-1312
- *
- * @since 1.6.0
- */
-public class Credentials {
-
-  private String principal;
-  private AuthenticationToken token;
-
-  /**
-   * Creates a new credentials object.
-   *
-   * @param principal
-   *          unique identifier for the entity (e.g. a user or service) authorized for these credentials
-   * @param token
-   *          authentication token used to prove that the principal for these credentials has been properly verified
-   */
-  public Credentials(String principal, AuthenticationToken token) {
-    this.principal = principal;
-    this.token = token;
-  }
-
-  /**
-   * Gets the principal.
-   *
-   * @return unique identifier for the entity (e.g. a user or service) authorized for these credentials
-   */
-  public String getPrincipal() {
-    return principal;
-  }
-
-  /**
-   * Gets the authentication token.
-   *
-   * @return authentication token used to prove that the principal for these credentials has been properly verified
-   */
-  public AuthenticationToken getToken() {
-    return token;
-  }
-
-  /**
-   * Converts the current object to the relevant thrift type. The object returned from this contains a non-destroyable version of the
-   * {@link AuthenticationToken}, so this should be used just before placing on the wire, and references to it should be tightly controlled.
-   *
-   * @param instance
-   *          client instance
-   * @return Thrift credentials
-   * @throws RuntimeException
-   *           if the authentication token has been destroyed (expired)
-   */
-  public TCredentials toThrift(Instance instance) {
-    TCredentials tCreds = new TCredentials(getPrincipal(), getToken().getClass().getName(),
-        ByteBuffer.wrap(AuthenticationTokenSerializer.serialize(getToken())), instance.getInstanceID());
-    if (getToken().isDestroyed())
-      throw new RuntimeException("Token has been destroyed", new AccumuloSecurityException(getPrincipal(), SecurityErrorCode.TOKEN_EXPIRED));
-    return tCreds;
-  }
-
-  /**
-   * Converts a given thrift object to our internal Credentials representation.
-   *
-   * @param serialized
-   *          a Thrift encoded set of credentials
-   * @return a new Credentials instance; destroy the token when you're done.
-   */
-  public static Credentials fromThrift(TCredentials serialized) {
-    return new Credentials(serialized.getPrincipal(), AuthenticationTokenSerializer.deserialize(serialized.getTokenClassName(), serialized.getToken()));
-  }
-
-  /**
-   * Converts the current object to a serialized form. The object returned from this contains a non-destroyable version of the {@link AuthenticationToken}, so
-   * references to it should be tightly controlled.
-   *
-   * @return serialized form of these credentials
-   */
-  public final String serialize() {
-    return (getPrincipal() == null ? "-" : Base64.encodeBase64String(getPrincipal().getBytes(UTF_8))) + ":"
-        + (getToken() == null ? "-" : Base64.encodeBase64String(getToken().getClass().getName().getBytes(UTF_8))) + ":"
-        + (getToken() == null ? "-" : Base64.encodeBase64String(AuthenticationTokenSerializer.serialize(getToken())));
-  }
-
-  /**
-   * Converts the serialized form to an instance of {@link Credentials}. The original serialized form will not be affected.
-   *
-   * @param serializedForm
-   *          serialized form of credentials
-   * @return deserialized credentials
-   */
-  public static final Credentials deserialize(String serializedForm) {
-    String[] split = serializedForm.split(":", 3);
-    String principal = split[0].equals("-") ? null : new String(Base64.decodeBase64(split[0]), UTF_8);
-    String tokenType = split[1].equals("-") ? null : new String(Base64.decodeBase64(split[1]), UTF_8);
-    AuthenticationToken token = null;
-    if (!split[2].equals("-")) {
-      byte[] tokenBytes = Base64.decodeBase64(split[2]);
-      token = AuthenticationTokenSerializer.deserialize(tokenType, tokenBytes);
-    }
-    return new Credentials(principal, token);
-  }
-
-  @Override
-  public int hashCode() {
-    return getPrincipal() == null ? 0 : getPrincipal().hashCode();
-  }
-
-  @Override
-  public boolean equals(Object obj) {
-    if (obj == null || !(obj instanceof Credentials))
-      return false;
-    Credentials other = Credentials.class.cast(obj);
-    boolean pEq = getPrincipal() == null ? (other.getPrincipal() == null) : (getPrincipal().equals(other.getPrincipal()));
-    if (!pEq)
-      return false;
-    boolean tEq = getToken() == null ? (other.getToken() == null) : (getToken().equals(other.getToken()));
-    return tEq;
-  }
-
-  @Override
-  public String toString() {
-    return getClass().getName() + ":" + getPrincipal() + ":" + (getToken() == null ? null : getToken().getClass().getName()) + ":<hidden>";
-  }
-}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/c6252914/core/src/main/java/org/apache/accumulo/core/security/VisibilityConstraint.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/security/VisibilityConstraint.java b/core/src/main/java/org/apache/accumulo/core/security/VisibilityConstraint.java
index 90cf951..611c8d4 100644
--- a/core/src/main/java/org/apache/accumulo/core/security/VisibilityConstraint.java
+++ b/core/src/main/java/org/apache/accumulo/core/security/VisibilityConstraint.java
@@ -14,78 +14,16 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.accumulo.core.security;
-
-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.constraints.Constraint;
-import org.apache.accumulo.core.data.ColumnUpdate;
-import org.apache.accumulo.core.data.Mutation;
-import org.apache.accumulo.core.util.BadArgumentException;
+package org.apache.accumulo.core.security;
 
 /**
- * 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>
+ *
+ * @deprecated since 1.7.0 This is server side code not intended to exist in a public API package. This class references types that are not in the public API
+ *             and therefore is not guaranteed to be stable. It was deprecated to clearly communicate this. Use
+ *             {@link org.apache.accumulo.core.constraints.VisibilityConstraint} instead.
  */
-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);
-
-          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);
-      }
-    }
+@Deprecated
+public class VisibilityConstraint extends org.apache.accumulo.core.constraints.VisibilityConstraint {
 
-    return null;
-  }
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/c6252914/core/src/main/java/org/apache/accumulo/core/security/VisibilityEvaluator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/security/VisibilityEvaluator.java b/core/src/main/java/org/apache/accumulo/core/security/VisibilityEvaluator.java
index dd07619..99aa383 100644
--- a/core/src/main/java/org/apache/accumulo/core/security/VisibilityEvaluator.java
+++ b/core/src/main/java/org/apache/accumulo/core/security/VisibilityEvaluator.java
@@ -18,7 +18,6 @@ package org.apache.accumulo.core.security;
 
 import java.util.ArrayList;
 
-import org.apache.accumulo.core.constraints.Constraint.Environment;
 import org.apache.accumulo.core.security.ColumnVisibility.Node;
 
 /**
@@ -80,13 +79,12 @@ public class VisibilityEvaluator {
   }
 
   /**
-   * Creates a new evaluator for the authorizations found in the given environment.
+   * Creates a new evaluator for the authorizations found in the given container.
    *
-   * @param env
-   *          environment containing authorizations
+   * @since 1.7.0
    */
-  VisibilityEvaluator(Environment env) {
-    this.auths = env.getAuthorizationsContainer();
+  public VisibilityEvaluator(AuthorizationContainer authsContainer) {
+    this.auths = authsContainer;
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/accumulo/blob/c6252914/core/src/main/java/org/apache/accumulo/core/tabletserver/log/LogEntry.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/tabletserver/log/LogEntry.java b/core/src/main/java/org/apache/accumulo/core/tabletserver/log/LogEntry.java
index f13f77e..7fe61d1 100644
--- a/core/src/main/java/org/apache/accumulo/core/tabletserver/log/LogEntry.java
+++ b/core/src/main/java/org/apache/accumulo/core/tabletserver/log/LogEntry.java
@@ -22,8 +22,8 @@ import java.util.Arrays;
 import java.util.Collection;
 
 import org.apache.accumulo.core.data.Key;
-import org.apache.accumulo.core.data.KeyExtent;
 import org.apache.accumulo.core.data.Value;
+import org.apache.accumulo.core.data.impl.KeyExtent;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema;
 import org.apache.hadoop.io.DataInputBuffer;
 import org.apache.hadoop.io.DataOutputBuffer;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/c6252914/core/src/main/java/org/apache/accumulo/core/util/Merge.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/util/Merge.java b/core/src/main/java/org/apache/accumulo/core/util/Merge.java
index 6bef51a..3c60d25 100644
--- a/core/src/main/java/org/apache/accumulo/core/util/Merge.java
+++ b/core/src/main/java/org/apache/accumulo/core/util/Merge.java
@@ -31,8 +31,8 @@ import org.apache.accumulo.core.conf.AccumuloConfiguration;
 import org.apache.accumulo.core.conf.ConfigurationCopy;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.data.Key;
-import org.apache.accumulo.core.data.KeyExtent;
 import org.apache.accumulo.core.data.Value;
+import org.apache.accumulo.core.data.impl.KeyExtent;
 import org.apache.accumulo.core.metadata.MetadataTable;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.DataFileColumnFamily;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/c6252914/core/src/test/java/org/apache/accumulo/core/client/impl/ScannerImplTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/accumulo/core/client/impl/ScannerImplTest.java b/core/src/test/java/org/apache/accumulo/core/client/impl/ScannerImplTest.java
index 0c65422..95b0903 100644
--- a/core/src/test/java/org/apache/accumulo/core/client/impl/ScannerImplTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/client/impl/ScannerImplTest.java
@@ -23,7 +23,6 @@ import org.apache.accumulo.core.client.Scanner;
 import org.apache.accumulo.core.client.mock.MockInstance;
 import org.apache.accumulo.core.client.security.tokens.PasswordToken;
 import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.core.security.Credentials;
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/c6252914/core/src/test/java/org/apache/accumulo/core/client/impl/TableOperationsImplTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/accumulo/core/client/impl/TableOperationsImplTest.java b/core/src/test/java/org/apache/accumulo/core/client/impl/TableOperationsImplTest.java
index 73a9024..523d157 100644
--- a/core/src/test/java/org/apache/accumulo/core/client/impl/TableOperationsImplTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/client/impl/TableOperationsImplTest.java
@@ -23,11 +23,10 @@ import org.apache.accumulo.core.client.Connector;
 import org.apache.accumulo.core.client.Instance;
 import org.apache.accumulo.core.client.Scanner;
 import org.apache.accumulo.core.client.security.tokens.PasswordToken;
-import org.apache.accumulo.core.data.KeyExtent;
 import org.apache.accumulo.core.data.Range;
+import org.apache.accumulo.core.data.impl.KeyExtent;
 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.hadoop.io.Text;
 import org.easymock.EasyMock;
 import org.junit.Test;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/c6252914/core/src/test/java/org/apache/accumulo/core/client/impl/TabletLocatorImplTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/accumulo/core/client/impl/TabletLocatorImplTest.java b/core/src/test/java/org/apache/accumulo/core/client/impl/TabletLocatorImplTest.java
index 939a64c..2e78bd8 100644
--- a/core/src/test/java/org/apache/accumulo/core/client/impl/TabletLocatorImplTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/client/impl/TabletLocatorImplTest.java
@@ -47,16 +47,15 @@ import org.apache.accumulo.core.client.impl.TabletLocatorImpl.TabletServerLockCh
 import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
 import org.apache.accumulo.core.data.Key;
-import org.apache.accumulo.core.data.KeyExtent;
 import org.apache.accumulo.core.data.Mutation;
 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.metadata.MetadataLocationObtainer;
 import org.apache.accumulo.core.metadata.MetadataTable;
 import org.apache.accumulo.core.metadata.RootTable;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection;
-import org.apache.accumulo.core.security.Credentials;
 import org.apache.hadoop.io.Text;
 import org.junit.Before;
 import org.junit.Test;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/c6252914/core/src/test/java/org/apache/accumulo/core/client/impl/TabletServerBatchReaderTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/accumulo/core/client/impl/TabletServerBatchReaderTest.java b/core/src/test/java/org/apache/accumulo/core/client/impl/TabletServerBatchReaderTest.java
index 5c7eb1b..23c223e 100644
--- a/core/src/test/java/org/apache/accumulo/core/client/impl/TabletServerBatchReaderTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/client/impl/TabletServerBatchReaderTest.java
@@ -21,7 +21,6 @@ import org.apache.accumulo.core.client.ClientConfiguration;
 import org.apache.accumulo.core.client.mock.MockInstance;
 import org.apache.accumulo.core.client.security.tokens.PasswordToken;
 import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.core.security.Credentials;
 import org.junit.Before;
 import org.junit.Test;
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/c6252914/core/src/test/java/org/apache/accumulo/core/client/mapred/AccumuloFileOutputFormatTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/accumulo/core/client/mapred/AccumuloFileOutputFormatTest.java b/core/src/test/java/org/apache/accumulo/core/client/mapred/AccumuloFileOutputFormatTest.java
index 9fead34..e389c0b 100644
--- a/core/src/test/java/org/apache/accumulo/core/client/mapred/AccumuloFileOutputFormatTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/client/mapred/AccumuloFileOutputFormatTest.java
@@ -29,6 +29,7 @@ import org.apache.accumulo.core.client.BatchWriter;
 import org.apache.accumulo.core.client.BatchWriterConfig;
 import org.apache.accumulo.core.client.Connector;
 import org.apache.accumulo.core.client.mapreduce.lib.impl.ConfiguratorBase;
+import org.apache.accumulo.core.client.mapreduce.lib.impl.FileOutputConfigurator;
 import org.apache.accumulo.core.client.mock.MockInstance;
 import org.apache.accumulo.core.client.security.tokens.PasswordToken;
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
@@ -224,7 +225,7 @@ public class AccumuloFileOutputFormatTest {
     AccumuloFileOutputFormat.setIndexBlockSize(job, d);
     AccumuloFileOutputFormat.setCompressionType(job, e);
 
-    AccumuloConfiguration acuconf = AccumuloFileOutputFormat.getAccumuloConfiguration(job);
+    AccumuloConfiguration acuconf = FileOutputConfigurator.getAccumuloConfiguration(AccumuloFileOutputFormat.class, job);
 
     assertEquals(7, acuconf.getCount(Property.TABLE_FILE_REPLICATION));
     assertEquals(300l, acuconf.getMemoryInBytes(Property.TABLE_FILE_BLOCK_SIZE));
@@ -245,7 +246,7 @@ public class AccumuloFileOutputFormatTest {
     AccumuloFileOutputFormat.setIndexBlockSize(job, d);
     AccumuloFileOutputFormat.setCompressionType(job, e);
 
-    acuconf = AccumuloFileOutputFormat.getAccumuloConfiguration(job);
+    acuconf = FileOutputConfigurator.getAccumuloConfiguration(AccumuloFileOutputFormat.class, job);
 
     assertEquals(17, acuconf.getCount(Property.TABLE_FILE_REPLICATION));
     assertEquals(1300l, acuconf.getMemoryInBytes(Property.TABLE_FILE_BLOCK_SIZE));

http://git-wip-us.apache.org/repos/asf/accumulo/blob/c6252914/core/src/test/java/org/apache/accumulo/core/client/mapred/TokenFileTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/accumulo/core/client/mapred/TokenFileTest.java b/core/src/test/java/org/apache/accumulo/core/client/mapred/TokenFileTest.java
index a8ca12e..f025783 100644
--- a/core/src/test/java/org/apache/accumulo/core/client/mapred/TokenFileTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/client/mapred/TokenFileTest.java
@@ -31,13 +31,13 @@ import org.apache.accumulo.core.client.BatchWriter;
 import org.apache.accumulo.core.client.BatchWriterConfig;
 import org.apache.accumulo.core.client.Connector;
 import org.apache.accumulo.core.client.Scanner;
+import org.apache.accumulo.core.client.impl.Credentials;
 import org.apache.accumulo.core.client.mock.MockInstance;
 import org.apache.accumulo.core.client.security.tokens.PasswordToken;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.core.security.Credentials;
 import org.apache.accumulo.core.util.CachedConfiguration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configured;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/c6252914/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloFileOutputFormatTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloFileOutputFormatTest.java b/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloFileOutputFormatTest.java
index d9658ae..abc99c9 100644
--- a/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloFileOutputFormatTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloFileOutputFormatTest.java
@@ -27,6 +27,7 @@ import java.io.IOException;
 import org.apache.accumulo.core.client.BatchWriter;
 import org.apache.accumulo.core.client.BatchWriterConfig;
 import org.apache.accumulo.core.client.Connector;
+import org.apache.accumulo.core.client.mapreduce.lib.impl.FileOutputConfigurator;
 import org.apache.accumulo.core.client.mock.MockInstance;
 import org.apache.accumulo.core.client.security.tokens.PasswordToken;
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
@@ -212,7 +213,7 @@ public class AccumuloFileOutputFormatTest {
     AccumuloFileOutputFormat.setIndexBlockSize(job1, d);
     AccumuloFileOutputFormat.setCompressionType(job1, e);
 
-    AccumuloConfiguration acuconf = AccumuloFileOutputFormat.getAccumuloConfiguration(job1);
+    AccumuloConfiguration acuconf = FileOutputConfigurator.getAccumuloConfiguration(AccumuloFileOutputFormat.class, job1.getConfiguration());
 
     assertEquals(7, acuconf.getCount(Property.TABLE_FILE_REPLICATION));
     assertEquals(300l, acuconf.getMemoryInBytes(Property.TABLE_FILE_BLOCK_SIZE));
@@ -233,7 +234,7 @@ public class AccumuloFileOutputFormatTest {
     AccumuloFileOutputFormat.setIndexBlockSize(job2, d);
     AccumuloFileOutputFormat.setCompressionType(job2, e);
 
-    acuconf = AccumuloFileOutputFormat.getAccumuloConfiguration(job2);
+    acuconf = FileOutputConfigurator.getAccumuloConfiguration(AccumuloFileOutputFormat.class, job2.getConfiguration());
 
     assertEquals(17, acuconf.getCount(Property.TABLE_FILE_REPLICATION));
     assertEquals(1300l, acuconf.getMemoryInBytes(Property.TABLE_FILE_BLOCK_SIZE));

http://git-wip-us.apache.org/repos/asf/accumulo/blob/c6252914/core/src/test/java/org/apache/accumulo/core/client/mapreduce/TokenFileTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/accumulo/core/client/mapreduce/TokenFileTest.java b/core/src/test/java/org/apache/accumulo/core/client/mapreduce/TokenFileTest.java
index 092c930..8f49751 100644
--- a/core/src/test/java/org/apache/accumulo/core/client/mapreduce/TokenFileTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/client/mapreduce/TokenFileTest.java
@@ -31,13 +31,13 @@ import org.apache.accumulo.core.client.BatchWriter;
 import org.apache.accumulo.core.client.BatchWriterConfig;
 import org.apache.accumulo.core.client.Connector;
 import org.apache.accumulo.core.client.Scanner;
+import org.apache.accumulo.core.client.impl.Credentials;
 import org.apache.accumulo.core.client.mock.MockInstance;
 import org.apache.accumulo.core.client.security.tokens.PasswordToken;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.core.security.Credentials;
 import org.apache.accumulo.core.util.CachedConfiguration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configured;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/c6252914/core/src/test/java/org/apache/accumulo/core/client/security/tokens/DelegationTokenImplTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/accumulo/core/client/security/tokens/DelegationTokenImplTest.java b/core/src/test/java/org/apache/accumulo/core/client/security/tokens/DelegationTokenImplTest.java
new file mode 100644
index 0000000..99de3b2
--- /dev/null
+++ b/core/src/test/java/org/apache/accumulo/core/client/security/tokens/DelegationTokenImplTest.java
@@ -0,0 +1,73 @@
+/*
+ * 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.security.tokens;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotEquals;
+
+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.client.impl.AuthenticationTokenIdentifier;
+import org.apache.accumulo.core.client.impl.DelegationTokenImpl;
+import org.junit.Test;
+
+public class DelegationTokenImplTest {
+
+  @Test
+  public void testSerialization() throws IOException {
+    AuthenticationTokenIdentifier identifier = new AuthenticationTokenIdentifier("user", 1, 1000l, 2000l, "instanceid");
+    // We don't need a real serialized Token for the password
+    DelegationTokenImpl token = new DelegationTokenImpl(new byte[] {'f', 'a', 'k', 'e'}, identifier);
+    assertEquals(token, token);
+    assertEquals(token.hashCode(), token.hashCode());
+
+    ByteArrayOutputStream baos = new ByteArrayOutputStream();
+    token.write(new DataOutputStream(baos));
+
+    DelegationTokenImpl copy = new DelegationTokenImpl();
+    copy.readFields(new DataInputStream(new ByteArrayInputStream(baos.toByteArray())));
+
+    assertEquals(token, copy);
+    assertEquals(token.hashCode(), copy.hashCode());
+  }
+
+  @Test
+  public void testEquality() throws IOException {
+    AuthenticationTokenIdentifier identifier = new AuthenticationTokenIdentifier("user", 1, 1000l, 2000l, "instanceid");
+    // We don't need a real serialized Token for the password
+    DelegationTokenImpl token = new DelegationTokenImpl(new byte[] {'f', 'a', 'k', 'e'}, identifier);
+
+    AuthenticationTokenIdentifier identifier2 = new AuthenticationTokenIdentifier("user1", 1, 1000l, 2000l, "instanceid");
+    // We don't need a real serialized Token for the password
+    DelegationTokenImpl token2 = new DelegationTokenImpl(new byte[] {'f', 'a', 'k', 'e'}, identifier2);
+
+    assertNotEquals(token, token2);
+    assertNotEquals(token.hashCode(), token2.hashCode());
+
+    // We don't need a real serialized Token for the password
+    DelegationTokenImpl token3 = new DelegationTokenImpl(new byte[] {'f', 'a', 'k', 'e', '0'}, identifier);
+
+    assertNotEquals(token, token3);
+    assertNotEquals(token.hashCode(), token3.hashCode());
+    assertNotEquals(token2, token3);
+    assertNotEquals(token2.hashCode(), token3.hashCode());
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/c6252914/core/src/test/java/org/apache/accumulo/core/client/security/tokens/DelegationTokenTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/accumulo/core/client/security/tokens/DelegationTokenTest.java b/core/src/test/java/org/apache/accumulo/core/client/security/tokens/DelegationTokenTest.java
deleted file mode 100644
index f66a1ee..0000000
--- a/core/src/test/java/org/apache/accumulo/core/client/security/tokens/DelegationTokenTest.java
+++ /dev/null
@@ -1,72 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.accumulo.core.client.security.tokens;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotEquals;
-
-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.security.AuthenticationTokenIdentifier;
-import org.junit.Test;
-
-public class DelegationTokenTest {
-
-  @Test
-  public void testSerialization() throws IOException {
-    AuthenticationTokenIdentifier identifier = new AuthenticationTokenIdentifier("user", 1, 1000l, 2000l, "instanceid");
-    // We don't need a real serialized Token for the password
-    DelegationToken token = new DelegationToken(new byte[] {'f', 'a', 'k', 'e'}, identifier);
-    assertEquals(token, token);
-    assertEquals(token.hashCode(), token.hashCode());
-
-    ByteArrayOutputStream baos = new ByteArrayOutputStream();
-    token.write(new DataOutputStream(baos));
-
-    DelegationToken copy = new DelegationToken();
-    copy.readFields(new DataInputStream(new ByteArrayInputStream(baos.toByteArray())));
-
-    assertEquals(token, copy);
-    assertEquals(token.hashCode(), copy.hashCode());
-  }
-
-  @Test
-  public void testEquality() throws IOException {
-    AuthenticationTokenIdentifier identifier = new AuthenticationTokenIdentifier("user", 1, 1000l, 2000l, "instanceid");
-    // We don't need a real serialized Token for the password
-    DelegationToken token = new DelegationToken(new byte[] {'f', 'a', 'k', 'e'}, identifier);
-
-    AuthenticationTokenIdentifier identifier2 = new AuthenticationTokenIdentifier("user1", 1, 1000l, 2000l, "instanceid");
-    // We don't need a real serialized Token for the password
-    DelegationToken token2 = new DelegationToken(new byte[] {'f', 'a', 'k', 'e'}, identifier2);
-
-    assertNotEquals(token, token2);
-    assertNotEquals(token.hashCode(), token2.hashCode());
-
-    // We don't need a real serialized Token for the password
-    DelegationToken token3 = new DelegationToken(new byte[] {'f', 'a', 'k', 'e', '0'}, identifier);
-
-    assertNotEquals(token, token3);
-    assertNotEquals(token.hashCode(), token3.hashCode());
-    assertNotEquals(token2, token3);
-    assertNotEquals(token2.hashCode(), token3.hashCode());
-  }
-}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/c6252914/core/src/test/java/org/apache/accumulo/core/data/KeyExtentTest.java
----------------------------------------------------------------------
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 749f14c..b1ac1c5 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
@@ -37,6 +37,7 @@ import java.util.SortedMap;
 import java.util.TreeMap;
 import java.util.TreeSet;
 
+import org.apache.accumulo.core.data.impl.KeyExtent;
 import org.apache.hadoop.io.Text;
 import org.junit.Before;
 import org.junit.Test;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/c6252914/core/src/test/java/org/apache/accumulo/core/data/RangeTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/accumulo/core/data/RangeTest.java b/core/src/test/java/org/apache/accumulo/core/data/RangeTest.java
index 1f9a40d..1e5e985 100644
--- a/core/src/test/java/org/apache/accumulo/core/data/RangeTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/data/RangeTest.java
@@ -27,6 +27,7 @@ import java.util.List;
 
 import junit.framework.TestCase;
 
+import org.apache.accumulo.core.data.impl.KeyExtent;
 import org.apache.accumulo.core.data.thrift.TRange;
 import org.apache.hadoop.io.Text;
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/c6252914/core/src/test/java/org/apache/accumulo/core/file/rfile/RFileTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/accumulo/core/file/rfile/RFileTest.java b/core/src/test/java/org/apache/accumulo/core/file/rfile/RFileTest.java
index eafadc0..2e2b346 100644
--- a/core/src/test/java/org/apache/accumulo/core/file/rfile/RFileTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/file/rfile/RFileTest.java
@@ -44,10 +44,10 @@ import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.data.ArrayByteSequence;
 import org.apache.accumulo.core.data.ByteSequence;
 import org.apache.accumulo.core.data.Key;
-import org.apache.accumulo.core.data.KeyExtent;
 import org.apache.accumulo.core.data.PartialKey;
 import org.apache.accumulo.core.data.Range;
 import org.apache.accumulo.core.data.Value;
+import org.apache.accumulo.core.data.impl.KeyExtent;
 import org.apache.accumulo.core.file.FileSKVIterator;
 import org.apache.accumulo.core.file.blockfile.cache.LruBlockCache;
 import org.apache.accumulo.core.file.blockfile.impl.CachableBlockFile;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/c6252914/core/src/test/java/org/apache/accumulo/core/iterators/IteratorUtilTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/accumulo/core/iterators/IteratorUtilTest.java b/core/src/test/java/org/apache/accumulo/core/iterators/IteratorUtilTest.java
index 5ff5722..87ad392 100644
--- a/core/src/test/java/org/apache/accumulo/core/iterators/IteratorUtilTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/iterators/IteratorUtilTest.java
@@ -32,9 +32,9 @@ import org.apache.accumulo.core.conf.ConfigurationCopy;
 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.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.data.thrift.IterInfo;
 import org.apache.accumulo.core.iterators.IteratorUtil.IteratorScope;
 import org.apache.accumulo.core.iterators.system.MultiIteratorTest;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/c6252914/core/src/test/java/org/apache/accumulo/core/iterators/system/MultiIteratorTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/accumulo/core/iterators/system/MultiIteratorTest.java b/core/src/test/java/org/apache/accumulo/core/iterators/system/MultiIteratorTest.java
index c8ef180..3fbf92d 100644
--- a/core/src/test/java/org/apache/accumulo/core/iterators/system/MultiIteratorTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/iterators/system/MultiIteratorTest.java
@@ -26,9 +26,9 @@ import junit.framework.TestCase;
 
 import org.apache.accumulo.core.data.ByteSequence;
 import org.apache.accumulo.core.data.Key;
-import org.apache.accumulo.core.data.KeyExtent;
 import org.apache.accumulo.core.data.Range;
 import org.apache.accumulo.core.data.Value;
+import org.apache.accumulo.core.data.impl.KeyExtent;
 import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
 import org.apache.accumulo.core.iterators.SortedMapIterator;
 import org.apache.accumulo.core.util.LocalityGroupUtil;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/c6252914/core/src/test/java/org/apache/accumulo/core/metadata/MetadataServicerTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/accumulo/core/metadata/MetadataServicerTest.java b/core/src/test/java/org/apache/accumulo/core/metadata/MetadataServicerTest.java
index b8ef346..0e59025 100644
--- a/core/src/test/java/org/apache/accumulo/core/metadata/MetadataServicerTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/metadata/MetadataServicerTest.java
@@ -28,9 +28,9 @@ import org.apache.accumulo.core.client.Connector;
 import org.apache.accumulo.core.client.TableExistsException;
 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.mock.MockInstance;
 import org.apache.accumulo.core.client.security.tokens.PasswordToken;
-import org.apache.accumulo.core.security.Credentials;
 import org.junit.Test;
 
 public class MetadataServicerTest {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/c6252914/core/src/test/java/org/apache/accumulo/core/replication/ReplicationConfigurationUtilTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/accumulo/core/replication/ReplicationConfigurationUtilTest.java b/core/src/test/java/org/apache/accumulo/core/replication/ReplicationConfigurationUtilTest.java
index b2b91d7..c060917 100644
--- a/core/src/test/java/org/apache/accumulo/core/replication/ReplicationConfigurationUtilTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/replication/ReplicationConfigurationUtilTest.java
@@ -22,7 +22,7 @@ import java.util.Map;
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
 import org.apache.accumulo.core.conf.ConfigurationCopy;
 import org.apache.accumulo.core.conf.Property;
-import org.apache.accumulo.core.data.KeyExtent;
+import org.apache.accumulo.core.data.impl.KeyExtent;
 import org.apache.accumulo.core.metadata.MetadataTable;
 import org.apache.accumulo.core.metadata.RootTable;
 import org.apache.hadoop.io.Text;