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

[6/9] accumulo git commit: ACCUMULO-3657 ACCUMULO-3548 Clarify Accumulo API

ACCUMULO-3657 ACCUMULO-3548 Clarify Accumulo API

The following changes were made.

 * Fix API problems with IsolatedScanner
 * Deprecated public inner class in ClientSiderIteratorScanner that was not intended to be in public API
 * ACCUMULO-3488 Deprecated KeyExtent from public API
 * Deprecated use of Property in ClientConfiguration
 * deprecated ZooKeeperInstance.lookupIntanceName(ZooCache, UUID)
 * deprecated IteratorUtil.getProperty(IteratorScope)
 * deprecated class ComparableBytes
 * ACCUMULO-3724 moved DelegationToken and AuthenticationTokenIdentifier out of public API
 * deprecated getTabletLocator() in both AbstractInputFormat classes
 * deprecated getAccumuloConfiguration() in both AccumuloFileOutputFormat classes
 * deperecated and moved VisibilityConstraint
 * updated README to accurately communicate Accumulo's API.  Update APILyzer config to enforce ACCUMULO-3720
 * moved Credentials from core.security to core.client.impl


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

Branch: refs/heads/master
Commit: c62529142b9ded096a1080d82c6934c3a309690c
Parents: 18a3188
Author: Keith Turner <kt...@apache.org>
Authored: Fri Apr 10 15:01:43 2015 -0400
Committer: Keith Turner <kt...@apache.org>
Committed: Tue Apr 21 17:53:31 2015 -0400

----------------------------------------------------------------------
 README.md                                       |  30 +-
 core/pom.xml                                    |  37 +-
 core/src/main/findbugs/exclude-filter.xml       |   1 +
 .../core/client/ClientConfiguration.java        |   8 +
 .../core/client/ClientSideIteratorScanner.java  |  27 +-
 .../accumulo/core/client/IsolatedScanner.java   |   4 +-
 .../core/client/MutationsRejectedException.java |  68 +-
 .../accumulo/core/client/ZooKeeperInstance.java |  21 +-
 .../core/client/admin/ActiveCompaction.java     |  12 +-
 .../accumulo/core/client/admin/ActiveScan.java  |  12 +-
 .../client/admin/DelegationTokenConfig.java     |   4 +-
 .../core/client/admin/SecurityOperations.java   |   3 +-
 .../core/client/impl/ActiveCompactionImpl.java  |  19 +-
 .../core/client/impl/ActiveScanImpl.java        |  14 +-
 .../impl/AuthenticationTokenIdentifier.java     | 210 +++++
 .../core/client/impl/ClientContext.java         |   1 -
 .../core/client/impl/ConditionalWriterImpl.java |   2 +-
 .../accumulo/core/client/impl/Credentials.java  | 157 ++++
 .../core/client/impl/DelegationTokenImpl.java   | 144 ++++
 .../client/impl/InstanceOperationsImpl.java     |  18 +
 .../client/impl/NamespaceOperationsImpl.java    |   1 -
 .../core/client/impl/OfflineScanner.java        |   3 +-
 .../core/client/impl/RootTabletLocator.java     |   2 +-
 .../client/impl/SecurityOperationsImpl.java     |   4 +-
 .../core/client/impl/TableOperationsImpl.java   |   2 +-
 .../core/client/impl/TabletLocator.java         |   2 +-
 .../core/client/impl/TabletLocatorImpl.java     |   2 +-
 .../impl/TabletServerBatchReaderIterator.java   |   2 +-
 .../client/impl/TabletServerBatchWriter.java    |   8 +-
 .../accumulo/core/client/impl/TabletType.java   |   2 +-
 .../core/client/impl/ThriftScanner.java         |   2 +-
 .../core/client/impl/TimeoutTabletLocator.java  |   2 +-
 .../accumulo/core/client/impl/Translator.java   |   2 +-
 .../accumulo/core/client/impl/Writer.java       |   2 +-
 .../core/client/mapred/AbstractInputFormat.java |  20 +-
 .../client/mapred/AccumuloFileOutputFormat.java |   5 +-
 .../client/mapred/AccumuloOutputFormat.java     |  23 +-
 .../client/mapreduce/AbstractInputFormat.java   |  26 +-
 .../mapreduce/AccumuloFileOutputFormat.java     |   5 +-
 .../client/mapreduce/AccumuloOutputFormat.java  |  21 +-
 .../mapreduce/lib/impl/ConfiguratorBase.java    |  16 +-
 .../mapreduce/lib/impl/InputConfigurator.java   |   8 +-
 .../core/client/mock/MockConnector.java         |   2 +-
 .../accumulo/core/client/mock/MockInstance.java |   2 +-
 .../client/mock/impl/MockTabletLocator.java     |   2 +-
 .../client/security/tokens/DelegationToken.java | 134 +---
 .../accumulo/core/constraints/Constraint.java   |   2 +-
 .../core/constraints/VisibilityConstraint.java  |  93 +++
 .../accumulo/core/data/ComparableBytes.java     |   5 +
 .../apache/accumulo/core/data/KeyExtent.java    | 717 +++--------------
 .../org/apache/accumulo/core/data/TabletID.java |  29 +
 .../core/data/impl/ComparableBytes.java         |  53 ++
 .../accumulo/core/data/impl/KeyExtent.java      | 768 +++++++++++++++++++
 .../accumulo/core/data/impl/TabletIDImpl.java   | 100 +++
 .../accumulo/core/iterators/IteratorUtil.java   |  40 +-
 .../core/iterators/system/MultiIterator.java    |   2 +-
 .../core/metadata/MetadataLocationObtainer.java |   2 +-
 .../core/metadata/MetadataServicer.java         |   2 +-
 .../accumulo/core/metadata/RootTable.java       |   2 +-
 .../core/metadata/ServicerForRootTable.java     |   2 +-
 .../core/metadata/TableMetadataServicer.java    |   2 +-
 .../ReplicationConfigurationUtil.java           |   2 +-
 .../rpc/SaslClientDigestCallbackHandler.java    |   4 +-
 .../accumulo/core/rpc/SaslConnectionParams.java |   6 +-
 .../security/AuthenticationTokenIdentifier.java | 210 -----
 .../accumulo/core/security/Credentials.java     | 157 ----
 .../core/security/VisibilityConstraint.java     |  76 +-
 .../core/security/VisibilityEvaluator.java      |  10 +-
 .../core/tabletserver/log/LogEntry.java         |   2 +-
 .../org/apache/accumulo/core/util/Merge.java    |   2 +-
 .../core/client/impl/ScannerImplTest.java       |   1 -
 .../client/impl/TableOperationsImplTest.java    |   3 +-
 .../core/client/impl/TabletLocatorImplTest.java |   3 +-
 .../impl/TabletServerBatchReaderTest.java       |   1 -
 .../mapred/AccumuloFileOutputFormatTest.java    |   5 +-
 .../core/client/mapred/TokenFileTest.java       |   2 +-
 .../mapreduce/AccumuloFileOutputFormatTest.java |   5 +-
 .../core/client/mapreduce/TokenFileTest.java    |   2 +-
 .../tokens/DelegationTokenImplTest.java         |  73 ++
 .../security/tokens/DelegationTokenTest.java    |  72 --
 .../accumulo/core/data/KeyExtentTest.java       |   1 +
 .../apache/accumulo/core/data/RangeTest.java    |   1 +
 .../accumulo/core/file/rfile/RFileTest.java     |   2 +-
 .../core/iterators/IteratorUtilTest.java        |   2 +-
 .../iterators/system/MultiIteratorTest.java     |   2 +-
 .../core/metadata/MetadataServicerTest.java     |   2 +-
 .../ReplicationConfigurationUtilTest.java       |   2 +-
 .../core/rpc/SaslConnectionParamsTest.java      |  12 +-
 .../AuthenticationTokenIdentifierTest.java      |   1 +
 .../accumulo/core/security/CredentialsTest.java |   1 +
 .../core/security/VisibilityConstraintTest.java |   1 +
 .../apache/accumulo/core/util/MergeTest.java    |   2 +-
 .../simple/client/RandomBatchWriter.java        |   6 +-
 minicluster/pom.xml                             |   5 +-
 .../impl/MiniAccumuloClusterImpl.java           |   2 +-
 .../org/apache/accumulo/proxy/ProxyServer.java  |   8 +-
 .../accumulo/server/AccumuloServerContext.java  |   2 +-
 .../accumulo/server/client/BulkImporter.java    |   2 +-
 .../server/client/ClientServiceHandler.java     |   2 +-
 .../accumulo/server/client/HdfsZooInstance.java |   6 +-
 .../server/conf/ServerConfiguration.java        |   2 +-
 .../server/conf/ServerConfigurationFactory.java |   2 +-
 .../server/constraints/MetadataConstraints.java |   2 +-
 .../accumulo/server/fs/VolumeManagerImpl.java   |   2 +-
 .../apache/accumulo/server/fs/VolumeUtil.java   |   2 +-
 .../apache/accumulo/server/init/Initialize.java |   2 +-
 .../accumulo/server/master/LiveTServerSet.java  |   2 +-
 .../master/balancer/ChaoticLoadBalancer.java    |   2 +-
 .../master/balancer/DefaultLoadBalancer.java    |   2 +-
 .../server/master/balancer/GroupBalancer.java   |   2 +-
 .../master/balancer/RegexGroupBalancer.java     |   2 +-
 .../master/balancer/TableLoadBalancer.java      |   2 +-
 .../server/master/balancer/TabletBalancer.java  |   2 +-
 .../server/master/state/Assignment.java         |   2 +-
 .../server/master/state/CurrentState.java       |   2 +-
 .../accumulo/server/master/state/MergeInfo.java |   2 +-
 .../master/state/MetaDataTableScanner.java      |   2 +-
 .../master/state/TabletLocationState.java       |   2 +-
 .../server/master/state/TabletMigration.java    |   2 +-
 .../master/state/TabletStateChangeIterator.java |   2 +-
 .../rpc/SaslServerDigestCallbackHandler.java    |   2 +-
 .../TCredentialsUpdatingInvocationHandler.java  |   4 +-
 .../security/AuditedSecurityOperation.java      |   4 +-
 .../server/security/SecurityOperation.java      |   2 +-
 .../server/security/SystemCredentials.java      |   2 +-
 .../AuthenticationTokenSecretManager.java       |   6 +-
 .../security/handler/KerberosAuthenticator.java |   4 +-
 .../tabletserver/LargestFirstMemoryManager.java |   2 +-
 .../tabletserver/MemoryManagementActions.java   |   2 +-
 .../server/tabletserver/MemoryManager.java      |   2 +-
 .../server/tabletserver/TabletState.java        |   2 +-
 .../org/apache/accumulo/server/util/Admin.java  |   2 +-
 .../server/util/CheckForMetadataProblems.java   |   2 +-
 .../apache/accumulo/server/util/FileUtil.java   |   2 +-
 .../server/util/FindOfflineTablets.java         |   2 +-
 .../server/util/MasterMetadataUtil.java         |   2 +-
 .../accumulo/server/util/MetadataTableUtil.java |   4 +-
 .../util/RemoveEntriesForMissingFiles.java      |   4 +-
 .../server/util/ReplicationTableUtil.java       |   4 +-
 .../accumulo/server/util/TableDiskUsage.java    |   2 +-
 .../accumulo/server/util/TabletIterator.java    |   2 +-
 .../server/util/VerifyTabletAssignments.java    |   4 +-
 .../server/AccumuloServerContextTest.java       |   2 +-
 .../server/client/BulkImporterTest.java         |   4 +-
 .../balancer/ChaoticLoadBalancerTest.java       |   2 +-
 .../balancer/DefaultLoadBalancerTest.java       |   2 +-
 .../master/balancer/GroupBalancerTest.java      |   2 +-
 .../master/balancer/TableLoadBalancerTest.java  |   2 +-
 .../server/master/state/MergeInfoTest.java      |   2 +-
 .../master/state/TabletLocationStateTest.java   |   2 +-
 .../rpc/SaslDigestCallbackHandlerTest.java      |   2 +-
 .../server/security/SystemCredentialsTest.java  |   2 +-
 .../AuthenticationTokenSecretManagerTest.java   |   2 +-
 .../apache/accumulo/server/util/CloneTest.java  |   2 +-
 .../server/util/ReplicationTableUtilTest.java   |   4 +-
 .../server/util/TabletIteratorTest.java         |   2 +-
 .../accumulo/gc/GarbageCollectionAlgorithm.java |   2 +-
 .../accumulo/gc/GarbageCollectionTest.java      |   2 +-
 .../accumulo/gc/SimpleGarbageCollectorTest.java |   2 +-
 .../CloseWriteAheadLogReferencesTest.java       |   2 +-
 .../java/org/apache/accumulo/master/Master.java |   2 +-
 .../master/MasterClientServiceHandler.java      |   4 +-
 .../accumulo/master/TabletGroupWatcher.java     |   2 +-
 .../master/recovery/RecoveryManager.java        |   2 +-
 .../accumulo/master/state/MergeStats.java       |   2 +-
 .../accumulo/master/tableOps/BulkImport.java    |   2 +-
 .../accumulo/master/tableOps/CompactRange.java  |   2 +-
 .../accumulo/master/tableOps/CreateTable.java   |   2 +-
 .../accumulo/master/tableOps/DeleteTable.java   |   2 +-
 .../accumulo/master/tableOps/ExportTable.java   |   2 +-
 .../accumulo/master/tableOps/ImportTable.java   |   2 +-
 .../accumulo/master/tableOps/TableRangeOp.java  |   2 +-
 .../master/ReplicationOperationsImplTest.java   |   4 +-
 .../apache/accumulo/master/TestMergeState.java  |   2 +-
 .../replication/SequentialWorkAssignerTest.java |   2 +-
 .../master/replication/StatusMakerTest.java     |   2 +-
 .../replication/UnorderedWorkAssignerTest.java  |   2 +-
 .../master/state/RootTabletStateStoreTest.java  |   2 +-
 .../monitor/servlets/TServersServlet.java       |   2 +-
 .../monitor/servlets/TablesServlet.java         |   2 +-
 .../tserver/ActiveAssignmentRunnable.java       |   2 +-
 .../tserver/ConditionalMutationSet.java         |   2 +-
 .../apache/accumulo/tserver/FileManager.java    |   2 +-
 .../org/apache/accumulo/tserver/RowLocks.java   |   2 +-
 .../apache/accumulo/tserver/TabletServer.java   |   2 +-
 .../tserver/TabletServerResourceManager.java    |   2 +-
 .../accumulo/tserver/TservConstraintEnv.java    |   2 +-
 .../apache/accumulo/tserver/WriteTracker.java   |   2 +-
 .../compaction/MajorCompactionRequest.java      |   2 +-
 .../tserver/constraints/ConstraintChecker.java  |   2 +-
 .../apache/accumulo/tserver/log/DfsLogger.java  |   2 +-
 .../accumulo/tserver/log/SortedLogRecovery.java |   2 +-
 .../tserver/log/TabletServerLogger.java         |   2 +-
 .../accumulo/tserver/logger/LogFileKey.java     |   2 +-
 .../accumulo/tserver/logger/LogReader.java      |   2 +-
 .../mastermessage/SplitReportMessage.java       |   2 +-
 .../mastermessage/TabletStatusMessage.java      |   2 +-
 .../replication/AccumuloReplicaSystem.java      |   2 +-
 .../accumulo/tserver/scan/LookupTask.java       |   2 +-
 .../tserver/session/MultiScanSession.java       |   2 +-
 .../accumulo/tserver/session/ScanSession.java   |   2 +-
 .../accumulo/tserver/session/UpdateSession.java |   2 +-
 .../accumulo/tserver/tablet/CommitSession.java  |   2 +-
 .../accumulo/tserver/tablet/CompactionInfo.java |   2 +-
 .../accumulo/tserver/tablet/Compactor.java      |   2 +-
 .../tserver/tablet/DatafileManager.java         |   2 +-
 .../apache/accumulo/tserver/tablet/Tablet.java  |   2 +-
 .../tserver/tablet/TabletCommitter.java         |   2 +-
 .../LargestFirstMemoryManagerTest.java          |   2 +-
 .../accumulo/tserver/AssignmentWatcherTest.java |   2 +-
 .../tserver/CheckTabletMetadataTest.java        |   2 +-
 .../tserver/TabletResourceManagerTest.java      |   2 +-
 .../DefaultCompactionStrategyTest.java          |   2 +-
 .../SizeLimitCompactionStrategyTest.java        |   2 +-
 .../ConfigurableCompactionStrategyTest.java     |   2 +-
 .../constraints/ConstraintCheckerTest.java      |   2 +-
 .../tserver/log/SortedLogRecoveryTest.java      |   2 +-
 .../accumulo/tserver/logger/LogFileTest.java    |   2 +-
 .../replication/AccumuloReplicaSystemTest.java  |   2 +-
 .../replication/ReplicationProcessorTest.java   |   2 +-
 .../accumulo/tserver/tablet/TabletTest.java     |   2 +-
 .../commands/ActiveCompactionIterator.java      |   2 +-
 .../shell/commands/ActiveScanIterator.java      |   2 +-
 .../shell/commands/CreateTableCommand.java      |   2 +-
 .../shell/commands/GetSplitsCommand.java        |   2 +-
 .../accumulo/shell/commands/InsertCommand.java  |   6 +-
 .../accumulo/test/QueryMetadataTable.java       |   2 +-
 .../org/apache/accumulo/test/TestIngest.java    |   6 +-
 .../apache/accumulo/test/WrongTabletTest.java   |   4 +-
 .../continuous/ContinuousStatsCollector.java    |   4 +-
 .../metadata/MetadataBatchScanTest.java         |   2 +-
 .../performance/scan/CollectTabletStats.java    |   4 +-
 .../test/performance/thrift/NullTserver.java    |   2 +-
 .../randomwalk/security/AlterTablePerm.java     |   2 +-
 .../test/randomwalk/security/Authenticate.java  |   2 +-
 .../test/randomwalk/security/ChangePass.java    |   2 +-
 .../test/randomwalk/security/DropTable.java     |   2 +-
 .../test/randomwalk/security/SetAuths.java      |   2 +-
 .../randomwalk/security/WalkingSecurity.java    |   2 +-
 .../randomwalk/sequential/MapRedVerifyTool.java |   4 +-
 .../server/security/SystemCredentialsIT.java    |   2 +-
 .../test/MasterRepairsDualAssignmentIT.java     |   4 +-
 .../accumulo/test/MetaConstraintRetryIT.java    |   4 +-
 .../accumulo/test/MultiTableBatchWriterIT.java  |   2 +-
 .../org/apache/accumulo/test/NamespacesIT.java  |   6 +-
 .../apache/accumulo/test/SplitRecoveryIT.java   |   2 +-
 .../accumulo/test/TransportCachingIT.java       |   2 +-
 .../java/org/apache/accumulo/test/VolumeIT.java |   2 +-
 .../test/functional/AccumuloInputFormatIT.java  |   2 +-
 .../functional/BalanceAfterCommsFailureIT.java  |   2 +-
 .../BalanceInPresenceOfOfflineTableIT.java      |   2 +-
 .../test/functional/DynamicThreadPoolsIT.java   |   2 +-
 .../accumulo/test/functional/KerberosIT.java    |  37 +-
 .../test/functional/MasterAssignmentIT.java     |   4 +-
 .../test/functional/MetadataMaxFilesIT.java     |   2 +-
 .../accumulo/test/functional/PermissionsIT.java |   2 +-
 .../functional/SimpleBalancerFairnessIT.java    |   2 +-
 .../accumulo/test/functional/SplitIT.java       |   2 +-
 .../test/functional/SplitRecoveryIT.java        |   2 +-
 .../accumulo/test/functional/TableIT.java       |   2 +-
 .../functional/TabletStateChangeIteratorIT.java |   2 +-
 ...bageCollectorCommunicatesWithTServersIT.java |   2 +-
 ...UnusedWalDoesntCloseReplicationStatusIT.java |   2 +-
 263 files changed, 2364 insertions(+), 1658 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/c6252914/README.md
----------------------------------------------------------------------
diff --git a/README.md b/README.md
index 15be8db..633e1b5 100644
--- a/README.md
+++ b/README.md
@@ -76,15 +76,26 @@ API
 
 The public Accumulo API is composed of :
 
- * All public classes and interfaces in the org.apache.accumulo.core.client
-   package, as as well as all of its subpackages excluding those named *impl*.
- * Key, Mutation, Value, Range, Condition, and ConditionalMutation in
-   org.apache.accumulo.core.data.
- * All public classes and interfaces in the org.apache.accumulo.minicluster
-   package, as well as all of its subpackages excluding those named *impl*.
- * Anything with public or protected acccess within any Class or Interface that
-   is in the public API. This includes, but is not limited to: methods, members
-   classes, interfaces, and enums.
+All public types in the following packages and their subpackages excluding
+those named *impl*, *thrift*, or *crypto*. 
+
+   * org.apache.accumulo.core.client
+   * org.apache.accumulo.core.data
+   * org.apache.accumulo.core.security
+   * org.apache.accumulo.minicluster
+
+A type is a class, interface, or enum.  Anything with public or protected
+acccess in an API type is in the API.  This includes, but is not limited to:
+methods, members classes, interfaces, and enums.  Package-private types in
+the above packages are *not* considered public API.
+
+The following regex matches imports that are *not* Accumulo public API.  This
+regex can be used with [RegexpSingleline][13] to automatically find suspicious
+imports in a project using Accumulo. 
+
+```
+import\s+org\.apache\.accumulo\.(.*\.(impl|thrift|crypto)\..*|(?!core|minicluster).*|core\.(?!client|data|security).*)
+```
 
 The Accumulo project maintains binary compatibility across this API within a
 major release, as defined in the Java Language Specification 3rd ed. Starting
@@ -100,3 +111,4 @@ with Accumulo 1.6.2 and 1.7.0 all API changes will follow [semver 2.0][12]
 [8]: http://accumulo.apache.org/notable_features.html
 [9]: http://maven.apache.org/
 [12]: http://semver.org/spec/v2.0.0.html
+[13]: http://checkstyle.sourceforge.net/config_regexp.html

http://git-wip-us.apache.org/repos/asf/accumulo/blob/c6252914/core/pom.xml
----------------------------------------------------------------------
diff --git a/core/pom.xml b/core/pom.xml
index f4918bc..521411c 100644
--- a/core/pom.xml
+++ b/core/pom.xml
@@ -177,19 +177,46 @@
             <configuration>
               <includes>
                 <include>org[.]apache[.]accumulo[.]core[.]client[.].*</include>
-                <include>org[.]apache[.]accumulo[.]core[.]data[.](Mutation|Key|Value|Condition|ConditionalMutation|Range|ByteSequence|PartialKey|Column)([$].*)?</include>
-                <include>org[.]apache[.]accumulo[.]core[.]security[.](ColumnVisibility|Authorizations|NamespacePermission|SystemPermission|TablePermission)([$].*)?</include>
+                <include>org[.]apache[.]accumulo[.]core[.]data[.].*</include>
+                <include>org[.]apache[.]accumulo[.]core[.]security[.].*</include>
+		<!-- The following types are not advertised as in API, but are
+		     used in a such a way by API classes that we want to
+		     ensure they do not reference non public API types.
+		     Ideally no new uses of these classes would occur in the
+		     API.  These types were not placed in the allows set
+		     because no analysis is done on types in that set. -->
+                <include>org[.]apache[.]accumulo[.]core[.]util[.](Pair|ComparablePair)</include>
+                <include>org[.]apache[.]accumulo[.]core[.]iterators[.]IteratorUtil[$]IteratorScope</include>
               </includes>
               <excludes>
                 <exclude>.*Impl</exclude>
                 <exclude>.*[.]impl[.].*</exclude>
+                <exclude>.*[.]thrift[.].*</exclude>
+                <exclude>org[.]apache[.]accumulo[.]core[.]security[.]crypto[.].*</exclude>
                 <exclude>org[.]apache[.]accumulo[.]core[.]client[.]mock[.].*</exclude>
               </excludes>
               <allows>
+		<!--Allow API data types to reference thrift types, but do not
+		     analyze thrift types -->
                 <allow>org[.]apache[.]accumulo[.].*[.]thrift[.].*</allow>
-                <allow>org[.]apache[.]commons[.].*</allow>
-                <allow>org[.]apache[.]hadoop[.].*</allow>
-                <allow>org[.]apache[.]log4j[.].*</allow>
+		<!--ClientConfiguration indirectly extends
+		     AbstractConfiguration and EventSource.  These two types
+		     bring in many types from apache commons.  Added these two
+		     types instead of all of the types they reference.-->
+                <allow>org[.]apache[.]commons[.]configuration[.]Configuration</allow>
+                <allow>org[.]apache[.]commons[.]configuration[.]AbstractConfiguration</allow>
+                <allow>org[.]apache[.]commons[.]configuration[.]event[.]EventSource</allow>
+		<!--Type from hadoop used in API.  If adding a new type from
+		     Hadoop to the Accumulo API ensure its annotated as
+		     stable.-->
+                <allow>org[.]apache[.]hadoop[.]conf[.]Configuration</allow>
+                <allow>org[.]apache[.]hadoop[.]fs[.](FileSystem|Path)</allow>
+                <allow>org[.]apache[.]hadoop[.]io[.](Text|Writable|WritableComparable|WritableComparator)</allow>
+                <allow>org[.]apache[.]hadoop[.]mapred[.](JobConf|RecordReader|InputSplit|RecordWriter|Reporter)</allow>
+                <allow>org[.]apache[.]hadoop[.]mapreduce[.](Job|JobContext|RecordReader|InputSplit|TaskAttemptContext|RecordWriter|OutputCommitter|TaskInputOutputContext)</allow>
+                <allow>org[.]apache[.]hadoop[.]util[.]Progressable</allow>
+                <!--ugghhh-->
+                <allow>org[.]apache[.]log4j[.](Level|Logger)</allow>
               </allows>
             </configuration>
           </execution>

http://git-wip-us.apache.org/repos/asf/accumulo/blob/c6252914/core/src/main/findbugs/exclude-filter.xml
----------------------------------------------------------------------
diff --git a/core/src/main/findbugs/exclude-filter.xml b/core/src/main/findbugs/exclude-filter.xml
index 3cc5e30..aa854f0 100644
--- a/core/src/main/findbugs/exclude-filter.xml
+++ b/core/src/main/findbugs/exclude-filter.xml
@@ -52,6 +52,7 @@
       <Class name="org.apache.accumulo.core.client.mapred.impl.BatchInputSplit" />
       <Class name="org.apache.accumulo.core.util.AddressUtil" />
       <Class name="org.apache.accumulo.core.zookeeper.ZooUtil" />
+      <Class name="org.apache.accumulo.core.security.VisibilityConstraint" />
     </Or>
     <Or>
       <Bug code="NM" pattern="NM_SAME_SIMPLE_NAME_AS_SUPERCLASS" />

http://git-wip-us.apache.org/repos/asf/accumulo/blob/c6252914/core/src/main/java/org/apache/accumulo/core/client/ClientConfiguration.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/ClientConfiguration.java b/core/src/main/java/org/apache/accumulo/core/client/ClientConfiguration.java
index 0677365..a926d35 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/ClientConfiguration.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/ClientConfiguration.java
@@ -119,6 +119,10 @@ public class ClientConfiguration extends CompositeConfiguration {
       return defaultValue;
     }
 
+    /**
+     * @deprecated since 1.7.0 This method returns a type that is not part of the public API and not guaranteed to be stable.
+     */
+    @Deprecated
     public PropertyType getType() {
       return type;
     }
@@ -127,6 +131,10 @@ public class ClientConfiguration extends CompositeConfiguration {
       return description;
     }
 
+    /**
+     * @deprecated since 1.7.0 This method returns a type that is not part of the public API and not guaranteed to be stable.
+     */
+    @Deprecated
     public Property getAccumuloProperty() {
       return accumuloProperty;
     }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/c6252914/core/src/main/java/org/apache/accumulo/core/client/ClientSideIteratorScanner.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/ClientSideIteratorScanner.java b/core/src/main/java/org/apache/accumulo/core/client/ClientSideIteratorScanner.java
index ec13b1f..c75f54a 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/ClientSideIteratorScanner.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/ClientSideIteratorScanner.java
@@ -62,9 +62,24 @@ public class ClientSideIteratorScanner extends ScannerOptions implements Scanner
   private long readaheadThreshold = Constants.SCANNER_DEFAULT_READAHEAD_THRESHOLD;
 
   /**
+   * @deprecated since 1.7.0 was never intended for public use.  However this could have been used by anything extending this class.
+   */
+  @Deprecated
+  public class ScannerTranslator extends ScannerTranslatorImpl {
+    public ScannerTranslator(Scanner scanner) {
+      super(scanner);
+    }
+
+    @Override
+    public SortedKeyValueIterator<Key,Value> deepCopy(final IteratorEnvironment env) {
+      return new ScannerTranslator(scanner);
+    }
+  }
+
+  /**
    * A class that wraps a Scanner in a SortedKeyValueIterator so that other accumulo iterators can use it as a source.
    */
-  public class ScannerTranslator implements SortedKeyValueIterator<Key,Value> {
+  private class ScannerTranslatorImpl implements SortedKeyValueIterator<Key,Value> {
     protected Scanner scanner;
     Iterator<Entry<Key,Value>> iter;
     Entry<Key,Value> top = null;
@@ -75,7 +90,7 @@ public class ClientSideIteratorScanner extends ScannerOptions implements Scanner
      * @param scanner
      *          the scanner to iterate over
      */
-    public ScannerTranslator(final Scanner scanner) {
+    public ScannerTranslatorImpl(final Scanner scanner) {
       this.scanner = scanner;
     }
 
@@ -123,11 +138,11 @@ public class ClientSideIteratorScanner extends ScannerOptions implements Scanner
 
     @Override
     public SortedKeyValueIterator<Key,Value> deepCopy(final IteratorEnvironment env) {
-      return new ScannerTranslator(scanner);
+      return new ScannerTranslatorImpl(scanner);
     }
   }
 
-  private ScannerTranslator smi;
+  private ScannerTranslatorImpl smi;
 
   /**
    * Constructs a scanner that can execute client-side iterators.
@@ -136,7 +151,7 @@ public class ClientSideIteratorScanner extends ScannerOptions implements Scanner
    *          the source scanner
    */
   public ClientSideIteratorScanner(final Scanner scanner) {
-    smi = new ScannerTranslator(scanner);
+    smi = new ScannerTranslatorImpl(scanner);
     this.range = scanner.getRange();
     this.size = scanner.getBatchSize();
     this.timeOut = scanner.getTimeout(TimeUnit.MILLISECONDS);
@@ -147,7 +162,7 @@ public class ClientSideIteratorScanner extends ScannerOptions implements Scanner
    * Sets the source Scanner.
    */
   public void setSource(final Scanner scanner) {
-    smi = new ScannerTranslator(scanner);
+    smi = new ScannerTranslatorImpl(scanner);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/accumulo/blob/c6252914/core/src/main/java/org/apache/accumulo/core/client/IsolatedScanner.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/IsolatedScanner.java b/core/src/main/java/org/apache/accumulo/core/client/IsolatedScanner.java
index 3546cdf..e530100 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/IsolatedScanner.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/IsolatedScanner.java
@@ -170,11 +170,11 @@ public class IsolatedScanner extends ScannerOptions implements Scanner {
 
   }
 
-  interface RowBufferFactory {
+  public static interface RowBufferFactory {
     RowBuffer newBuffer();
   }
 
-  interface RowBuffer extends Iterable<Entry<Key,Value>> {
+  public static interface RowBuffer extends Iterable<Entry<Key,Value>> {
     void add(Entry<Key,Value> entry);
 
     @Override

http://git-wip-us.apache.org/repos/asf/accumulo/blob/c6252914/core/src/main/java/org/apache/accumulo/core/client/MutationsRejectedException.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/MutationsRejectedException.java b/core/src/main/java/org/apache/accumulo/core/client/MutationsRejectedException.java
index 962ef25..1a7e1ac 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/MutationsRejectedException.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/MutationsRejectedException.java
@@ -28,7 +28,11 @@ import java.util.Set;
 import org.apache.accumulo.core.client.impl.Tables;
 import org.apache.accumulo.core.client.security.SecurityErrorCode;
 import org.apache.accumulo.core.data.ConstraintViolationSummary;
-import org.apache.accumulo.core.data.KeyExtent;
+import org.apache.accumulo.core.data.TabletID;
+import org.apache.accumulo.core.data.impl.TabletIDImpl;
+
+import com.google.common.base.Function;
+import com.google.common.collect.Collections2;
 
 /**
  * Communicate the failed mutations of a BatchWriter back to the client.
@@ -38,10 +42,19 @@ public class MutationsRejectedException extends AccumuloException {
   private static final long serialVersionUID = 1L;
 
   private List<ConstraintViolationSummary> cvsl;
-  private Map<KeyExtent,Set<SecurityErrorCode>> af;
+  private Map<TabletID,Set<SecurityErrorCode>> af;
   private Collection<String> es;
   private int unknownErrors;
 
+  private static <K, V, L> Map<L, V> transformKeys(Map<K, V> map, Function<K, L> keyFunction) {
+    HashMap<L, V> ret = new HashMap<L,V>();
+    for(Entry<K, V> entry : map.entrySet()){
+      ret.put(keyFunction.apply(entry.getKey()), entry.getValue());
+    }
+
+    return ret;
+  }
+
   /**
    * @param cvsList
    *          list of constraint violations
@@ -55,12 +68,12 @@ public class MutationsRejectedException extends AccumuloException {
    * @deprecated since 1.6.0, see {@link #MutationsRejectedException(Instance, List, HashMap, Collection, int, Throwable)}
    */
   @Deprecated
-  public MutationsRejectedException(List<ConstraintViolationSummary> cvsList, HashMap<KeyExtent,Set<SecurityErrorCode>> hashMap,
+  public MutationsRejectedException(List<ConstraintViolationSummary> cvsList, HashMap<org.apache.accumulo.core.data.KeyExtent,Set<SecurityErrorCode>> hashMap,
       Collection<String> serverSideErrors, int unknownErrors, Throwable cause) {
     super("# constraint violations : " + cvsList.size() + "  security codes: " + hashMap.values() + "  # server errors " + serverSideErrors.size()
         + " # exceptions " + unknownErrors, cause);
     this.cvsl = cvsList;
-    this.af = hashMap;
+    this.af = transformKeys(hashMap, TabletIDImpl.KE_2_TID_OLD);
     this.es = serverSideErrors;
     this.unknownErrors = unknownErrors;
   }
@@ -74,8 +87,34 @@ public class MutationsRejectedException extends AccumuloException {
    *          server side errors
    * @param unknownErrors
    *          number of unknown errors
+   *
+   * @deprecated since 1.7.0 see {@link #MutationsRejectedException(Instance, List, Map, Collection, int, Throwable)}
    */
-  public MutationsRejectedException(Instance instance, List<ConstraintViolationSummary> cvsList, HashMap<KeyExtent,Set<SecurityErrorCode>> hashMap,
+  @Deprecated
+  public MutationsRejectedException(Instance instance, List<ConstraintViolationSummary> cvsList, HashMap<org.apache.accumulo.core.data.KeyExtent,Set<SecurityErrorCode>> hashMap,
+      Collection<String> serverSideErrors, int unknownErrors, Throwable cause) {
+    super("# constraint violations : " + cvsList.size() + "  security codes: " + format(transformKeys(hashMap, TabletIDImpl.KE_2_TID_OLD), instance)
+        + "  # server errors " + serverSideErrors.size() + " # exceptions " + unknownErrors, cause);
+    this.cvsl = cvsList;
+    this.af = transformKeys(hashMap, TabletIDImpl.KE_2_TID_OLD);
+    this.es = serverSideErrors;
+    this.unknownErrors = unknownErrors;
+  }
+
+  /**
+   *
+   * @param cvsList
+   *          list of constraint violations
+   * @param hashMap
+   *          authorization failures
+   * @param serverSideErrors
+   *          server side errors
+   * @param unknownErrors
+   *          number of unknown errors
+   *
+   * @since 1.7.0
+   */
+  public MutationsRejectedException(Instance instance, List<ConstraintViolationSummary> cvsList, Map<TabletID,Set<SecurityErrorCode>> hashMap,
       Collection<String> serverSideErrors, int unknownErrors, Throwable cause) {
     super("# constraint violations : " + cvsList.size() + "  security codes: " + format(hashMap, instance) + "  # server errors " + serverSideErrors.size()
         + " # exceptions " + unknownErrors, cause);
@@ -85,10 +124,10 @@ public class MutationsRejectedException extends AccumuloException {
     this.unknownErrors = unknownErrors;
   }
 
-  private static String format(HashMap<KeyExtent,Set<SecurityErrorCode>> hashMap, Instance instance) {
+  private static String format(Map<TabletID,Set<SecurityErrorCode>> hashMap, Instance instance) {
     Map<String,Set<SecurityErrorCode>> result = new HashMap<String,Set<SecurityErrorCode>>();
 
-    for (Entry<KeyExtent,Set<SecurityErrorCode>> entry : hashMap.entrySet()) {
+    for (Entry<TabletID,Set<SecurityErrorCode>> entry : hashMap.entrySet()) {
       String tableInfo = Tables.getPrintableTableInfoFromId(instance, entry.getKey().getTableId().toString());
 
       if (!result.containsKey(tableInfo)) {
@@ -113,15 +152,24 @@ public class MutationsRejectedException extends AccumuloException {
    * @deprecated since 1.5, see {@link #getAuthorizationFailuresMap()}
    */
   @Deprecated
-  public List<KeyExtent> getAuthorizationFailures() {
-    return new ArrayList<KeyExtent>(af.keySet());
+  public List<org.apache.accumulo.core.data.KeyExtent> getAuthorizationFailures() {
+    return new ArrayList<org.apache.accumulo.core.data.KeyExtent>(Collections2.transform(af.keySet(), TabletIDImpl.TID_2_KE_OLD));
   }
 
   /**
    * @return the internal mapping of keyextent mappings to SecurityErrorCode
    * @since 1.5.0
+   * @deprecated since 1.7.0 see {@link #getSecurityErrorCodes()}
+   */
+  @Deprecated
+  public Map<org.apache.accumulo.core.data.KeyExtent,Set<SecurityErrorCode>> getAuthorizationFailuresMap() {
+    return transformKeys(af, TabletIDImpl.TID_2_KE_OLD);
+  }
+
+  /**
+   * @return the internal mapping of TabletID to SecurityErrorCodes
    */
-  public Map<KeyExtent,Set<SecurityErrorCode>> getAuthorizationFailuresMap() {
+  public Map<TabletID,Set<SecurityErrorCode>> getSecurityErrorCodes(){
     return af;
   }
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/c6252914/core/src/main/java/org/apache/accumulo/core/client/ZooKeeperInstance.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/ZooKeeperInstance.java b/core/src/main/java/org/apache/accumulo/core/client/ZooKeeperInstance.java
index 38078d2..c5cb482 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/ZooKeeperInstance.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/ZooKeeperInstance.java
@@ -28,12 +28,13 @@ import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.ClientConfiguration.ClientProperty;
 import org.apache.accumulo.core.client.impl.ClientContext;
 import org.apache.accumulo.core.client.impl.ConnectorImpl;
+import org.apache.accumulo.core.client.impl.Credentials;
+import org.apache.accumulo.core.client.impl.InstanceOperationsImpl;
 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.metadata.RootTable;
-import org.apache.accumulo.core.security.Credentials;
 import org.apache.accumulo.core.util.ByteBufferUtil;
 import org.apache.accumulo.core.util.OpTimer;
 import org.apache.accumulo.core.util.TextUtil;
@@ -215,7 +216,7 @@ public class ZooKeeperInstance implements Instance {
   @Override
   public String getInstanceName() {
     if (instanceName == null)
-      instanceName = lookupInstanceName(zooCache, UUID.fromString(getInstanceID()));
+      instanceName = InstanceOperationsImpl.lookupInstanceName(zooCache, UUID.fromString(getInstanceID()));
 
     return instanceName;
   }
@@ -267,19 +268,13 @@ public class ZooKeeperInstance implements Instance {
 
   /**
    * Given a zooCache and instanceId, look up the instance name.
+   *
+   * @deprecated since 1.7.0 {@link ZooCache} is not part of the public API, but its a parameter to this method. Therefore code that uses this method is not
+   *             guaranteed to be stable. This method was deprecated to discourage its use.
    */
+  @Deprecated
   public static String lookupInstanceName(ZooCache zooCache, UUID instanceId) {
-    checkArgument(zooCache != null, "zooCache is null");
-    checkArgument(instanceId != null, "instanceId is null");
-    for (String name : zooCache.getChildren(Constants.ZROOT + Constants.ZINSTANCES)) {
-      String instanceNamePath = Constants.ZROOT + Constants.ZINSTANCES + "/" + name;
-      byte[] bytes = zooCache.get(instanceNamePath);
-      UUID iid = UUID.fromString(new String(bytes, UTF_8));
-      if (iid.equals(instanceId)) {
-        return name;
-      }
-    }
-    return null;
+    return InstanceOperationsImpl.lookupInstanceName(zooCache, instanceId);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/accumulo/blob/c6252914/core/src/main/java/org/apache/accumulo/core/client/admin/ActiveCompaction.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/admin/ActiveCompaction.java b/core/src/main/java/org/apache/accumulo/core/client/admin/ActiveCompaction.java
index b9de4c2..8f3a5aa 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/admin/ActiveCompaction.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/admin/ActiveCompaction.java
@@ -20,7 +20,7 @@ import java.util.List;
 
 import org.apache.accumulo.core.client.IteratorSetting;
 import org.apache.accumulo.core.client.TableNotFoundException;
-import org.apache.accumulo.core.data.KeyExtent;
+import org.apache.accumulo.core.data.TabletID;
 
 /**
  *
@@ -78,8 +78,16 @@ public abstract class ActiveCompaction {
 
   /**
    * @return tablet thats is compacting
+   * @deprecated since 1.7.0 use {@link #getTablet()}
    */
-  public abstract KeyExtent getExtent();
+  @Deprecated
+  public abstract org.apache.accumulo.core.data.KeyExtent getExtent();
+
+  /**
+   * @return tablet thats is compacting
+   * @since 1.7.0
+   */
+  public abstract TabletID getTablet();
 
   /**
    * @return how long the compaction has been running in milliseconds

http://git-wip-us.apache.org/repos/asf/accumulo/blob/c6252914/core/src/main/java/org/apache/accumulo/core/client/admin/ActiveScan.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/admin/ActiveScan.java b/core/src/main/java/org/apache/accumulo/core/client/admin/ActiveScan.java
index cde5d27..4621399 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/admin/ActiveScan.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/admin/ActiveScan.java
@@ -20,7 +20,7 @@ import java.util.List;
 import java.util.Map;
 
 import org.apache.accumulo.core.data.Column;
-import org.apache.accumulo.core.data.KeyExtent;
+import org.apache.accumulo.core.data.TabletID;
 import org.apache.accumulo.core.security.Authorizations;
 
 /**
@@ -65,8 +65,16 @@ public abstract class ActiveScan {
 
   /**
    * @return tablet the scan is running against, if a batch scan may be one of many or null
+   * @deprecated since 1.7.0 use {@link #getTablet()}
    */
-  public abstract KeyExtent getExtent();
+  @Deprecated
+  public abstract org.apache.accumulo.core.data.KeyExtent getExtent();
+
+  /**
+   * @return tablet the scan is running against, if a batch scan may be one of many or null
+   * @since 1.7.0
+   */
+  public abstract TabletID getTablet();
 
   /**
    * @return columns requested by the scan

http://git-wip-us.apache.org/repos/asf/accumulo/blob/c6252914/core/src/main/java/org/apache/accumulo/core/client/admin/DelegationTokenConfig.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/admin/DelegationTokenConfig.java b/core/src/main/java/org/apache/accumulo/core/client/admin/DelegationTokenConfig.java
index 2e25c3d..dab146c 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/admin/DelegationTokenConfig.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/admin/DelegationTokenConfig.java
@@ -21,10 +21,8 @@ import static com.google.common.base.Preconditions.checkNotNull;
 
 import java.util.concurrent.TimeUnit;
 
-import org.apache.accumulo.core.client.security.tokens.DelegationToken;
-
 /**
- * Configuration options for obtaining a {@link DelegationToken}
+ * Configuration options for obtaining a delegation token created by {@link SecurityOperations#getDelegationToken(DelegationTokenConfig)}
  *
  * @since 1.7.0
  */

http://git-wip-us.apache.org/repos/asf/accumulo/blob/c6252914/core/src/main/java/org/apache/accumulo/core/client/admin/SecurityOperations.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/admin/SecurityOperations.java b/core/src/main/java/org/apache/accumulo/core/client/admin/SecurityOperations.java
index 2682f95..cb916ef 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/admin/SecurityOperations.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/admin/SecurityOperations.java
@@ -352,9 +352,8 @@ public interface SecurityOperations {
   Set<String> listLocalUsers() throws AccumuloException, AccumuloSecurityException;
 
   /**
-   * Obtain a {@link DelegationToken} for use when Kerberos credentials are unavailable (e.g. YARN Jobs)
+   * Obtain a {@link DelegationToken} for use when Kerberos credentials cannot be used (e.g. YARN Jobs)
    *
-   * @return a {@link DelegationToken} for this user
    * @since 1.7.0
    */
   DelegationToken getDelegationToken(DelegationTokenConfig cfg) throws AccumuloException, AccumuloSecurityException;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/c6252914/core/src/main/java/org/apache/accumulo/core/client/impl/ActiveCompactionImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/ActiveCompactionImpl.java b/core/src/main/java/org/apache/accumulo/core/client/impl/ActiveCompactionImpl.java
index 46259d1..6dbac73 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/impl/ActiveCompactionImpl.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/ActiveCompactionImpl.java
@@ -24,7 +24,9 @@ import org.apache.accumulo.core.client.Instance;
 import org.apache.accumulo.core.client.IteratorSetting;
 import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.client.admin.ActiveCompaction;
-import org.apache.accumulo.core.data.KeyExtent;
+import org.apache.accumulo.core.data.TabletID;
+import org.apache.accumulo.core.data.impl.KeyExtent;
+import org.apache.accumulo.core.data.impl.TabletIDImpl;
 import org.apache.accumulo.core.data.thrift.IterInfo;
 
 /**
@@ -43,12 +45,21 @@ public class ActiveCompactionImpl extends ActiveCompaction {
 
   @Override
   public String getTable() throws TableNotFoundException {
-    return Tables.getTableName(instance, getExtent().getTableId().toString());
+    return Tables.getTableName(instance, new KeyExtent(tac.getExtent()).getTableId().toString());
+  }
+
+
+  @Override
+  @Deprecated
+  public org.apache.accumulo.core.data.KeyExtent getExtent() {
+    KeyExtent ke = new KeyExtent(tac.getExtent());
+    org.apache.accumulo.core.data.KeyExtent oke = new org.apache.accumulo.core.data.KeyExtent(ke.getTableId(), ke.getEndRow(), ke.getPrevEndRow());
+    return oke;
   }
 
   @Override
-  public KeyExtent getExtent() {
-    return new KeyExtent(tac.getExtent());
+  public TabletID getTablet() {
+    return new TabletIDImpl(new KeyExtent(tac.getExtent()));
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/accumulo/blob/c6252914/core/src/main/java/org/apache/accumulo/core/client/impl/ActiveScanImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/ActiveScanImpl.java b/core/src/main/java/org/apache/accumulo/core/client/impl/ActiveScanImpl.java
index 5f953e2..602a422 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/impl/ActiveScanImpl.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/ActiveScanImpl.java
@@ -26,7 +26,9 @@ import org.apache.accumulo.core.client.admin.ActiveScan;
 import org.apache.accumulo.core.client.admin.ScanState;
 import org.apache.accumulo.core.client.admin.ScanType;
 import org.apache.accumulo.core.data.Column;
-import org.apache.accumulo.core.data.KeyExtent;
+import org.apache.accumulo.core.data.TabletID;
+import org.apache.accumulo.core.data.impl.KeyExtent;
+import org.apache.accumulo.core.data.impl.TabletIDImpl;
 import org.apache.accumulo.core.data.thrift.IterInfo;
 import org.apache.accumulo.core.data.thrift.TColumn;
 import org.apache.accumulo.core.security.Authorizations;
@@ -117,8 +119,14 @@ public class ActiveScanImpl extends ActiveScan {
   }
 
   @Override
-  public KeyExtent getExtent() {
-    return extent;
+  @Deprecated
+  public org.apache.accumulo.core.data.KeyExtent getExtent() {
+    return new org.apache.accumulo.core.data.KeyExtent(extent.getTableId(), extent.getEndRow(), extent.getPrevEndRow());
+  }
+
+  @Override
+  public TabletID getTablet() {
+    return new TabletIDImpl(extent);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/accumulo/blob/c6252914/core/src/main/java/org/apache/accumulo/core/client/impl/AuthenticationTokenIdentifier.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/AuthenticationTokenIdentifier.java b/core/src/main/java/org/apache/accumulo/core/client/impl/AuthenticationTokenIdentifier.java
new file mode 100644
index 0000000..1f548bc
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/AuthenticationTokenIdentifier.java
@@ -0,0 +1,210 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.accumulo.core.client.impl;
+
+import static com.google.common.base.Preconditions.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/client/impl/ClientContext.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/ClientContext.java b/core/src/main/java/org/apache/accumulo/core/client/impl/ClientContext.java
index 9def3ab..cba7e8d 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/impl/ClientContext.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/ClientContext.java
@@ -36,7 +36,6 @@ import org.apache.accumulo.core.conf.DefaultConfiguration;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.rpc.SaslConnectionParams;
 import org.apache.accumulo.core.rpc.SslConnectionParams;
-import org.apache.accumulo.core.security.Credentials;
 import org.apache.accumulo.core.security.thrift.TCredentials;
 import org.apache.commons.configuration.Configuration;
 import org.slf4j.Logger;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/c6252914/core/src/main/java/org/apache/accumulo/core/client/impl/ConditionalWriterImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/ConditionalWriterImpl.java b/core/src/main/java/org/apache/accumulo/core/client/impl/ConditionalWriterImpl.java
index 0aef3b8..a2430d4 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/impl/ConditionalWriterImpl.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/ConditionalWriterImpl.java
@@ -53,7 +53,7 @@ import org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException;
 import org.apache.accumulo.core.data.ByteSequence;
 import org.apache.accumulo.core.data.Condition;
 import org.apache.accumulo.core.data.ConditionalMutation;
-import org.apache.accumulo.core.data.KeyExtent;
+import org.apache.accumulo.core.data.impl.KeyExtent;
 import org.apache.accumulo.core.data.thrift.TCMResult;
 import org.apache.accumulo.core.data.thrift.TCMStatus;
 import org.apache.accumulo.core.data.thrift.TCondition;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/c6252914/core/src/main/java/org/apache/accumulo/core/client/impl/Credentials.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/Credentials.java b/core/src/main/java/org/apache/accumulo/core/client/impl/Credentials.java
new file mode 100644
index 0000000..28a704a
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/Credentials.java
@@ -0,0 +1,157 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.accumulo.core.client.impl;
+
+import static 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/client/impl/DelegationTokenImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/DelegationTokenImpl.java b/core/src/main/java/org/apache/accumulo/core/client/impl/DelegationTokenImpl.java
new file mode 100644
index 0000000..119f157
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/DelegationTokenImpl.java
@@ -0,0 +1,144 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.accumulo.core.client.impl;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.Collections;
+import java.util.Set;
+
+import org.apache.accumulo.core.client.Instance;
+import org.apache.accumulo.core.client.security.tokens.DelegationToken;
+import org.apache.accumulo.core.client.security.tokens.PasswordToken;
+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;
+
+public class DelegationTokenImpl extends PasswordToken implements DelegationToken {
+  private static final Logger log = LoggerFactory.getLogger(DelegationTokenImpl.class);
+
+  public static final String SERVICE_NAME = "AccumuloDelegationToken";
+
+  private AuthenticationTokenIdentifier identifier;
+
+  public DelegationTokenImpl() {
+    super();
+  }
+
+  public DelegationTokenImpl(byte[] delegationTokenPassword, AuthenticationTokenIdentifier identifier) {
+    checkNotNull(delegationTokenPassword);
+    checkNotNull(identifier);
+    setPassword(delegationTokenPassword);
+    this.identifier = identifier;
+  }
+
+  public DelegationTokenImpl(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 DelegationTokenImpl(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 DelegationTokenImpl clone() {
+    DelegationTokenImpl copy = (DelegationTokenImpl) 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(((DelegationTokenImpl) obj).identifier);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/c6252914/core/src/main/java/org/apache/accumulo/core/client/impl/InstanceOperationsImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/InstanceOperationsImpl.java b/core/src/main/java/org/apache/accumulo/core/client/impl/InstanceOperationsImpl.java
index 7b3ee12..6383967 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/impl/InstanceOperationsImpl.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/InstanceOperationsImpl.java
@@ -23,6 +23,7 @@ import java.util.ArrayList;
 import java.util.Collections;
 import java.util.List;
 import java.util.Map;
+import java.util.UUID;
 
 import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.AccumuloException;
@@ -221,4 +222,21 @@ public class InstanceOperationsImpl implements InstanceOperations {
     }
 
   }
+
+  /**
+   * Given a zooCache and instanceId, look up the instance name.
+   */
+  public static String lookupInstanceName(ZooCache zooCache, UUID instanceId) {
+    checkArgument(zooCache != null, "zooCache is null");
+    checkArgument(instanceId != null, "instanceId is null");
+    for (String name : zooCache.getChildren(Constants.ZROOT + Constants.ZINSTANCES)) {
+      String instanceNamePath = Constants.ZROOT + Constants.ZINSTANCES + "/" + name;
+      byte[] bytes = zooCache.get(instanceNamePath);
+      UUID iid = UUID.fromString(new String(bytes, UTF_8));
+      if (iid.equals(instanceId)) {
+        return name;
+      }
+    }
+    return null;
+  }
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/c6252914/core/src/main/java/org/apache/accumulo/core/client/impl/NamespaceOperationsImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/NamespaceOperationsImpl.java b/core/src/main/java/org/apache/accumulo/core/client/impl/NamespaceOperationsImpl.java
index 264e05a..ce6bc09 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/impl/NamespaceOperationsImpl.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/NamespaceOperationsImpl.java
@@ -48,7 +48,6 @@ import org.apache.accumulo.core.iterators.IteratorUtil.IteratorScope;
 import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
 import org.apache.accumulo.core.master.thrift.FateOperation;
 import org.apache.accumulo.core.master.thrift.MasterClientService;
-import org.apache.accumulo.core.security.Credentials;
 import org.apache.accumulo.core.trace.Tracer;
 import org.apache.accumulo.core.util.OpTimer;
 import org.apache.log4j.Level;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/c6252914/core/src/main/java/org/apache/accumulo/core/client/impl/OfflineScanner.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/OfflineScanner.java b/core/src/main/java/org/apache/accumulo/core/client/impl/OfflineScanner.java
index 0263bfa..2f31319 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/impl/OfflineScanner.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/OfflineScanner.java
@@ -38,11 +38,11 @@ import org.apache.accumulo.core.conf.ConfigurationCopy;
 import org.apache.accumulo.core.conf.Property;
 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.KeyValue;
 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.FileOperations;
 import org.apache.accumulo.core.file.FileSKVIterator;
 import org.apache.accumulo.core.iterators.IteratorEnvironment;
@@ -60,7 +60,6 @@ import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.DataFileColumnFamily;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.security.ColumnVisibility;
-import org.apache.accumulo.core.security.Credentials;
 import org.apache.accumulo.core.util.CachedConfiguration;
 import org.apache.accumulo.core.util.LocalityGroupUtil;
 import org.apache.accumulo.core.util.Pair;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/c6252914/core/src/main/java/org/apache/accumulo/core/client/impl/RootTabletLocator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/RootTabletLocator.java b/core/src/main/java/org/apache/accumulo/core/client/impl/RootTabletLocator.java
index 49d5c9e..3a0c0d7 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/impl/RootTabletLocator.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/RootTabletLocator.java
@@ -27,9 +27,9 @@ 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.TabletLocatorImpl.TabletServerLockChecker;
-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.accumulo.core.metadata.RootTable;
 import org.apache.accumulo.core.util.OpTimer;
 import org.apache.accumulo.core.util.UtilWaitThread;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/c6252914/core/src/main/java/org/apache/accumulo/core/client/impl/SecurityOperationsImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/SecurityOperationsImpl.java b/core/src/main/java/org/apache/accumulo/core/client/impl/SecurityOperationsImpl.java
index bfba270..73f17a7 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/impl/SecurityOperationsImpl.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/SecurityOperationsImpl.java
@@ -35,9 +35,7 @@ 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.PasswordToken;
 import org.apache.accumulo.core.master.thrift.MasterClientService.Client;
-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.security.NamespacePermission;
 import org.apache.accumulo.core.security.SystemPermission;
 import org.apache.accumulo.core.security.TablePermission;
@@ -382,7 +380,7 @@ public class SecurityOperationsImpl implements SecurityOperations {
     AuthenticationTokenIdentifier identifier = new AuthenticationTokenIdentifier(thriftToken.getIdentifier());
 
     // Get the password out of the thrift delegation token
-    return new DelegationToken(thriftToken.getPassword(), identifier);
+    return new DelegationTokenImpl(thriftToken.getPassword(), identifier);
   }
 
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/c6252914/core/src/main/java/org/apache/accumulo/core/client/impl/TableOperationsImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/TableOperationsImpl.java b/core/src/main/java/org/apache/accumulo/core/client/impl/TableOperationsImpl.java
index d4bcac4..0e30d52 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/impl/TableOperationsImpl.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/TableOperationsImpl.java
@@ -79,9 +79,9 @@ import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.constraints.Constraint;
 import org.apache.accumulo.core.data.ByteSequence;
 import org.apache.accumulo.core.data.Key;
-import org.apache.accumulo.core.data.KeyExtent;
 import org.apache.accumulo.core.data.Range;
 import org.apache.accumulo.core.data.Value;
+import org.apache.accumulo.core.data.impl.KeyExtent;
 import org.apache.accumulo.core.iterators.IteratorUtil;
 import org.apache.accumulo.core.iterators.IteratorUtil.IteratorScope;
 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/client/impl/TabletLocator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/TabletLocator.java b/core/src/main/java/org/apache/accumulo/core/client/impl/TabletLocator.java
index 2af45d3..1fbaee8 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/impl/TabletLocator.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/TabletLocator.java
@@ -30,9 +30,9 @@ import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.Instance;
 import org.apache.accumulo.core.client.TableNotFoundException;
-import org.apache.accumulo.core.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.accumulo.core.metadata.MetadataLocationObtainer;
 import org.apache.accumulo.core.metadata.MetadataTable;
 import org.apache.accumulo.core.metadata.RootTable;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/c6252914/core/src/main/java/org/apache/accumulo/core/client/impl/TabletLocatorImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/TabletLocatorImpl.java b/core/src/main/java/org/apache/accumulo/core/client/impl/TabletLocatorImpl.java
index 99cd8aa..c28320d 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/impl/TabletLocatorImpl.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/TabletLocatorImpl.java
@@ -38,10 +38,10 @@ 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.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.impl.KeyExtent;
 import org.apache.accumulo.core.util.OpTimer;
 import org.apache.accumulo.core.util.Pair;
 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/impl/TabletServerBatchReaderIterator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchReaderIterator.java b/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchReaderIterator.java
index 61f6d15..2a0e1d8 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchReaderIterator.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchReaderIterator.java
@@ -45,9 +45,9 @@ import org.apache.accumulo.core.client.TimedOutException;
 import org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException;
 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.Range;
 import org.apache.accumulo.core.data.Value;
+import org.apache.accumulo.core.data.impl.KeyExtent;
 import org.apache.accumulo.core.data.thrift.InitialMultiScan;
 import org.apache.accumulo.core.data.thrift.MultiScanResult;
 import org.apache.accumulo.core.data.thrift.TKeyExtent;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/c6252914/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchWriter.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchWriter.java b/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchWriter.java
index 469495a..6a4d348 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchWriter.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchWriter.java
@@ -49,8 +49,10 @@ import org.apache.accumulo.core.client.impl.thrift.SecurityErrorCode;
 import org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException;
 import org.apache.accumulo.core.constraints.Violations;
 import org.apache.accumulo.core.data.ConstraintViolationSummary;
-import org.apache.accumulo.core.data.KeyExtent;
 import org.apache.accumulo.core.data.Mutation;
+import org.apache.accumulo.core.data.TabletID;
+import org.apache.accumulo.core.data.impl.KeyExtent;
+import org.apache.accumulo.core.data.impl.TabletIDImpl;
 import org.apache.accumulo.core.data.thrift.TMutation;
 import org.apache.accumulo.core.data.thrift.UpdateErrors;
 import org.apache.accumulo.core.master.state.tables.TableState;
@@ -534,7 +536,7 @@ public class TabletServerBatchWriter {
   private void checkForFailures() throws MutationsRejectedException {
     if (somethingFailed) {
       List<ConstraintViolationSummary> cvsList = violations.asList();
-      HashMap<KeyExtent,Set<org.apache.accumulo.core.client.security.SecurityErrorCode>> af = new HashMap<KeyExtent,Set<org.apache.accumulo.core.client.security.SecurityErrorCode>>();
+      HashMap<TabletID,Set<org.apache.accumulo.core.client.security.SecurityErrorCode>> af = new HashMap<TabletID,Set<org.apache.accumulo.core.client.security.SecurityErrorCode>>();
       for (Entry<KeyExtent,Set<SecurityErrorCode>> entry : authorizationFailures.entrySet()) {
         HashSet<org.apache.accumulo.core.client.security.SecurityErrorCode> codes = new HashSet<org.apache.accumulo.core.client.security.SecurityErrorCode>();
 
@@ -542,7 +544,7 @@ public class TabletServerBatchWriter {
           codes.add(org.apache.accumulo.core.client.security.SecurityErrorCode.valueOf(sce.name()));
         }
 
-        af.put(entry.getKey(), codes);
+        af.put(new TabletIDImpl(entry.getKey()), codes);
       }
 
       throw new MutationsRejectedException(context.getInstance(), cvsList, af, serverSideErrors, unknownErrors, lastUnknownError);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/c6252914/core/src/main/java/org/apache/accumulo/core/client/impl/TabletType.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/TabletType.java b/core/src/main/java/org/apache/accumulo/core/client/impl/TabletType.java
index d57bf94..ad7d43e 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/impl/TabletType.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/TabletType.java
@@ -18,7 +18,7 @@ package org.apache.accumulo.core.client.impl;
 
 import java.util.Collection;
 
-import org.apache.accumulo.core.data.KeyExtent;
+import org.apache.accumulo.core.data.impl.KeyExtent;
 
 public enum TabletType {
   ROOT, METADATA, USER;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/c6252914/core/src/main/java/org/apache/accumulo/core/client/impl/ThriftScanner.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/ThriftScanner.java b/core/src/main/java/org/apache/accumulo/core/client/impl/ThriftScanner.java
index abe91d9..4bfbfa0 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/impl/ThriftScanner.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/ThriftScanner.java
@@ -38,11 +38,11 @@ import org.apache.accumulo.core.client.impl.TabletLocator.TabletLocation;
 import org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException;
 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.KeyValue;
 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.InitialScan;
 import org.apache.accumulo.core.data.thrift.IterInfo;
 import org.apache.accumulo.core.data.thrift.ScanResult;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/c6252914/core/src/main/java/org/apache/accumulo/core/client/impl/TimeoutTabletLocator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/TimeoutTabletLocator.java b/core/src/main/java/org/apache/accumulo/core/client/impl/TimeoutTabletLocator.java
index 81c1ccc..c0cb219 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/impl/TimeoutTabletLocator.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/TimeoutTabletLocator.java
@@ -25,9 +25,9 @@ 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.TimedOutException;
-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;
 
 /**

http://git-wip-us.apache.org/repos/asf/accumulo/blob/c6252914/core/src/main/java/org/apache/accumulo/core/client/impl/Translator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/Translator.java b/core/src/main/java/org/apache/accumulo/core/client/impl/Translator.java
index 2503d95..e5141cf 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/impl/Translator.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/Translator.java
@@ -25,8 +25,8 @@ import java.util.Map.Entry;
 
 import org.apache.accumulo.core.data.Column;
 import org.apache.accumulo.core.data.ConstraintViolationSummary;
-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.TColumn;
 import org.apache.accumulo.core.data.thrift.TConstraintViolationSummary;
 import org.apache.accumulo.core.data.thrift.TKeyExtent;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/c6252914/core/src/main/java/org/apache/accumulo/core/client/impl/Writer.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/Writer.java b/core/src/main/java/org/apache/accumulo/core/client/impl/Writer.java
index 74b3811..cf2d642 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/impl/Writer.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/Writer.java
@@ -24,8 +24,8 @@ import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.client.impl.TabletLocator.TabletLocation;
 import org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException;
-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.rpc.ThriftUtil;
 import org.apache.accumulo.core.tabletserver.thrift.ConstraintViolationException;
 import org.apache.accumulo.core.tabletserver.thrift.NotServingTabletException;