You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@accumulo.apache.org by ct...@apache.org on 2019/02/11 22:01:19 UTC

[accumulo] branch master updated: Move TableId and NamespaceId to public API (#950)

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

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


The following commit(s) were added to refs/heads/master by this push:
     new 757f62b  Move TableId and NamespaceId to public API (#950)
757f62b is described below

commit 757f62b5a1e2abd6d97d803f5325fdae622167eb
Author: Christopher Tubbs <ct...@apache.org>
AuthorDate: Mon Feb 11 17:01:14 2019 -0500

    Move TableId and NamespaceId to public API (#950)
    
    * Move TableId and NamespaceId to public API package for use in SPI and
    elsewhere, as needed.
    * Retain AbstractId (in public API) for common implementation, and
    because our internal namespace/table locking mechanism and unique ID
    generation is agnostic to the specific ID type.
    * Consolidate some constants for built-in tables/namespaces.
    * Shorten/simplify/reduce API method names.
---
 .../core/client/MutationsRejectedException.java    |   4 +-
 .../core/client/mapred/AbstractInputFormat.java    |  16 +--
 .../core/client/mapreduce/AbstractInputFormat.java |  16 +--
 .../accumulo/core/clientImpl/ActiveScanImpl.java   |   3 +-
 .../accumulo/core/clientImpl/BatchWriterImpl.java  |   5 +-
 .../accumulo/core/clientImpl/ClientContext.java    |   5 +-
 .../core/clientImpl/ConditionalWriterImpl.java     |   9 +-
 .../core/clientImpl/MultiTableBatchWriterImpl.java |  11 +-
 .../apache/accumulo/core/clientImpl/Namespace.java |  53 +++-----
 .../core/clientImpl/NamespaceOperationsHelper.java |   4 +-
 .../core/clientImpl/NamespaceOperationsImpl.java   |   9 +-
 .../accumulo/core/clientImpl/Namespaces.java       |  48 ++++----
 .../accumulo/core/clientImpl/OfflineIterator.java  |   5 +-
 .../accumulo/core/clientImpl/OfflineScanner.java   |   5 +-
 .../core/clientImpl/ReplicationOperationsImpl.java |   7 +-
 .../accumulo/core/clientImpl/ScannerImpl.java      |   5 +-
 .../accumulo/core/clientImpl/ScannerIterator.java  |   3 +-
 .../core/clientImpl/SyncingTabletLocator.java      |   3 +-
 .../org/apache/accumulo/core/clientImpl/Table.java |  64 ----------
 .../apache/accumulo/core/clientImpl/TableMap.java  |  24 ++--
 .../core/clientImpl/TableOperationsImpl.java       |  71 +++++------
 .../apache/accumulo/core/clientImpl/Tables.java    |  54 +++++----
 .../accumulo/core/clientImpl/TabletLocator.java    |   7 +-
 .../core/clientImpl/TabletLocatorImpl.java         |   9 +-
 .../core/clientImpl/TabletServerBatchDeleter.java  |   5 +-
 .../core/clientImpl/TabletServerBatchReader.java   |   5 +-
 .../TabletServerBatchReaderIterator.java           |  11 +-
 .../core/clientImpl/TabletServerBatchWriter.java   |  47 +++----
 .../accumulo/core/clientImpl/ThriftScanner.java    |   9 +-
 .../core/clientImpl/TimeoutTabletLocator.java      |   3 +-
 .../apache/accumulo/core/clientImpl/Writer.java    |   5 +-
 .../apache/accumulo/core/clientImpl/bulk/Bulk.java |   8 +-
 .../accumulo/core/clientImpl/bulk/BulkImport.java  |  16 +--
 .../core/clientImpl/bulk/BulkSerialize.java        |   6 +-
 .../clientImpl/bulk/ConcurrentKeyExtentCache.java  |   6 +-
 .../core/clientImpl/bulk/LoadMappingIterator.java  |   6 +-
 .../core/clientImpl/mapred/BatchInputSplit.java    |   4 +-
 .../core/clientImpl/mapreduce/BatchInputSplit.java |   6 +-
 .../mapreduce/lib/InputConfigurator.java           |   4 +-
 .../core/{clientImpl => data}/AbstractId.java      |  46 ++-----
 .../org/apache/accumulo/core/data/NamespaceId.java |  51 ++++++++
 .../org/apache/accumulo/core/data/TableId.java     |  51 ++++++++
 .../apache/accumulo/core/dataImpl/KeyExtent.java   |  28 ++---
 .../accumulo/core/dataImpl/TabletIdImpl.java       |   2 +-
 .../accumulo/core/file/rfile/BlockIndex.java       |   3 +-
 .../accumulo/core/metadata/MetadataServicer.java   |   8 +-
 .../accumulo/core/metadata/MetadataTable.java      |   6 +-
 .../apache/accumulo/core/metadata/RootTable.java   |   7 +-
 .../core/metadata/ServicerForRootTable.java        |   4 +-
 .../core/metadata/ServicerForUserTables.java       |   4 +-
 .../core/metadata/TableMetadataServicer.java       |   8 +-
 .../core/metadata/schema/MetadataSchema.java       |  16 +--
 .../core/metadata/schema/TabletMetadata.java       |   8 +-
 .../core/metadata/schema/TabletsMetadata.java      |   9 +-
 .../core/replication/ReplicationSchema.java        |  14 +--
 .../core/replication/ReplicationTable.java         |   6 +-
 .../core/replication/ReplicationTarget.java        |  14 +--
 .../org/apache/accumulo/core/summary/Gatherer.java |   6 +-
 .../apache/accumulo/core/util/ByteBufferUtil.java  |   6 +-
 .../accumulo/core/util/LocalityGroupUtil.java      |   4 +-
 .../java/org/apache/accumulo/core/util/Merge.java  |   4 +-
 .../accumulo/core/clientImpl/ScannerImplTest.java  |   9 +-
 .../apache/accumulo/core/clientImpl/TableTest.java |  92 --------------
 .../core/clientImpl/TabletLocatorImplTest.java     |  23 ++--
 .../clientImpl/TabletServerBatchReaderTest.java    |   5 +-
 .../core/clientImpl/bulk/BulkSerializeTest.java    |   8 +-
 .../bulk/ConcurrentKeyExtentCacheTest.java         |   6 +-
 .../clientImpl/mapreduce/BatchInputSplitTest.java  |   6 +-
 .../apache/accumulo/core/data/KeyExtentTest.java   |   9 +-
 .../NamespaceIdTest.java}                          |  54 +++++----
 .../org/apache/accumulo/core/data/RangeTest.java   |  23 ++--
 .../org/apache/accumulo/core/data/TableIdTest.java | 101 +++++++++++++++
 .../accumulo/core/iterators/IteratorUtilTest.java  |  12 +-
 .../core/iterators/system/MultiIteratorTest.java   |   4 +-
 .../core/metadata/MetadataServicerTest.java        |  12 +-
 .../core/metadata/schema/TabletMetadataTest.java   |   8 +-
 .../ReplicationConfigurationUtilTest.java          |   6 +-
 .../core/replication/ReplicationSchemaTest.java    |  10 +-
 .../core/replication/ReplicationTargetTest.java    |  22 ++--
 .../org/apache/accumulo/core/util/MergeTest.java   |   4 +-
 .../hadoopImpl/mapred/AccumuloRecordReader.java    |  17 ++-
 .../hadoopImpl/mapred/BatchInputSplit.java         |   4 +-
 .../hadoopImpl/mapreduce/AccumuloRecordReader.java |  16 +--
 .../hadoopImpl/mapreduce/BatchInputSplit.java      |   6 +-
 .../mapreduce/lib/InputConfigurator.java           |   6 +-
 .../hadoopImpl/mapreduce/BatchInputSplitTest.java  |   6 +-
 .../MiniAccumuloClusterImplTest.java               |   4 +-
 .../org/apache/accumulo/proxy/ProxyServer.java     |   4 +-
 .../accumulo/server/client/BulkImporter.java       |   4 +-
 .../server/client/ClientServiceHandler.java        |  40 +++---
 .../accumulo/server/conf/NamespaceConfWatcher.java |   4 +-
 .../server/conf/NamespaceConfiguration.java        |  13 +-
 .../accumulo/server/conf/ServerConfiguration.java  |   8 +-
 .../server/conf/ServerConfigurationFactory.java    |  26 ++--
 .../accumulo/server/conf/TableConfWatcher.java     |   4 +-
 .../accumulo/server/conf/TableConfiguration.java   |  11 +-
 .../server/constraints/MetadataConstraints.java    |   2 +-
 .../accumulo/server/fs/PerTableVolumeChooser.java  |   4 +-
 .../server/fs/VolumeChooserEnvironment.java        |   8 +-
 .../apache/accumulo/server/fs/VolumeManager.java   |   4 +-
 .../accumulo/server/fs/VolumeManagerImpl.java      |   8 +-
 .../apache/accumulo/server/init/Initialize.java    |  20 +--
 .../accumulo/server/master/LiveTServerSet.java     |   8 +-
 .../master/balancer/ChaoticLoadBalancer.java       |   4 +-
 .../master/balancer/DefaultLoadBalancer.java       |  36 +++---
 .../server/master/balancer/GroupBalancer.java      |   6 +-
 .../balancer/HostRegexTableLoadBalancer.java       |  16 +--
 .../server/master/balancer/RegexGroupBalancer.java |   6 +-
 .../server/master/balancer/TableLoadBalancer.java  |  18 +--
 .../server/master/balancer/TabletBalancer.java     |   6 +-
 .../accumulo/server/master/state/CurrentState.java |   4 +-
 .../master/state/TabletStateChangeIterator.java    |  18 +--
 .../accumulo/server/problems/ProblemReport.java    |  20 +--
 .../server/problems/ProblemReportingIterator.java  |   6 +-
 .../accumulo/server/problems/ProblemReports.java   |  14 +--
 .../DistributedWorkQueueWorkAssignerHelper.java    |   4 +-
 .../server/replication/ReplicationUtil.java        |   6 +-
 .../server/security/AuditedSecurityOperation.java  |  68 +++++------
 .../server/security/SecurityOperation.java         | 135 +++++++++++----------
 .../server/security/handler/ZKAuthorizor.java      |   4 +-
 .../server/security/handler/ZKPermHandler.java     |  19 +--
 .../accumulo/server/tables/TableManager.java       |  38 +++---
 .../accumulo/server/tables/TableObserver.java      |   4 +-
 .../tabletserver/LargestFirstMemoryManager.java    |   8 +-
 .../server/util/CheckForMetadataProblems.java      |   2 +-
 .../accumulo/server/util/FindOfflineTablets.java   |   4 +-
 .../accumulo/server/util/MasterMetadataUtil.java   |   6 +-
 .../accumulo/server/util/MetadataTableUtil.java    |  36 +++---
 .../accumulo/server/util/NamespacePropUtil.java    |   8 +-
 .../accumulo/server/util/RandomizeVolumes.java     |   6 +-
 .../server/util/RemoveEntriesForMissingFiles.java  |   4 +-
 .../accumulo/server/util/ReplicationTableUtil.java |   2 +-
 .../accumulo/server/util/TableDiskUsage.java       |  44 +++----
 .../apache/accumulo/server/util/TablePropUtil.java |  12 +-
 .../server/util/VerifyTabletAssignments.java       |   4 +-
 .../accumulo/server/client/BulkImporterTest.java   |  12 +-
 .../server/conf/NamespaceConfigurationTest.java    |   7 +-
 .../conf/ServerConfigurationFactoryTest.java       |   4 +-
 .../server/conf/TableConfigurationTest.java        |   4 +-
 .../server/fs/PerTableVolumeChooserTest.java       |   4 +-
 .../server/fs/PreferredVolumeChooserTest.java      |   4 +-
 .../accumulo/server/fs/VolumeManagerImplTest.java  |   4 +-
 .../BaseHostRegexTableLoadBalancerTest.java        |  38 +++---
 .../master/balancer/ChaoticLoadBalancerTest.java   |  12 +-
 .../master/balancer/DefaultLoadBalancerTest.java   |  14 +--
 .../server/master/balancer/GroupBalancerTest.java  |   6 +-
 ...tRegexTableLoadBalancerReconfigurationTest.java |   4 +-
 .../balancer/HostRegexTableLoadBalancerTest.java   |  19 +--
 .../master/balancer/TableLoadBalancerTest.java     |  26 ++--
 .../server/master/state/MergeInfoTest.java         |  26 ++--
 .../server/problems/ProblemReportTest.java         |  13 +-
 .../problems/ProblemReportingIteratorTest.java     |   4 +-
 .../server/util/ReplicationTableUtilTest.java      |   8 +-
 .../accumulo/gc/GarbageCollectionAlgorithm.java    |  10 +-
 .../accumulo/gc/GarbageCollectionEnvironment.java  |  11 +-
 .../apache/accumulo/gc/SimpleGarbageCollector.java |   8 +-
 .../apache/accumulo/gc/GarbageCollectionTest.java  |  22 ++--
 .../apache/accumulo/master/FateServiceHandler.java |  80 ++++++------
 .../java/org/apache/accumulo/master/Master.java    |  59 ++++-----
 .../master/MasterClientServiceHandler.java         |  44 +++----
 .../apache/accumulo/master/TabletGroupWatcher.java |  14 +--
 .../master/replication/FinishedWorkUpdater.java    |   8 +-
 .../RemoveCompleteReplicationRecords.java          |  12 +-
 .../master/replication/SequentialWorkAssigner.java |  28 ++---
 .../accumulo/master/replication/StatusMaker.java   |  10 +-
 .../accumulo/master/replication/WorkMaker.java     |   6 +-
 .../apache/accumulo/master/state/MergeStats.java   |   4 +-
 .../apache/accumulo/master/state/TableStats.java   |  12 +-
 .../accumulo/master/tableOps/ChangeTableState.java |  10 +-
 .../apache/accumulo/master/tableOps/TableInfo.java |  17 ++-
 .../org/apache/accumulo/master/tableOps/Utils.java |  37 +++---
 .../master/tableOps/bulkVer1/BulkImport.java       |  20 +--
 .../tableOps/bulkVer1/CleanUpBulkImport.java       |   6 +-
 .../tableOps/bulkVer1/CompleteBulkImport.java      |   6 +-
 .../master/tableOps/bulkVer1/CopyFailed.java       |   6 +-
 .../master/tableOps/bulkVer1/LoadFiles.java        |  10 +-
 .../master/tableOps/bulkVer2/BulkImportMove.java   |   6 +-
 .../master/tableOps/bulkVer2/BulkInfo.java         |   4 +-
 .../master/tableOps/bulkVer2/LoadFiles.java        |   4 +-
 .../master/tableOps/bulkVer2/PrepBulkImport.java   |  10 +-
 .../accumulo/master/tableOps/clone/CloneInfo.java  |  12 +-
 .../accumulo/master/tableOps/clone/CloneTable.java |   8 +-
 .../master/tableOps/compact/CompactRange.java      |  18 +--
 .../master/tableOps/compact/CompactionDriver.java  |  20 +--
 .../tableOps/compact/cancel/CancelCompactions.java |  10 +-
 .../compact/cancel/FinishCancelCompaction.java     |  10 +-
 .../master/tableOps/create/CreateTable.java        |   8 +-
 .../master/tableOps/create/FinishCreateTable.java  |   2 +-
 .../master/tableOps/create/PopulateMetadata.java   |   4 +-
 .../accumulo/master/tableOps/delete/CleanUp.java   |  12 +-
 .../master/tableOps/delete/DeleteTable.java        |  10 +-
 .../master/tableOps/merge/TableRangeOp.java        |  12 +-
 .../master/tableOps/merge/TableRangeOpWait.java    |  10 +-
 .../tableOps/namespace/create/CreateNamespace.java |   4 +-
 .../namespace/create/FinishCreateNamespace.java    |   2 +-
 .../tableOps/namespace/create/NamespaceInfo.java   |   4 +-
 .../tableOps/namespace/delete/DeleteNamespace.java |   6 +-
 .../namespace/delete/NamespaceCleanUp.java         |   6 +-
 .../tableOps/namespace/rename/RenameNamespace.java |   6 +-
 .../master/tableOps/rename/RenameTable.java        |  12 +-
 .../master/tableOps/tableExport/ExportInfo.java    |   8 +-
 .../master/tableOps/tableExport/ExportTable.java   |   7 +-
 .../tableOps/tableExport/WriteExportFiles.java     |  16 +--
 .../tableOps/tableImport/CreateImportDir.java      |   2 +-
 .../tableOps/tableImport/FinishImportTable.java    |   2 +-
 .../tableImport/ImportPopulateZookeeper.java       |   8 +-
 .../master/tableOps/tableImport/ImportTable.java   |   8 +-
 .../tableOps/tableImport/ImportedTableInfo.java    |   8 +-
 .../tableOps/tableImport/MapImportFileNames.java   |   2 +-
 .../tableOps/tableImport/MoveExportedFiles.java    |   4 +-
 .../tableImport/PopulateMetadataTable.java         |   4 +-
 .../accumulo/master/util/TableValidators.java      |  20 +--
 ...DistributedWorkQueueWorkAssignerHelperTest.java |   6 +-
 .../replication/SequentialWorkAssignerTest.java    |  22 ++--
 .../replication/UnorderedWorkAssignerTest.java     |   4 +-
 .../master/state/RootTabletStateStoreTest.java     |   4 +-
 .../tableOps/bulkVer2/PrepBulkImportTest.java      |   4 +-
 .../tableOps/tableImport/ImportTableTest.java      |   4 +-
 .../java/org/apache/accumulo/monitor/Monitor.java  |   6 +-
 .../rest/problems/ProblemDetailInformation.java    |   6 +-
 .../rest/problems/ProblemSummaryInformation.java   |   6 +-
 .../monitor/rest/problems/ProblemsResource.java    |  12 +-
 .../rest/replication/ReplicationResource.java      |  14 +--
 .../monitor/rest/tables/TableInformation.java      |   8 +-
 .../monitor/rest/tables/TablesResource.java        |  17 ++-
 .../monitor/rest/tservers/CurrentOperations.java   |  10 +-
 .../rest/tservers/TabletServerResource.java        |   4 +-
 .../monitor/util/JaxbAbstractIdSerializer.java     |  10 +-
 .../org/apache/accumulo/monitor/view/WebViews.java |   4 +-
 .../org/apache/accumulo/monitor/it/WebViewsIT.java |   4 +-
 .../org/apache/accumulo/tserver/InMemoryMap.java   |   4 +-
 .../org/apache/accumulo/tserver/TabletServer.java  |  66 +++++-----
 .../apache/accumulo/tserver/logger/LogReader.java  |   4 +-
 .../replication/ReplicationServicerHandler.java    |   4 +-
 .../tserver/session/ConditionalSession.java        |   6 +-
 .../accumulo/tserver/session/MultiScanSession.java |   2 +-
 .../accumulo/tserver/session/SessionManager.java   |  12 +-
 .../tserver/session/SingleScanSession.java         |   2 +-
 .../accumulo/tserver/tablet/DatafileManager.java   |   2 +-
 .../org/apache/accumulo/tserver/tablet/Tablet.java |   4 +-
 .../accumulo/tserver/AssignmentWatcherTest.java    |   4 +-
 .../accumulo/tserver/CheckTabletMetadataTest.java  |   8 +-
 .../apache/accumulo/tserver/InMemoryMapTest.java   |  14 +--
 .../tserver/LargestFirstMemoryManagerTest.java     |  28 ++---
 .../tserver/TabletServerSyncCheckTest.java         |   4 +-
 .../compaction/DefaultCompactionStrategyTest.java  |   4 +-
 .../SizeLimitCompactionStrategyTest.java           |   4 +-
 .../strategies/BasicCompactionStrategyTest.java    |   8 +-
 .../ConfigurableCompactionStrategyTest.java        |   4 +-
 .../apache/accumulo/tserver/log/LogEntryTest.java  |   4 +-
 .../tserver/log/SortedLogRecoveryTest.java         |   9 +-
 .../accumulo/tserver/logger/LogFileTest.java       |   4 +-
 .../replication/AccumuloReplicaSystemTest.java     |  38 +++---
 .../replication/ReplicationProcessorTest.java      |   4 +-
 .../apache/accumulo/shell/commands/DUCommand.java  |   4 +-
 .../shell/commands/DeleteNamespaceCommand.java     |   4 +-
 .../shell/commands/DeleteTableCommand.java         |   2 +-
 .../accumulo/shell/commands/NamespacesCommand.java |   2 +-
 .../shell/commands/RenameNamespaceCommand.java     |  10 +-
 .../accumulo/shell/commands/TableOperation.java    |   8 +-
 .../java/org/apache/accumulo/test/CloneIT.java     |  64 +++++-----
 .../org/apache/accumulo/test/ImportExportIT.java   |   4 +-
 .../org/apache/accumulo/test/InMemoryMapIT.java    |   5 +-
 .../org/apache/accumulo/test/LargeSplitRowIT.java  |   2 +-
 .../java/org/apache/accumulo/test/LocatorIT.java   |   4 +-
 .../accumulo/test/MetaConstraintRetryIT.java       |   4 +-
 .../test/MissingWalHeaderCompletesRecoveryIT.java  |   6 +-
 .../org/apache/accumulo/test/NamespacesIT.java     |  79 ++++++------
 .../accumulo/test/RewriteTabletDirectoriesIT.java  |   4 +-
 .../java/org/apache/accumulo/test/SampleIT.java    |   4 +-
 .../org/apache/accumulo/test/ShellServerIT.java    |   4 +-
 .../org/apache/accumulo/test/SplitRecoveryIT.java  |   5 +-
 .../accumulo/test/TableConfigurationUpdateIT.java  |   6 +-
 .../org/apache/accumulo/test/VolumeChooserIT.java  |   4 +-
 .../java/org/apache/accumulo/test/VolumeIT.java    |   6 +-
 .../accumulo/test/functional/BulkFailureIT.java    |   6 +-
 .../accumulo/test/functional/BulkLoadIT.java       |   4 +-
 .../test/functional/FunctionalTestUtils.java       |   4 +-
 .../test/functional/MasterAssignmentIT.java        |   4 +-
 .../test/functional/RegexGroupBalanceIT.java       |   6 +-
 .../apache/accumulo/test/functional/ScanIdIT.java  |   4 +-
 .../apache/accumulo/test/functional/SplitIT.java   |   4 +-
 .../accumulo/test/functional/SplitRecoveryIT.java  |   4 +-
 .../test/functional/TableChangeStateIT.java        |   6 +-
 .../apache/accumulo/test/functional/TableIT.java   |   4 +-
 .../functional/TabletStateChangeIteratorIT.java    |  14 +--
 .../accumulo/test/functional/WALSunnyDayIT.java    |   2 +-
 .../CloseWriteAheadLogReferencesIT.java            |   4 +-
 .../apache/accumulo/test/master/MergeStateIT.java  |   8 +-
 .../accumulo/test/master/SuspendedTabletsIT.java   |   2 +-
 .../accumulo/test/performance/NullTserver.java     |   4 +-
 .../test/performance/scan/CollectTabletStats.java  |   6 +-
 .../accumulo/test/proxy/SimpleProxyBase.java       |   4 +-
 .../test/replication/FinishedWorkUpdaterIT.java    |  22 ++--
 ...GarbageCollectorCommunicatesWithTServersIT.java |   4 +-
 .../RemoveCompleteReplicationRecordsIT.java        |  30 ++---
 .../accumulo/test/replication/ReplicationIT.java   |  30 ++---
 .../replication/ReplicationOperationsImplIT.java   |  50 ++++----
 .../test/replication/SequentialWorkAssignerIT.java |  36 +++---
 .../test/replication/StatusCombinerMacIT.java      |   6 +-
 .../accumulo/test/replication/StatusMakerIT.java   |   6 +-
 .../test/replication/UnorderedWorkAssignerIT.java  |  12 +-
 .../UnusedWalDoesntCloseReplicationStatusIT.java   |  10 +-
 .../accumulo/test/replication/WorkMakerIT.java     |  12 +-
 304 files changed, 1993 insertions(+), 1967 deletions(-)

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 f6d1740..b259a63 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
@@ -26,9 +26,9 @@ import java.util.Set;
 
 import org.apache.accumulo.core.client.security.SecurityErrorCode;
 import org.apache.accumulo.core.clientImpl.ClientContext;
-import org.apache.accumulo.core.clientImpl.Table;
 import org.apache.accumulo.core.clientImpl.Tables;
 import org.apache.accumulo.core.data.ConstraintViolationSummary;
+import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.data.TabletId;
 
 /**
@@ -106,7 +106,7 @@ public class MutationsRejectedException extends AccumuloException {
     for (Entry<TabletId,Set<SecurityErrorCode>> entry : hashMap.entrySet()) {
       TabletId tabletId = entry.getKey();
       String tableInfo = Tables.getPrintableTableInfoFromId(context,
-          Table.ID.of(tabletId.getTableId().toString()));
+          TableId.of(tabletId.getTableId().toString()));
 
       if (!result.containsKey(tableInfo)) {
         result.put(tableInfo, new HashSet<>());
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapred/AbstractInputFormat.java b/core/src/main/java/org/apache/accumulo/core/client/mapred/AbstractInputFormat.java
index 7246d24..024536f 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mapred/AbstractInputFormat.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mapred/AbstractInputFormat.java
@@ -54,12 +54,12 @@ import org.apache.accumulo.core.clientImpl.ClientContext;
 import org.apache.accumulo.core.clientImpl.DelegationTokenImpl;
 import org.apache.accumulo.core.clientImpl.OfflineScanner;
 import org.apache.accumulo.core.clientImpl.ScannerImpl;
-import org.apache.accumulo.core.clientImpl.Table;
 import org.apache.accumulo.core.clientImpl.Tables;
 import org.apache.accumulo.core.clientImpl.TabletLocator;
 import org.apache.accumulo.core.clientImpl.mapreduce.lib.InputConfigurator;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Range;
+import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.dataImpl.KeyExtent;
 import org.apache.accumulo.core.master.state.tables.TableState;
@@ -532,10 +532,10 @@ public abstract class AbstractInputFormat<K,V> implements InputFormat<K,V> {
 
         try {
           if (isOffline) {
-            scanner = new OfflineScanner(client, Table.ID.of(baseSplit.getTableId()),
+            scanner = new OfflineScanner(client, TableId.of(baseSplit.getTableId()),
                 authorizations);
           } else {
-            scanner = new ScannerImpl(client, Table.ID.of(baseSplit.getTableId()), authorizations);
+            scanner = new ScannerImpl(client, TableId.of(baseSplit.getTableId()), authorizations);
           }
           if (isIsolated) {
             log.info("Creating isolated scanner");
@@ -608,7 +608,7 @@ public abstract class AbstractInputFormat<K,V> implements InputFormat<K,V> {
 
   }
 
-  Map<String,Map<KeyExtent,List<Range>>> binOfflineTable(JobConf job, Table.ID tableId,
+  Map<String,Map<KeyExtent,List<Range>>> binOfflineTable(JobConf job, TableId tableId,
       List<Range> ranges)
       throws TableNotFoundException, AccumuloException, AccumuloSecurityException {
     return InputConfigurator.binOffline(tableId, ranges, InputConfigurator.client(CLASS, job));
@@ -648,7 +648,7 @@ public abstract class AbstractInputFormat<K,V> implements InputFormat<K,V> {
         throw new IOException(e);
       }
 
-      Table.ID tableId;
+      TableId tableId;
       // resolve table name to id once, and use id from this point forward
       try {
         tableId = Tables.getTableId(client, tableName);
@@ -694,7 +694,7 @@ public abstract class AbstractInputFormat<K,V> implements InputFormat<K,V> {
           tl.invalidateCache();
 
           while (!tl.binRanges(client, ranges, binnedRanges).isEmpty()) {
-            String tableIdStr = tableId.canonicalID();
+            String tableIdStr = tableId.canonical();
             if (!Tables.exists(client, tableId))
               throw new TableDeletedException(tableIdStr);
             if (Tables.getTableState(client, tableId) == TableState.OFFLINE)
@@ -745,7 +745,7 @@ public abstract class AbstractInputFormat<K,V> implements InputFormat<K,V> {
             for (Range r : extentRanges.getValue()) {
               if (autoAdjust) {
                 // divide ranges into smaller ranges, based on the tablets
-                RangeInputSplit split = new RangeInputSplit(tableName, tableId.canonicalID(),
+                RangeInputSplit split = new RangeInputSplit(tableName, tableId.canonical(),
                     ke.clip(r), new String[] {location});
                 org.apache.accumulo.core.clientImpl.mapreduce.SplitUtils.updateSplit(split,
                     tableConfig, logLevel);
@@ -768,7 +768,7 @@ public abstract class AbstractInputFormat<K,V> implements InputFormat<K,V> {
 
       if (!autoAdjust)
         for (Map.Entry<Range,ArrayList<String>> entry : splitsToAdd.entrySet()) {
-          RangeInputSplit split = new RangeInputSplit(tableName, tableId.canonicalID(),
+          RangeInputSplit split = new RangeInputSplit(tableName, tableId.canonical(),
               entry.getKey(), entry.getValue().toArray(new String[0]));
           org.apache.accumulo.core.clientImpl.mapreduce.SplitUtils.updateSplit(split, tableConfig,
               logLevel);
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AbstractInputFormat.java b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AbstractInputFormat.java
index e8e2f15..9082159 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AbstractInputFormat.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AbstractInputFormat.java
@@ -54,12 +54,12 @@ import org.apache.accumulo.core.clientImpl.ClientContext;
 import org.apache.accumulo.core.clientImpl.DelegationTokenImpl;
 import org.apache.accumulo.core.clientImpl.OfflineScanner;
 import org.apache.accumulo.core.clientImpl.ScannerImpl;
-import org.apache.accumulo.core.clientImpl.Table;
 import org.apache.accumulo.core.clientImpl.Tables;
 import org.apache.accumulo.core.clientImpl.TabletLocator;
 import org.apache.accumulo.core.clientImpl.mapreduce.lib.InputConfigurator;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Range;
+import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.dataImpl.KeyExtent;
 import org.apache.accumulo.core.master.state.tables.TableState;
@@ -547,11 +547,11 @@ public abstract class AbstractInputFormat<K,V> extends InputFormat<K,V> {
 
         try {
           if (isOffline) {
-            scanner = new OfflineScanner(client, Table.ID.of(split.getTableId()), authorizations);
+            scanner = new OfflineScanner(client, TableId.of(split.getTableId()), authorizations);
           } else {
             // Not using public API to create scanner so that we can use table ID
             // Table ID is used in case of renames during M/R job
-            scanner = new ScannerImpl(client, Table.ID.of(split.getTableId()), authorizations);
+            scanner = new ScannerImpl(client, TableId.of(split.getTableId()), authorizations);
           }
           if (isIsolated) {
             log.info("Creating isolated scanner");
@@ -642,7 +642,7 @@ public abstract class AbstractInputFormat<K,V> extends InputFormat<K,V> {
     }
   }
 
-  Map<String,Map<KeyExtent,List<Range>>> binOfflineTable(JobContext job, Table.ID tableId,
+  Map<String,Map<KeyExtent,List<Range>>> binOfflineTable(JobContext job, TableId tableId,
       List<Range> ranges)
       throws TableNotFoundException, AccumuloException, AccumuloSecurityException {
     return InputConfigurator.binOffline(tableId, ranges,
@@ -680,7 +680,7 @@ public abstract class AbstractInputFormat<K,V> extends InputFormat<K,V> {
         throw new IOException(e);
       }
 
-      Table.ID tableId;
+      TableId tableId;
       // resolve table name to id once, and use id from this point forward
       try {
         tableId = Tables.getTableId(client, tableName);
@@ -726,7 +726,7 @@ public abstract class AbstractInputFormat<K,V> extends InputFormat<K,V> {
           tl.invalidateCache();
 
           while (!tl.binRanges(client, ranges, binnedRanges).isEmpty()) {
-            String tableIdStr = tableId.canonicalID();
+            String tableIdStr = tableId.canonical();
             if (!Tables.exists(client, tableId))
               throw new TableDeletedException(tableIdStr);
             if (Tables.getTableState(client, tableId) == TableState.OFFLINE)
@@ -777,7 +777,7 @@ public abstract class AbstractInputFormat<K,V> extends InputFormat<K,V> {
             for (Range r : extentRanges.getValue()) {
               if (autoAdjust) {
                 // divide ranges into smaller ranges, based on the tablets
-                RangeInputSplit split = new RangeInputSplit(tableName, tableId.canonicalID(),
+                RangeInputSplit split = new RangeInputSplit(tableName, tableId.canonical(),
                     ke.clip(r), new String[] {location});
                 org.apache.accumulo.core.clientImpl.mapreduce.SplitUtils.updateSplit(split,
                     tableConfig, logLevel);
@@ -800,7 +800,7 @@ public abstract class AbstractInputFormat<K,V> extends InputFormat<K,V> {
 
       if (!autoAdjust)
         for (Map.Entry<Range,ArrayList<String>> entry : splitsToAdd.entrySet()) {
-          RangeInputSplit split = new RangeInputSplit(tableName, tableId.canonicalID(),
+          RangeInputSplit split = new RangeInputSplit(tableName, tableId.canonical(),
               entry.getKey(), entry.getValue().toArray(new String[0]));
           org.apache.accumulo.core.clientImpl.mapreduce.SplitUtils.updateSplit(split, tableConfig,
               logLevel);
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/ActiveScanImpl.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/ActiveScanImpl.java
index af08848..98c6f3f 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/ActiveScanImpl.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/ActiveScanImpl.java
@@ -25,6 +25,7 @@ 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.TableId;
 import org.apache.accumulo.core.data.TabletId;
 import org.apache.accumulo.core.dataImpl.KeyExtent;
 import org.apache.accumulo.core.dataImpl.TabletIdImpl;
@@ -61,7 +62,7 @@ public class ActiveScanImpl extends ActiveScan {
     this.user = activeScan.user;
     this.age = activeScan.age;
     this.idle = activeScan.idleTime;
-    this.tableName = Tables.getTableName(context, Table.ID.of(activeScan.tableId));
+    this.tableName = Tables.getTableName(context, TableId.of(activeScan.tableId));
     this.type = ScanType.valueOf(activeScan.getType().name());
     this.state = ScanState.valueOf(activeScan.state.name());
     this.extent = new KeyExtent(activeScan.extent);
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/BatchWriterImpl.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/BatchWriterImpl.java
index a6c91a6..65ccd60 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/BatchWriterImpl.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/BatchWriterImpl.java
@@ -22,13 +22,14 @@ import org.apache.accumulo.core.client.BatchWriter;
 import org.apache.accumulo.core.client.BatchWriterConfig;
 import org.apache.accumulo.core.client.MutationsRejectedException;
 import org.apache.accumulo.core.data.Mutation;
+import org.apache.accumulo.core.data.TableId;
 
 public class BatchWriterImpl implements BatchWriter {
 
-  private final Table.ID tableId;
+  private final TableId tableId;
   private final TabletServerBatchWriter bw;
 
-  public BatchWriterImpl(ClientContext context, Table.ID tableId, BatchWriterConfig config) {
+  public BatchWriterImpl(ClientContext context, TableId tableId, BatchWriterConfig config) {
     checkArgument(context != null, "context is null");
     checkArgument(tableId != null, "tableId is null");
     if (config == null)
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/ClientContext.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/ClientContext.java
index 6294e6c..7451d1c 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/ClientContext.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/ClientContext.java
@@ -52,6 +52,7 @@ import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
 import org.apache.accumulo.core.conf.ClientProperty;
 import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.master.state.tables.TableState;
 import org.apache.accumulo.core.metadata.RootTable;
 import org.apache.accumulo.core.rpc.SaslConnectionParams;
@@ -453,8 +454,8 @@ public class ClientContext implements AccumuloClient {
     return zooCache;
   }
 
-  Table.ID getTableId(String tableName) throws TableNotFoundException {
-    Table.ID tableId = Tables.getTableId(this, tableName);
+  TableId getTableId(String tableName) throws TableNotFoundException {
+    TableId tableId = Tables.getTableId(this, tableName);
     if (Tables.getTableState(this, tableId) == TableState.OFFLINE)
       throw new TableOfflineException(Tables.getTableOfflineMsg(this, tableId));
     return tableId;
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/ConditionalWriterImpl.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/ConditionalWriterImpl.java
index 71a6e11..88cad63 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/ConditionalWriterImpl.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/ConditionalWriterImpl.java
@@ -54,6 +54,7 @@ import org.apache.accumulo.core.clientImpl.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.TableId;
 import org.apache.accumulo.core.dataImpl.KeyExtent;
 import org.apache.accumulo.core.dataImpl.thrift.TCMResult;
 import org.apache.accumulo.core.dataImpl.thrift.TCMStatus;
@@ -113,7 +114,7 @@ class ConditionalWriterImpl implements ConditionalWriter {
   private Map<Text,Boolean> cache = Collections.synchronizedMap(new LRUMap(1000));
   private final ClientContext context;
   private TabletLocator locator;
-  private final Table.ID tableId;
+  private final TableId tableId;
   private long timeout;
   private final Durability durability;
   private final String classLoaderContext;
@@ -298,7 +299,7 @@ class ConditionalWriterImpl implements ConditionalWriter {
 
       if (failures.size() == mutations.size())
         if (!Tables.exists(context, tableId))
-          throw new TableDeletedException(tableId.canonicalID());
+          throw new TableDeletedException(tableId.canonical());
         else if (Tables.getTableState(context, tableId) == TableState.OFFLINE)
           throw new TableOfflineException(Tables.getTableOfflineMsg(context, tableId));
 
@@ -385,7 +386,7 @@ class ConditionalWriterImpl implements ConditionalWriter {
     }
   }
 
-  ConditionalWriterImpl(ClientContext context, Table.ID tableId, ConditionalWriterConfig config) {
+  ConditionalWriterImpl(ClientContext context, TableId tableId, ConditionalWriterConfig config) {
     this.context = context;
     this.auths = config.getAuthorizations();
     this.ve = new VisibilityEvaluator(config.getAuthorizations());
@@ -512,7 +513,7 @@ class ConditionalWriterImpl implements ConditionalWriter {
     }
 
     TConditionalSession tcs = client.startConditionalUpdate(tinfo, context.rpcCreds(),
-        ByteBufferUtil.toByteBuffers(auths.getAuthorizations()), tableId.canonicalID(),
+        ByteBufferUtil.toByteBuffers(auths.getAuthorizations()), tableId.canonical(),
         DurabilityImpl.toThrift(durability), this.classLoaderContext);
 
     synchronized (cachedSessionIDs) {
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/MultiTableBatchWriterImpl.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/MultiTableBatchWriterImpl.java
index f2e972c..0f5b973 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/MultiTableBatchWriterImpl.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/MultiTableBatchWriterImpl.java
@@ -28,6 +28,7 @@ import org.apache.accumulo.core.client.MutationsRejectedException;
 import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.client.TableOfflineException;
 import org.apache.accumulo.core.data.Mutation;
+import org.apache.accumulo.core.data.TableId;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -40,9 +41,9 @@ public class MultiTableBatchWriterImpl implements MultiTableBatchWriter {
 
   private class TableBatchWriter implements BatchWriter {
 
-    private Table.ID tableId;
+    private TableId tableId;
 
-    TableBatchWriter(Table.ID tableId) {
+    TableBatchWriter(TableId tableId) {
       this.tableId = tableId;
     }
 
@@ -72,7 +73,7 @@ public class MultiTableBatchWriterImpl implements MultiTableBatchWriter {
   }
 
   private TabletServerBatchWriter bw;
-  private ConcurrentHashMap<Table.ID,BatchWriter> tableWriters;
+  private ConcurrentHashMap<TableId,BatchWriter> tableWriters;
   private final ClientContext context;
 
   public MultiTableBatchWriterImpl(ClientContext context, BatchWriterConfig config) {
@@ -119,7 +120,7 @@ public class MultiTableBatchWriterImpl implements MultiTableBatchWriter {
    *          The name of the table which to find the ID for
    * @return The table ID, or null if the table name doesn't exist
    */
-  private Table.ID getId(String tableName) throws TableNotFoundException {
+  private TableId getId(String tableName) throws TableNotFoundException {
     try {
       return Tables.getTableId(context, tableName);
     } catch (UncheckedExecutionException e) {
@@ -143,7 +144,7 @@ public class MultiTableBatchWriterImpl implements MultiTableBatchWriter {
   public BatchWriter getBatchWriter(String tableName) throws TableNotFoundException {
     checkArgument(tableName != null, "tableName is null");
 
-    Table.ID tableId = getId(tableName);
+    TableId tableId = getId(tableName);
 
     BatchWriter tbw = tableWriters.get(tableId);
     if (tbw == null) {
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/Namespace.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/Namespace.java
index 52630d8..23c51a0 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/Namespace.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/Namespace.java
@@ -16,50 +16,29 @@
  */
 package org.apache.accumulo.core.clientImpl;
 
-import java.util.concurrent.ExecutionException;
+import java.util.Objects;
 
-import com.google.common.cache.Cache;
-import com.google.common.cache.CacheBuilder;
+import org.apache.accumulo.core.data.NamespaceId;
 
 public class Namespace {
-  public static final String ACCUMULO = "accumulo";
-  public static final String DEFAULT = "";
+
+  public static final Namespace DEFAULT = new Namespace("", NamespaceId.of("+default"));
+  public static final Namespace ACCUMULO = new Namespace("accumulo", NamespaceId.of("+accumulo"));
   public static final String SEPARATOR = ".";
 
-  /**
-   * Object representing an internal Namespace ID. This class was created to help with type safety.
-   * For help obtaining the value of a namespace ID from Zookeeper, see
-   * {@link Namespaces#getNamespaceId(ClientContext, String)}
-   *
-   * Uses an internal cache and private constructor for storing a WeakReference of every
-   * Namespace.ID. Therefore, a Namespace.ID can't be instantiated outside this class and is
-   * accessed by calling Namespace.ID.{@link #of(String)}.
-   */
-  public static class ID extends AbstractId {
-    private static final long serialVersionUID = 8931104141709170293L;
-    static final Cache<String,ID> cache = CacheBuilder.newBuilder().weakValues().build();
+  private final String name;
+  private final NamespaceId id;
 
-    public static final ID ACCUMULO = of("+accumulo");
-    public static final ID DEFAULT = of("+default");
+  public Namespace(String name, NamespaceId id) {
+    this.name = Objects.requireNonNull(name);
+    this.id = Objects.requireNonNull(id);
+  }
 
-    private ID(String canonical) {
-      super(canonical);
-    }
+  public String name() {
+    return name;
+  }
 
-    /**
-     * Get a Namespace.ID object for the provided canonical string.
-     *
-     * @param canonical
-     *          Namespace ID string
-     * @return Namespace.ID object
-     */
-    public static Namespace.ID of(final String canonical) {
-      try {
-        return cache.get(canonical, () -> new Namespace.ID(canonical));
-      } catch (ExecutionException e) {
-        throw new AssertionError(
-            "This should never happen: ID constructor should never return null.");
-      }
-    }
+  public NamespaceId id() {
+    return id;
   }
 }
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/NamespaceOperationsHelper.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/NamespaceOperationsHelper.java
index 9160a7a..9ee008f 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/NamespaceOperationsHelper.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/NamespaceOperationsHelper.java
@@ -35,12 +35,12 @@ public abstract class NamespaceOperationsHelper implements NamespaceOperations {
 
   @Override
   public String systemNamespace() {
-    return Namespace.ACCUMULO;
+    return Namespace.ACCUMULO.name();
   }
 
   @Override
   public String defaultNamespace() {
-    return Namespace.DEFAULT;
+    return Namespace.DEFAULT.name();
   }
 
   @Override
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/NamespaceOperationsImpl.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/NamespaceOperationsImpl.java
index 041a960..838b362 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/NamespaceOperationsImpl.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/NamespaceOperationsImpl.java
@@ -46,6 +46,7 @@ import org.apache.accumulo.core.clientImpl.thrift.SecurityErrorCode;
 import org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException;
 import org.apache.accumulo.core.clientImpl.thrift.ThriftTableOperationException;
 import org.apache.accumulo.core.constraints.Constraint;
+import org.apache.accumulo.core.data.NamespaceId;
 import org.apache.accumulo.core.iterators.IteratorUtil.IteratorScope;
 import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
 import org.apache.accumulo.core.master.thrift.FateOperation;
@@ -131,9 +132,9 @@ public class NamespaceOperationsImpl extends NamespaceOperationsHelper {
   public void delete(String namespace) throws AccumuloException, AccumuloSecurityException,
       NamespaceNotFoundException, NamespaceNotEmptyException {
     checkArgument(namespace != null, "namespace is null");
-    Namespace.ID namespaceId = Namespaces.getNamespaceId(context, namespace);
+    NamespaceId namespaceId = Namespaces.getNamespaceId(context, namespace);
 
-    if (namespaceId.equals(Namespace.ID.ACCUMULO) || namespaceId.equals(Namespace.ID.DEFAULT)) {
+    if (namespaceId.equals(Namespace.ACCUMULO.id()) || namespaceId.equals(Namespace.DEFAULT.id())) {
       Credentials credentials = context.getCredentials();
       log.debug("{} attempted to delete the {} namespace", credentials.getPrincipal(), namespaceId);
       throw new AccumuloSecurityException(credentials.getPrincipal(),
@@ -141,7 +142,7 @@ public class NamespaceOperationsImpl extends NamespaceOperationsHelper {
     }
 
     if (Namespaces.getTableIds(context, namespaceId).size() > 0) {
-      throw new NamespaceNotEmptyException(namespaceId.canonicalID(), namespace, null);
+      throw new NamespaceNotEmptyException(namespaceId.canonical(), namespace, null);
     }
 
     List<ByteBuffer> args = Arrays.asList(ByteBuffer.wrap(namespace.getBytes(UTF_8)));
@@ -216,7 +217,7 @@ public class NamespaceOperationsImpl extends NamespaceOperationsHelper {
   @Override
   public Map<String,String> namespaceIdMap() {
     return Namespaces.getNameToIdMap(context).entrySet().stream()
-        .collect(Collectors.toMap(Map.Entry::getKey, e -> e.getValue().canonicalID(), (v1, v2) -> {
+        .collect(Collectors.toMap(Map.Entry::getKey, e -> e.getValue().canonical(), (v1, v2) -> {
           throw new RuntimeException(String.format("Duplicate key for values %s and %s", v1, v2));
         }, TreeMap::new));
   }
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/Namespaces.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/Namespaces.java
index 4ca7d20..a8c56f4 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/Namespaces.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/Namespaces.java
@@ -28,6 +28,8 @@ import java.util.function.BiConsumer;
 
 import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.NamespaceNotFoundException;
+import org.apache.accumulo.core.data.NamespaceId;
+import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.util.Validator;
 import org.apache.accumulo.fate.zookeeper.ZooCache;
 import org.slf4j.Logger;
@@ -55,7 +57,7 @@ public class Namespaces {
   public static final Validator<String> NOT_DEFAULT = new Validator<String>() {
     @Override
     public boolean test(String namespace) {
-      return !Namespace.DEFAULT.equals(namespace);
+      return !Namespace.DEFAULT.name().equals(namespace);
     }
 
     @Override
@@ -67,32 +69,32 @@ public class Namespaces {
   public static final Validator<String> NOT_ACCUMULO = new Validator<String>() {
     @Override
     public boolean test(String namespace) {
-      return !Namespace.ACCUMULO.equals(namespace);
+      return !Namespace.ACCUMULO.name().equals(namespace);
     }
 
     @Override
     public String invalidMessage(String namespace) {
-      return "Namespace cannot be the reserved namespace, " + Namespace.ACCUMULO;
+      return "Namespace cannot be the reserved namespace, " + Namespace.ACCUMULO.name();
     }
   };
 
-  public static boolean exists(ClientContext context, Namespace.ID namespaceId) {
+  public static boolean exists(ClientContext context, NamespaceId namespaceId) {
     ZooCache zc = context.getZooCache();
     List<String> namespaceIds = zc.getChildren(context.getZooKeeperRoot() + Constants.ZNAMESPACES);
-    return namespaceIds.contains(namespaceId.canonicalID());
+    return namespaceIds.contains(namespaceId.canonical());
   }
 
-  public static List<Table.ID> getTableIds(ClientContext context, Namespace.ID namespaceId)
+  public static List<TableId> getTableIds(ClientContext context, NamespaceId namespaceId)
       throws NamespaceNotFoundException {
     String namespace = getNamespaceName(context, namespaceId);
-    List<Table.ID> tableIds = new LinkedList<>();
-    for (Entry<String,Table.ID> nameToId : Tables.getNameToIdMap(context).entrySet())
+    List<TableId> tableIds = new LinkedList<>();
+    for (Entry<String,TableId> nameToId : Tables.getNameToIdMap(context).entrySet())
       if (namespace.equals(Tables.qualify(nameToId.getKey()).getFirst()))
         tableIds.add(nameToId.getValue());
     return tableIds;
   }
 
-  public static List<String> getTableNames(ClientContext context, Namespace.ID namespaceId)
+  public static List<String> getTableNames(ClientContext context, NamespaceId namespaceId)
       throws NamespaceNotFoundException {
     String namespace = getNamespaceName(context, namespaceId);
     List<String> names = new LinkedList<>();
@@ -122,30 +124,30 @@ public class Namespaces {
   /**
    * Return sorted map with key = ID, value = namespaceName
    */
-  public static SortedMap<Namespace.ID,String> getIdToNameMap(ClientContext context) {
-    SortedMap<Namespace.ID,String> idMap = new TreeMap<>();
-    getAllNamespaces(context, (id, name) -> idMap.put(Namespace.ID.of(id), name));
+  public static SortedMap<NamespaceId,String> getIdToNameMap(ClientContext context) {
+    SortedMap<NamespaceId,String> idMap = new TreeMap<>();
+    getAllNamespaces(context, (id, name) -> idMap.put(NamespaceId.of(id), name));
     return idMap;
   }
 
   /**
    * Return sorted map with key = namespaceName, value = ID
    */
-  public static SortedMap<String,Namespace.ID> getNameToIdMap(ClientContext context) {
-    SortedMap<String,Namespace.ID> nameMap = new TreeMap<>();
-    getAllNamespaces(context, (id, name) -> nameMap.put(name, Namespace.ID.of(id)));
+  public static SortedMap<String,NamespaceId> getNameToIdMap(ClientContext context) {
+    SortedMap<String,NamespaceId> nameMap = new TreeMap<>();
+    getAllNamespaces(context, (id, name) -> nameMap.put(name, NamespaceId.of(id)));
     return nameMap;
   }
 
   /**
    * Look for namespace ID in ZK. Throw NamespaceNotFoundException if not found.
    */
-  public static Namespace.ID getNamespaceId(ClientContext context, String namespaceName)
+  public static NamespaceId getNamespaceId(ClientContext context, String namespaceName)
       throws NamespaceNotFoundException {
-    final ArrayList<Namespace.ID> singleId = new ArrayList<>(1);
+    final ArrayList<NamespaceId> singleId = new ArrayList<>(1);
     getAllNamespaces(context, (id, name) -> {
       if (name.equals(namespaceName))
-        singleId.add(Namespace.ID.of(id));
+        singleId.add(NamespaceId.of(id));
     });
     if (singleId.isEmpty())
       throw new NamespaceNotFoundException(null, namespaceName,
@@ -156,8 +158,8 @@ public class Namespaces {
   /**
    * Look for namespace ID in ZK. Fail quietly by logging and returning null.
    */
-  public static Namespace.ID lookupNamespaceId(ClientContext context, String namespaceName) {
-    Namespace.ID id = null;
+  public static NamespaceId lookupNamespaceId(ClientContext context, String namespaceName) {
+    NamespaceId id = null;
     try {
       id = getNamespaceId(context, namespaceName);
     } catch (NamespaceNotFoundException e) {
@@ -177,16 +179,16 @@ public class Namespaces {
   /**
    * Look for namespace name in ZK. Throw NamespaceNotFoundException if not found.
    */
-  public static String getNamespaceName(ClientContext context, Namespace.ID namespaceId)
+  public static String getNamespaceName(ClientContext context, NamespaceId namespaceId)
       throws NamespaceNotFoundException {
     String name;
     ZooCache zc = context.getZooCache();
     byte[] path = zc.get(context.getZooKeeperRoot() + Constants.ZNAMESPACES + "/"
-        + namespaceId.canonicalID() + Constants.ZNAMESPACE_NAME);
+        + namespaceId.canonical() + Constants.ZNAMESPACE_NAME);
     if (path != null)
       name = new String(path, UTF_8);
     else
-      throw new NamespaceNotFoundException(namespaceId.canonicalID(), null,
+      throw new NamespaceNotFoundException(namespaceId.canonical(), null,
           "getNamespaceName() failed to find namespace");
     return name;
   }
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/OfflineIterator.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/OfflineIterator.java
index 1f83f8e..afa987c 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/OfflineIterator.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/OfflineIterator.java
@@ -40,6 +40,7 @@ import org.apache.accumulo.core.data.Key;
 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.TableId;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.dataImpl.KeyExtent;
 import org.apache.accumulo.core.file.FileOperations;
@@ -146,7 +147,7 @@ class OfflineIterator implements Iterator<Entry<Key,Value>> {
   private SortedKeyValueIterator<Key,Value> iter;
   private Range range;
   private KeyExtent currentExtent;
-  private Table.ID tableId;
+  private TableId tableId;
   private Authorizations authorizations;
   private ClientContext context;
   private ScannerOptions options;
@@ -164,7 +165,7 @@ class OfflineIterator implements Iterator<Entry<Key,Value>> {
           this.options.fetchedColumns.last());
     }
 
-    this.tableId = Table.ID.of(table.toString());
+    this.tableId = TableId.of(table.toString());
     this.authorizations = authorizations;
     this.readers = new ArrayList<>();
 
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/OfflineScanner.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/OfflineScanner.java
index dd4e5bc..7110ca4 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/OfflineScanner.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/OfflineScanner.java
@@ -25,6 +25,7 @@ import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.Scanner;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Range;
+import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.hadoop.io.Text;
@@ -38,12 +39,12 @@ public class OfflineScanner extends ScannerOptions implements Scanner {
   private Authorizations authorizations;
   private Text tableId;
 
-  public OfflineScanner(ClientContext context, Table.ID tableId, Authorizations authorizations) {
+  public OfflineScanner(ClientContext context, TableId tableId, Authorizations authorizations) {
     checkArgument(context != null, "context is null");
     checkArgument(tableId != null, "tableId is null");
     checkArgument(authorizations != null, "authorizations is null");
     this.context = context;
-    this.tableId = new Text(tableId.getUtf8());
+    this.tableId = new Text(tableId.canonical());
     this.range = new Range((Key) null, (Key) null);
     this.authorizations = authorizations;
     this.batchSize = Constants.SCAN_BATCH_SIZE;
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/ReplicationOperationsImpl.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/ReplicationOperationsImpl.java
index f098990..5a73784 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/ReplicationOperationsImpl.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/ReplicationOperationsImpl.java
@@ -34,6 +34,7 @@ import org.apache.accumulo.core.client.admin.ReplicationOperations;
 import org.apache.accumulo.core.client.admin.TableOperations;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.data.Key;
+import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.metadata.MetadataTable;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema;
@@ -116,7 +117,7 @@ public class ReplicationOperationsImpl implements ReplicationOperations {
         client -> client.drainReplicationTable(tinfo, rpcCreds, tableName, wals));
   }
 
-  protected Table.ID getTableId(AccumuloClient client, String tableName)
+  protected TableId getTableId(AccumuloClient client, String tableName)
       throws TableNotFoundException {
     TableOperations tops = client.tableOperations();
 
@@ -132,7 +133,7 @@ public class ReplicationOperationsImpl implements ReplicationOperations {
       }
     }
 
-    return Table.ID.of(tableId);
+    return TableId.of(tableId);
   }
 
   @Override
@@ -141,7 +142,7 @@ public class ReplicationOperationsImpl implements ReplicationOperations {
 
     log.debug("Collecting referenced files for replication of table {}", tableName);
 
-    Table.ID tableId = getTableId(context, tableName);
+    TableId tableId = getTableId(context, tableName);
 
     log.debug("Found id of {} for name {}", tableId, tableName);
 
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/ScannerImpl.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/ScannerImpl.java
index 4b865f3..edb2435 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/ScannerImpl.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/ScannerImpl.java
@@ -28,6 +28,7 @@ import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.Scanner;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Range;
+import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.security.Authorizations;
 
@@ -49,7 +50,7 @@ public class ScannerImpl extends ScannerOptions implements Scanner {
 
   private final ClientContext context;
   private Authorizations authorizations;
-  private Table.ID tableId;
+  private TableId tableId;
 
   private int size;
 
@@ -105,7 +106,7 @@ public class ScannerImpl extends ScannerOptions implements Scanner {
       throw new IllegalArgumentException("Scanner is closed");
   }
 
-  public ScannerImpl(ClientContext context, Table.ID tableId, Authorizations authorizations) {
+  public ScannerImpl(ClientContext context, TableId tableId, Authorizations authorizations) {
     checkArgument(context != null, "context is null");
     checkArgument(tableId != null, "tableId is null");
     checkArgument(authorizations != null, "authorizations is null");
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/ScannerIterator.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/ScannerIterator.java
index e5bb6bc..c3133b7 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/ScannerIterator.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/ScannerIterator.java
@@ -34,6 +34,7 @@ import org.apache.accumulo.core.clientImpl.ThriftScanner.ScanState;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.KeyValue;
 import org.apache.accumulo.core.data.Range;
+import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.util.NamingThreadFactory;
@@ -67,7 +68,7 @@ public class ScannerIterator implements Iterator<Entry<Key,Value>> {
 
   private boolean closed = false;
 
-  ScannerIterator(ClientContext context, Table.ID tableId, Authorizations authorizations,
+  ScannerIterator(ClientContext context, TableId tableId, Authorizations authorizations,
       Range range, int size, long timeOut, ScannerOptions options, boolean isolated,
       long readaheadThreshold, ScannerImpl.Reporter reporter) {
     this.timeOut = timeOut;
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/SyncingTabletLocator.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/SyncingTabletLocator.java
index 7706fcb..30b632d 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/SyncingTabletLocator.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/SyncingTabletLocator.java
@@ -26,6 +26,7 @@ import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.data.Range;
+import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.dataImpl.KeyExtent;
 import org.apache.hadoop.io.Text;
 import org.apache.log4j.Logger;
@@ -51,7 +52,7 @@ public class SyncingTabletLocator extends TabletLocator {
     }
   }
 
-  public SyncingTabletLocator(final ClientContext context, final Table.ID tableId) {
+  public SyncingTabletLocator(final ClientContext context, final TableId tableId) {
     this(() -> TabletLocator.getLocator(context, tableId));
   }
 
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/Table.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/Table.java
deleted file mode 100644
index 9ccd70c..0000000
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/Table.java
+++ /dev/null
@@ -1,64 +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.clientImpl;
-
-import java.util.concurrent.ExecutionException;
-
-import com.google.common.cache.Cache;
-import com.google.common.cache.CacheBuilder;
-
-public class Table {
-
-  /**
-   * Object representing an internal table ID. This class was created to help with type safety. For
-   * help obtaining the value of a table ID from Zookeeper, see
-   * {@link Tables#getTableId(ClientContext, String)}
-   *
-   * Uses an internal cache and private constructor for storing a WeakReference of every Table.ID.
-   * Therefore, a Table.ID can't be instantiated outside this class and is accessed by calling
-   * Table.ID.{@link #of(String)}.
-   */
-  public static class ID extends AbstractId {
-    private static final long serialVersionUID = 7399913185860577809L;
-    static final Cache<String,ID> cache = CacheBuilder.newBuilder().weakValues().build();
-
-    public static final ID METADATA = of("!0");
-    public static final ID REPLICATION = of("+rep");
-    public static final ID ROOT = of("+r");
-
-    private ID(final String canonical) {
-      super(canonical);
-    }
-
-    /**
-     * Get a Table.ID object for the provided canonical string.
-     *
-     * @param canonical
-     *          table ID string
-     * @return Table.ID object
-     */
-    public static ID of(final String canonical) {
-      try {
-        return cache.get(canonical, () -> new Table.ID(canonical));
-      } catch (ExecutionException e) {
-        throw new AssertionError(
-            "This should never happen: ID constructor should never return null.");
-      }
-    }
-  }
-
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/TableMap.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/TableMap.java
index 2515b4e..5be2a9a 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/TableMap.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/TableMap.java
@@ -25,6 +25,8 @@ import java.util.Map;
 
 import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.NamespaceNotFoundException;
+import org.apache.accumulo.core.data.NamespaceId;
+import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.fate.zookeeper.ZooCache;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -37,14 +39,14 @@ import com.google.common.collect.ImmutableMap;
 public class TableMap {
   private static final Logger log = LoggerFactory.getLogger(TableMap.class);
 
-  private final Map<String,Table.ID> tableNameToIdMap;
-  private final Map<Table.ID,String> tableIdToNameMap;
+  private final Map<String,TableId> tableNameToIdMap;
+  private final Map<TableId,String> tableIdToNameMap;
 
   public TableMap(ClientContext context, ZooCache zooCache) {
     List<String> tableIds = zooCache.getChildren(context.getZooKeeperRoot() + Constants.ZTABLES);
-    Map<Namespace.ID,String> namespaceIdToNameMap = new HashMap<>();
-    ImmutableMap.Builder<String,Table.ID> tableNameToIdBuilder = new ImmutableMap.Builder<>();
-    ImmutableMap.Builder<Table.ID,String> tableIdToNameBuilder = new ImmutableMap.Builder<>();
+    Map<NamespaceId,String> namespaceIdToNameMap = new HashMap<>();
+    ImmutableMap.Builder<String,TableId> tableNameToIdBuilder = new ImmutableMap.Builder<>();
+    ImmutableMap.Builder<TableId,String> tableIdToNameBuilder = new ImmutableMap.Builder<>();
     // use StringBuilder to construct zPath string efficiently across many tables
     StringBuilder zPathBuilder = new StringBuilder();
     zPathBuilder.append(context.getZooKeeperRoot()).append(Constants.ZTABLES).append("/");
@@ -59,13 +61,13 @@ public class TableMap {
       zPathBuilder.append(tableIdStr).append(Constants.ZTABLE_NAMESPACE);
       byte[] nId = zooCache.get(zPathBuilder.toString());
 
-      String namespaceName = Namespace.DEFAULT;
+      String namespaceName = Namespace.DEFAULT.name();
       // create fully qualified table name
       if (nId == null) {
         namespaceName = null;
       } else {
-        Namespace.ID namespaceId = Namespace.ID.of(new String(nId, UTF_8));
-        if (!namespaceId.equals(Namespace.ID.DEFAULT)) {
+        NamespaceId namespaceId = NamespaceId.of(new String(nId, UTF_8));
+        if (!namespaceId.equals(Namespace.DEFAULT.id())) {
           try {
             namespaceName = namespaceIdToNameMap.get(namespaceId);
             if (namespaceName == null) {
@@ -81,7 +83,7 @@ public class TableMap {
       }
       if (tableName != null && namespaceName != null) {
         String tableNameStr = qualified(new String(tableName, UTF_8), namespaceName);
-        Table.ID tableId = Table.ID.of(tableIdStr);
+        TableId tableId = TableId.of(tableIdStr);
         tableNameToIdBuilder.put(tableNameStr, tableId);
         tableIdToNameBuilder.put(tableId, tableNameStr);
       }
@@ -90,11 +92,11 @@ public class TableMap {
     tableIdToNameMap = tableIdToNameBuilder.build();
   }
 
-  public Map<String,Table.ID> getNameToIdMap() {
+  public Map<String,TableId> getNameToIdMap() {
     return tableNameToIdMap;
   }
 
-  public Map<Table.ID,String> getIdtoNameMap() {
+  public Map<TableId,String> getIdtoNameMap() {
     return tableIdToNameMap;
   }
 }
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/TableOperationsImpl.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/TableOperationsImpl.java
index 3893e73..d950518 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/TableOperationsImpl.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/TableOperationsImpl.java
@@ -92,6 +92,7 @@ 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.Range;
+import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.data.TabletId;
 import org.apache.accumulo.core.dataImpl.KeyExtent;
 import org.apache.accumulo.core.dataImpl.TabletIdImpl;
@@ -399,12 +400,12 @@ public class TableOperationsImpl extends TableOperationsHelper {
 
   private static class SplitEnv {
     private String tableName;
-    private Table.ID tableId;
+    private TableId tableId;
     private ExecutorService executor;
     private CountDownLatch latch;
     private AtomicReference<Throwable> exception;
 
-    SplitEnv(String tableName, Table.ID tableId, ExecutorService executor, CountDownLatch latch,
+    SplitEnv(String tableName, TableId tableId, ExecutorService executor, CountDownLatch latch,
         AtomicReference<Throwable> exception) {
       this.tableName = tableName;
       this.tableId = tableId;
@@ -458,7 +459,7 @@ public class TableOperationsImpl extends TableOperationsHelper {
   @Override
   public void addSplits(String tableName, SortedSet<Text> partitionKeys)
       throws TableNotFoundException, AccumuloException, AccumuloSecurityException {
-    Table.ID tableId = Tables.getTableId(context, tableName);
+    TableId tableId = Tables.getTableId(context, tableName);
 
     List<Text> splits = new ArrayList<>(partitionKeys);
     // should be sorted because we copied from a sorted set, but that makes assumptions about
@@ -483,7 +484,7 @@ public class TableOperationsImpl extends TableOperationsHelper {
           // user would only have the stack trace for the background thread.
           if (excep instanceof TableNotFoundException) {
             TableNotFoundException tnfe = (TableNotFoundException) excep;
-            throw new TableNotFoundException(tableId.canonicalID(), tableName,
+            throw new TableNotFoundException(tableId.canonical(), tableName,
                 "Table not found by background thread", tnfe);
           } else if (excep instanceof TableOfflineException) {
             log.debug("TableOfflineException occurred in background thread. Throwing new exception",
@@ -510,7 +511,7 @@ public class TableOperationsImpl extends TableOperationsHelper {
     }
   }
 
-  private void addSplits(String tableName, SortedSet<Text> partitionKeys, Table.ID tableId)
+  private void addSplits(String tableName, SortedSet<Text> partitionKeys, TableId tableId)
       throws AccumuloException, AccumuloSecurityException, TableNotFoundException,
       AccumuloServerException {
     TabletLocator tabLocator = TabletLocator.getLocator(context, tableId);
@@ -531,7 +532,7 @@ public class TableOperationsImpl extends TableOperationsHelper {
 
         if (tl == null) {
           if (!Tables.exists(context, tableId))
-            throw new TableNotFoundException(tableId.canonicalID(), tableName, null);
+            throw new TableNotFoundException(tableId.canonical(), tableName, null);
           else if (Tables.getTableState(context, tableId) == TableState.OFFLINE)
             throw new TableOfflineException(Tables.getTableOfflineMsg(context, tableId));
           continue;
@@ -575,7 +576,7 @@ public class TableOperationsImpl extends TableOperationsHelper {
         } catch (ThriftSecurityException e) {
           Tables.clearCache(context);
           if (!Tables.exists(context, tableId))
-            throw new TableNotFoundException(tableId.canonicalID(), tableName, null);
+            throw new TableNotFoundException(tableId.canonical(), tableName, null);
           throw new AccumuloSecurityException(e.user, e.code, e);
         } catch (NotServingTabletException e) {
           // Do not silently spin when we repeatedly fail to get the location for a tablet
@@ -641,7 +642,7 @@ public class TableOperationsImpl extends TableOperationsHelper {
 
     checkArgument(tableName != null, "tableName is null");
 
-    Table.ID tableId = Tables.getTableId(context, tableName);
+    TableId tableId = Tables.getTableId(context, tableName);
 
     TreeMap<KeyExtent,String> tabletLocations = new TreeMap<>();
 
@@ -655,7 +656,7 @@ public class TableOperationsImpl extends TableOperationsHelper {
         throw ase;
       } catch (Exception e) {
         if (!Tables.exists(context, tableId)) {
-          throw new TableNotFoundException(tableId.canonicalID(), tableName, null);
+          throw new TableNotFoundException(tableId.canonical(), tableName, null);
         }
 
         if (e instanceof RuntimeException && e.getCause() instanceof AccumuloSecurityException) {
@@ -729,7 +730,7 @@ public class TableOperationsImpl extends TableOperationsHelper {
     checkArgument(srcTableName != null, "srcTableName is null");
     checkArgument(newTableName != null, "newTableName is null");
 
-    Table.ID srcTableId = Tables.getTableId(context, srcTableName);
+    TableId srcTableId = Tables.getTableId(context, srcTableName);
 
     if (flush)
       _flush(srcTableId, null, null, true);
@@ -740,7 +741,7 @@ public class TableOperationsImpl extends TableOperationsHelper {
     if (propertiesToSet == null)
       propertiesToSet = Collections.emptyMap();
 
-    List<ByteBuffer> args = Arrays.asList(ByteBuffer.wrap(srcTableId.getUtf8()),
+    List<ByteBuffer> args = Arrays.asList(ByteBuffer.wrap(srcTableId.canonical().getBytes(UTF_8)),
         ByteBuffer.wrap(newTableName.getBytes(UTF_8)));
     Map<String,String> opts = new HashMap<>();
     for (Entry<String,String> entry : propertiesToSet.entrySet()) {
@@ -782,7 +783,7 @@ public class TableOperationsImpl extends TableOperationsHelper {
       throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
     checkArgument(tableName != null, "tableName is null");
 
-    Table.ID tableId = Tables.getTableId(context, tableName);
+    TableId tableId = Tables.getTableId(context, tableName);
     _flush(tableId, start, end, wait);
   }
 
@@ -826,7 +827,7 @@ public class TableOperationsImpl extends TableOperationsHelper {
       }
     }
 
-    Table.ID tableId = Tables.getTableId(context, tableName);
+    TableId tableId = Tables.getTableId(context, tableName);
 
     Text start = config.getStartRow();
     Text end = config.getEndRow();
@@ -834,7 +835,7 @@ public class TableOperationsImpl extends TableOperationsHelper {
     if (config.getFlush())
       _flush(tableId, start, end, true);
 
-    List<ByteBuffer> args = Arrays.asList(ByteBuffer.wrap(tableId.getUtf8()),
+    List<ByteBuffer> args = Arrays.asList(ByteBuffer.wrap(tableId.canonical().getBytes(UTF_8)),
         start == null ? EMPTY : TextUtil.getByteBuffer(start),
         end == null ? EMPTY : TextUtil.getByteBuffer(end),
         ByteBuffer.wrap(IteratorUtil.encodeIteratorSettings(config.getIterators())),
@@ -854,9 +855,9 @@ public class TableOperationsImpl extends TableOperationsHelper {
   @Override
   public void cancelCompaction(String tableName)
       throws AccumuloSecurityException, TableNotFoundException, AccumuloException {
-    Table.ID tableId = Tables.getTableId(context, tableName);
+    TableId tableId = Tables.getTableId(context, tableName);
 
-    List<ByteBuffer> args = Arrays.asList(ByteBuffer.wrap(tableId.getUtf8()));
+    List<ByteBuffer> args = Arrays.asList(ByteBuffer.wrap(tableId.canonical().getBytes(UTF_8)));
 
     Map<String,String> opts = new HashMap<>();
     try {
@@ -869,7 +870,7 @@ public class TableOperationsImpl extends TableOperationsHelper {
 
   }
 
-  private void _flush(Table.ID tableId, Text start, Text end, boolean wait)
+  private void _flush(TableId tableId, Text start, Text end, boolean wait)
       throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
 
     try {
@@ -884,7 +885,7 @@ public class TableOperationsImpl extends TableOperationsHelper {
         try {
           client = MasterClient.getConnectionWithRetry(context);
           flushID = client.initiateFlush(Tracer.traceInfo(), context.rpcCreds(),
-              tableId.canonicalID());
+              tableId.canonical());
           break;
         } catch (TTransportException tte) {
           log.debug("Failed to call initiateFlush, retrying ... ", tte);
@@ -902,7 +903,7 @@ public class TableOperationsImpl extends TableOperationsHelper {
         MasterClientService.Iface client = null;
         try {
           client = MasterClient.getConnectionWithRetry(context);
-          client.waitForFlush(Tracer.traceInfo(), context.rpcCreds(), tableId.canonicalID(),
+          client.waitForFlush(Tracer.traceInfo(), context.rpcCreds(), tableId.canonical(),
               TextUtil.getByteBuffer(start), TextUtil.getByteBuffer(end), flushID,
               wait ? Long.MAX_VALUE : 1);
           break;
@@ -920,7 +921,7 @@ public class TableOperationsImpl extends TableOperationsHelper {
     } catch (ThriftSecurityException e) {
       switch (e.getCode()) {
         case TABLE_DOESNT_EXIST:
-          throw new TableNotFoundException(tableId.canonicalID(), null, e.getMessage(), e);
+          throw new TableNotFoundException(tableId.canonical(), null, e.getMessage(), e);
         default:
           log.debug("flush security exception on table id {}", tableId);
           throw new AccumuloSecurityException(e.user, e.code, e);
@@ -1092,14 +1093,14 @@ public class TableOperationsImpl extends TableOperationsHelper {
 
     Random random = new SecureRandom();
     Map<String,Map<KeyExtent,List<Range>>> binnedRanges = new HashMap<>();
-    Table.ID tableId = Tables.getTableId(context, tableName);
+    TableId tableId = Tables.getTableId(context, tableName);
     TabletLocator tl = TabletLocator.getLocator(context, tableId);
     // its possible that the cache could contain complete, but old information about a tables
     // tablets... so clear it
     tl.invalidateCache();
     while (!tl.binRanges(context, Collections.singletonList(range), binnedRanges).isEmpty()) {
       if (!Tables.exists(context, tableId))
-        throw new TableDeletedException(tableId.canonicalID());
+        throw new TableDeletedException(tableId.canonical());
       if (Tables.getTableState(context, tableId) == TableState.OFFLINE)
         throw new TableOfflineException(Tables.getTableOfflineMsg(context, tableId));
 
@@ -1207,7 +1208,7 @@ public class TableOperationsImpl extends TableOperationsHelper {
     }
   }
 
-  private void waitForTableStateTransition(Table.ID tableId, TableState expectedState)
+  private void waitForTableStateTransition(TableId tableId, TableState expectedState)
       throws AccumuloException, TableNotFoundException {
 
     Text startRow = null;
@@ -1220,9 +1221,9 @@ public class TableOperationsImpl extends TableOperationsHelper {
         TableState currentState = Tables.getTableState(context, tableId);
         if (currentState != expectedState) {
           if (!Tables.exists(context, tableId))
-            throw new TableDeletedException(tableId.canonicalID());
+            throw new TableDeletedException(tableId.canonical());
           if (currentState == TableState.DELETING)
-            throw new TableNotFoundException(tableId.canonicalID(), "", "Table is being deleted.");
+            throw new TableNotFoundException(tableId.canonical(), "", "Table is being deleted.");
           throw new AccumuloException("Unexpected table state " + tableId + " "
               + Tables.getTableState(context, tableId) + " != " + expectedState);
         }
@@ -1315,8 +1316,8 @@ public class TableOperationsImpl extends TableOperationsHelper {
       throws AccumuloSecurityException, AccumuloException, TableNotFoundException {
 
     checkArgument(tableName != null, "tableName is null");
-    Table.ID tableId = Tables.getTableId(context, tableName);
-    List<ByteBuffer> args = Arrays.asList(ByteBuffer.wrap(tableId.getUtf8()));
+    TableId tableId = Tables.getTableId(context, tableName);
+    List<ByteBuffer> args = Arrays.asList(ByteBuffer.wrap(tableId.canonical().getBytes(UTF_8)));
     Map<String,String> opts = new HashMap<>();
 
     try {
@@ -1342,7 +1343,7 @@ public class TableOperationsImpl extends TableOperationsHelper {
       throws AccumuloSecurityException, AccumuloException, TableNotFoundException {
     checkArgument(tableName != null, "tableName is null");
 
-    Table.ID tableId = Tables.getTableId(context, tableName);
+    TableId tableId = Tables.getTableId(context, tableName);
 
     /**
      * ACCUMULO-4574 if table is already online return without executing fate operation.
@@ -1355,7 +1356,7 @@ public class TableOperationsImpl extends TableOperationsHelper {
       return;
     }
 
-    List<ByteBuffer> args = Arrays.asList(ByteBuffer.wrap(tableId.getUtf8()));
+    List<ByteBuffer> args = Arrays.asList(ByteBuffer.wrap(tableId.canonical().getBytes(UTF_8)));
     Map<String,String> opts = new HashMap<>();
 
     try {
@@ -1381,7 +1382,7 @@ public class TableOperationsImpl extends TableOperationsHelper {
   @Override
   public Map<String,String> tableIdMap() {
     return Tables.getNameToIdMap(context).entrySet().stream()
-        .collect(Collectors.toMap(Entry::getKey, e -> e.getValue().canonicalID(), (v1, v2) -> {
+        .collect(Collectors.toMap(Entry::getKey, e -> e.getValue().canonical(), (v1, v2) -> {
           throw new RuntimeException(String.format("Duplicate key for values %s and %s", v1, v2));
         }, TreeMap::new));
   }
@@ -1724,7 +1725,7 @@ public class TableOperationsImpl extends TableOperationsHelper {
     requireNonNull(tableName, "tableName must be non null");
     requireNonNull(ranges, "ranges must be non null");
 
-    Table.ID tableId = Tables.getTableId(context, tableName);
+    TableId tableId = Tables.getTableId(context, tableName);
     TabletLocator locator = TabletLocator.getLocator(context, tableId);
 
     List<Range> rangeList = null;
@@ -1745,7 +1746,7 @@ public class TableOperationsImpl extends TableOperationsHelper {
     while (!locator.binRanges(context, rangeList, binnedRanges).isEmpty()) {
 
       if (!Tables.exists(context, tableId))
-        throw new TableNotFoundException(tableId.canonicalID(), tableName, null);
+        throw new TableNotFoundException(tableId.canonical(), tableName, null);
       if (Tables.getTableState(context, tableId) == TableState.OFFLINE)
         throw new TableOfflineException(Tables.getTableOfflineMsg(context, tableId));
 
@@ -1793,14 +1794,14 @@ public class TableOperationsImpl extends TableOperationsHelper {
       @Override
       public List<Summary> retrieve()
           throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
-        Table.ID tableId = Tables.getTableId(context, tableName);
+        TableId tableId = Tables.getTableId(context, tableName);
         if (Tables.getTableState(context, tableId) == TableState.OFFLINE)
           throw new TableOfflineException(Tables.getTableOfflineMsg(context, tableId));
 
         TRowRange range = new TRowRange(TextUtil.getByteBuffer(startRow),
             TextUtil.getByteBuffer(endRow));
-        TSummaryRequest request = new TSummaryRequest(tableId.canonicalID(), range,
-            summariesToFetch, summarizerClassRegex);
+        TSummaryRequest request = new TSummaryRequest(tableId.canonical(), range, summariesToFetch,
+            summarizerClassRegex);
         if (flush) {
           _flush(tableId, startRow, endRow, true);
         }
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/Tables.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/Tables.java
index d539f98..1f393a3 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/Tables.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/Tables.java
@@ -28,6 +28,8 @@ import java.util.concurrent.TimeUnit;
 import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.NamespaceNotFoundException;
 import org.apache.accumulo.core.client.TableNotFoundException;
+import org.apache.accumulo.core.data.NamespaceId;
+import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.master.state.tables.TableState;
 import org.apache.accumulo.core.metadata.MetadataTable;
 import org.apache.accumulo.core.singletons.SingletonManager;
@@ -84,7 +86,7 @@ public class Tables {
    * NamespaceNotFoundException in TableNotFoundException if namespace is not found.
    */
 
-  public static Table.ID getTableId(ClientContext context, String tableName)
+  public static TableId getTableId(ClientContext context, String tableName)
       throws TableNotFoundException {
     try {
       return _getTableId(context, tableName);
@@ -120,9 +122,9 @@ public class Tables {
   /**
    * Lookup table ID in ZK. If not found, clears cache and tries again.
    */
-  public static Table.ID _getTableId(ClientContext context, String tableName)
+  public static TableId _getTableId(ClientContext context, String tableName)
       throws NamespaceNotFoundException, TableNotFoundException {
-    Table.ID tableId = getNameToIdMap(context).get(tableName);
+    TableId tableId = getNameToIdMap(context).get(tableName);
     if (tableId == null) {
       // maybe the table exist, but the cache was not updated yet... so try to clear the cache and
       // check again
@@ -139,30 +141,30 @@ public class Tables {
     return tableId;
   }
 
-  public static String getTableName(ClientContext context, Table.ID tableId)
+  public static String getTableName(ClientContext context, TableId tableId)
       throws TableNotFoundException {
     String tableName = getIdToNameMap(context).get(tableId);
     if (tableName == null)
-      throw new TableNotFoundException(tableId.canonicalID(), null, null);
+      throw new TableNotFoundException(tableId.canonical(), null, null);
     return tableName;
   }
 
-  public static String getTableOfflineMsg(ClientContext context, Table.ID tableId) {
+  public static String getTableOfflineMsg(ClientContext context, TableId tableId) {
     if (tableId == null)
       return "Table <unknown table> is offline";
     try {
       String tableName = Tables.getTableName(context, tableId);
-      return "Table " + tableName + " (" + tableId.canonicalID() + ") is offline";
+      return "Table " + tableName + " (" + tableId.canonical() + ") is offline";
     } catch (TableNotFoundException e) {
-      return "Table <unknown table> (" + tableId.canonicalID() + ") is offline";
+      return "Table <unknown table> (" + tableId.canonical() + ") is offline";
     }
   }
 
-  public static Map<String,Table.ID> getNameToIdMap(ClientContext context) {
+  public static Map<String,TableId> getNameToIdMap(ClientContext context) {
     return getTableMap(context).getNameToIdMap();
   }
 
-  public static Map<Table.ID,String> getIdToNameMap(ClientContext context) {
+  public static Map<TableId,String> getIdToNameMap(ClientContext context) {
     return getTableMap(context).getIdtoNameMap();
   }
 
@@ -182,10 +184,10 @@ public class Tables {
     return map;
   }
 
-  public static boolean exists(ClientContext context, Table.ID tableId) {
+  public static boolean exists(ClientContext context, TableId tableId) {
     ZooCache zc = getZooCache(context);
     List<String> tableIds = zc.getChildren(context.getZooKeeperRoot() + Constants.ZTABLES);
-    return tableIds.contains(tableId.canonicalID());
+    return tableIds.contains(tableId.canonical());
   }
 
   public static void clearCache(ClientContext context) {
@@ -208,29 +210,29 @@ public class Tables {
     instanceToMapCache.invalidate(context.getInstanceID());
   }
 
-  public static String getPrintableTableInfoFromId(ClientContext context, Table.ID tableId) {
+  public static String getPrintableTableInfoFromId(ClientContext context, TableId tableId) {
     String tableName = null;
     try {
       tableName = getTableName(context, tableId);
     } catch (TableNotFoundException e) {
       // handled in the string formatting
     }
-    return tableName == null ? String.format("?(ID:%s)", tableId.canonicalID())
-        : String.format("%s(ID:%s)", tableName, tableId.canonicalID());
+    return tableName == null ? String.format("?(ID:%s)", tableId.canonical())
+        : String.format("%s(ID:%s)", tableName, tableId.canonical());
   }
 
   public static String getPrintableTableInfoFromName(ClientContext context, String tableName) {
-    Table.ID tableId = null;
+    TableId tableId = null;
     try {
       tableId = getTableId(context, tableName);
     } catch (TableNotFoundException e) {
       // handled in the string formatting
     }
     return tableId == null ? String.format("%s(?)", tableName)
-        : String.format("%s(ID:%s)", tableName, tableId.canonicalID());
+        : String.format("%s(ID:%s)", tableName, tableId.canonical());
   }
 
-  public static TableState getTableState(ClientContext context, Table.ID tableId) {
+  public static TableState getTableState(ClientContext context, TableId tableId) {
     return getTableState(context, tableId, false);
   }
 
@@ -246,10 +248,10 @@ public class Tables {
    *          if true clear the table state in zookeeper before checking status
    * @return the table state.
    */
-  public static TableState getTableState(ClientContext context, Table.ID tableId,
+  public static TableState getTableState(ClientContext context, TableId tableId,
       boolean clearCachedState) {
 
-    String statePath = context.getZooKeeperRoot() + Constants.ZTABLES + "/" + tableId.canonicalID()
+    String statePath = context.getZooKeeperRoot() + Constants.ZTABLES + "/" + tableId.canonical()
         + Constants.ZTABLE_STATE;
 
     if (clearCachedState) {
@@ -266,19 +268,19 @@ public class Tables {
   }
 
   public static String qualified(String tableName) {
-    return qualified(tableName, Namespace.DEFAULT);
+    return qualified(tableName, Namespace.DEFAULT.name());
   }
 
   public static String qualified(String tableName, String defaultNamespace) {
     Pair<String,String> qualifiedTableName = qualify(tableName, defaultNamespace);
-    if (Namespace.DEFAULT.equals(qualifiedTableName.getFirst()))
+    if (Namespace.DEFAULT.name().equals(qualifiedTableName.getFirst()))
       return qualifiedTableName.getSecond();
     else
       return qualifiedTableName.toString("", ".", "");
   }
 
   public static Pair<String,String> qualify(String tableName) {
-    return qualify(tableName, Namespace.DEFAULT);
+    return qualify(tableName, Namespace.DEFAULT.name());
   }
 
   public static Pair<String,String> qualify(String tableName, String defaultNamespace) {
@@ -304,7 +306,7 @@ public class Tables {
    * @throws IllegalArgumentException
    *           if the table doesn't exist in ZooKeeper
    */
-  public static Namespace.ID getNamespaceId(ClientContext context, Table.ID tableId)
+  public static NamespaceId getNamespaceId(ClientContext context, TableId tableId)
       throws TableNotFoundException {
     checkArgument(context != null, "instance is null");
     checkArgument(tableId != null, "tableId is null");
@@ -315,9 +317,9 @@ public class Tables {
 
     // We might get null out of ZooCache if this tableID doesn't exist
     if (n == null) {
-      throw new TableNotFoundException(tableId.canonicalID(), null, null);
+      throw new TableNotFoundException(tableId.canonical(), null, null);
     }
 
-    return Namespace.ID.of(new String(n, UTF_8));
+    return NamespaceId.of(new String(n, UTF_8));
   }
 }
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/TabletLocator.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/TabletLocator.java
index 5955566..b868f3f 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/TabletLocator.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/TabletLocator.java
@@ -31,6 +31,7 @@ import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.data.Range;
+import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.dataImpl.KeyExtent;
 import org.apache.accumulo.core.metadata.MetadataLocationObtainer;
 import org.apache.accumulo.core.metadata.MetadataTable;
@@ -80,9 +81,9 @@ public abstract class TabletLocator {
 
   private static class LocatorKey {
     String instanceId;
-    Table.ID tableId;
+    TableId tableId;
 
-    LocatorKey(String instanceId, Table.ID table) {
+    LocatorKey(String instanceId, TableId table) {
       this.instanceId = instanceId;
       this.tableId = table;
     }
@@ -128,7 +129,7 @@ public abstract class TabletLocator {
     enabled = true;
   }
 
-  public static synchronized TabletLocator getLocator(ClientContext context, Table.ID tableId) {
+  public static synchronized TabletLocator getLocator(ClientContext context, TableId tableId) {
     Preconditions.checkState(enabled, "The Accumulo singleton that that tracks tablet locations is "
         + "disabled. This is likely caused by all AccumuloClients being closed or garbage collected"
         + ".");
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/TabletLocatorImpl.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/TabletLocatorImpl.java
index 7dd934d..8dbe2c3 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/TabletLocatorImpl.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/TabletLocatorImpl.java
@@ -42,6 +42,7 @@ import org.apache.accumulo.core.data.Key;
 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.TableId;
 import org.apache.accumulo.core.dataImpl.KeyExtent;
 import org.apache.accumulo.core.util.OpTimer;
 import org.apache.accumulo.core.util.Pair;
@@ -72,7 +73,7 @@ public class TabletLocatorImpl extends TabletLocator {
     return o1.compareTo(o2);
   };
 
-  protected Table.ID tableId;
+  protected TableId tableId;
   protected TabletLocator parent;
   protected TreeMap<Text,TabletLocation> metaCache = new TreeMap<>(END_ROW_COMPARATOR);
   protected TabletLocationObtainer locationObtainer;
@@ -140,14 +141,14 @@ public class TabletLocatorImpl extends TabletLocator {
     }
   }
 
-  public TabletLocatorImpl(Table.ID tableId, TabletLocator parent, TabletLocationObtainer tlo,
+  public TabletLocatorImpl(TableId tableId, TabletLocator parent, TabletLocationObtainer tlo,
       TabletServerLockChecker tslc) {
     this.tableId = tableId;
     this.parent = parent;
     this.locationObtainer = tlo;
     this.lockChecker = tslc;
 
-    this.lastTabletRow = new Text(tableId.getUtf8());
+    this.lastTabletRow = new Text(tableId.canonical());
     lastTabletRow.append(new byte[] {'<'}, 0, 1);
   }
 
@@ -476,7 +477,7 @@ public class TabletLocatorImpl extends TabletLocator {
   private void lookupTabletLocation(ClientContext context, Text row, boolean retry,
       LockCheckerSession lcSession)
       throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
-    Text metadataRow = new Text(tableId.getUtf8());
+    Text metadataRow = new Text(tableId.canonical());
     metadataRow.append(new byte[] {';'}, 0, 1);
     metadataRow.append(row.getBytes(), 0, row.getLength());
     TabletLocation ptl = parent.locateTablet(context, metadataRow, false, retry);
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/TabletServerBatchDeleter.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/TabletServerBatchDeleter.java
index b338f25..ce8092e 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/TabletServerBatchDeleter.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/TabletServerBatchDeleter.java
@@ -26,6 +26,7 @@ import org.apache.accumulo.core.client.IteratorSetting;
 import org.apache.accumulo.core.client.MutationsRejectedException;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Mutation;
+import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.iterators.SortedKeyIterator;
 import org.apache.accumulo.core.security.Authorizations;
@@ -34,10 +35,10 @@ import org.apache.accumulo.core.security.ColumnVisibility;
 public class TabletServerBatchDeleter extends TabletServerBatchReader implements BatchDeleter {
 
   private final ClientContext context;
-  private Table.ID tableId;
+  private TableId tableId;
   private BatchWriterConfig bwConfig;
 
-  public TabletServerBatchDeleter(ClientContext context, Table.ID tableId,
+  public TabletServerBatchDeleter(ClientContext context, TableId tableId,
       Authorizations authorizations, int numQueryThreads, BatchWriterConfig bwConfig) {
     super(context, tableId, authorizations, numQueryThreads);
     this.context = context;
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/TabletServerBatchReader.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/TabletServerBatchReader.java
index cc73b83..527fd86 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/TabletServerBatchReader.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/TabletServerBatchReader.java
@@ -27,6 +27,7 @@ import java.util.concurrent.ExecutorService;
 import org.apache.accumulo.core.client.BatchScanner;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Range;
+import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.util.SimpleThreadPool;
@@ -36,7 +37,7 @@ import org.slf4j.LoggerFactory;
 public class TabletServerBatchReader extends ScannerOptions implements BatchScanner {
   private static final Logger log = LoggerFactory.getLogger(TabletServerBatchReader.class);
 
-  private Table.ID tableId;
+  private TableId tableId;
   private int numThreads;
   private ExecutorService queryThreadPool;
 
@@ -54,7 +55,7 @@ public class TabletServerBatchReader extends ScannerOptions implements BatchScan
 
   private final int batchReaderInstance = getNextBatchReaderInstance();
 
-  public TabletServerBatchReader(ClientContext context, Table.ID tableId,
+  public TabletServerBatchReader(ClientContext context, TableId tableId,
       Authorizations authorizations, int numQueryThreads) {
     checkArgument(context != null, "context is null");
     checkArgument(tableId != null, "tableId is null");
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/TabletServerBatchReaderIterator.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/TabletServerBatchReaderIterator.java
index c37eae8..49f0c68 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/TabletServerBatchReaderIterator.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/TabletServerBatchReaderIterator.java
@@ -47,6 +47,7 @@ import org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException;
 import org.apache.accumulo.core.data.Column;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Range;
+import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.dataImpl.KeyExtent;
 import org.apache.accumulo.core.dataImpl.thrift.InitialMultiScan;
@@ -78,7 +79,7 @@ public class TabletServerBatchReaderIterator implements Iterator<Entry<Key,Value
   private static final Logger log = LoggerFactory.getLogger(TabletServerBatchReaderIterator.class);
 
   private final ClientContext context;
-  private final Table.ID tableId;
+  private final TableId tableId;
   private Authorizations authorizations = Authorizations.EMPTY;
   private final int numThreads;
   private final ExecutorService queryThreadPool;
@@ -104,7 +105,7 @@ public class TabletServerBatchReaderIterator implements Iterator<Entry<Key,Value
     void receive(List<Entry<Key,Value>> entries);
   }
 
-  public TabletServerBatchReaderIterator(ClientContext context, Table.ID tableId,
+  public TabletServerBatchReaderIterator(ClientContext context, TableId tableId,
       Authorizations authorizations, ArrayList<Range> ranges, int numThreads,
       ExecutorService queryThreadPool, ScannerOptions scannerOptions, long timeout) {
 
@@ -242,7 +243,7 @@ public class TabletServerBatchReaderIterator implements Iterator<Entry<Key,Value
         // need to always do the check when failures occur
         if (failures.size() >= lastFailureSize)
           if (!Tables.exists(context, tableId))
-            throw new TableDeletedException(tableId.canonicalID());
+            throw new TableDeletedException(tableId.canonical());
           else if (Tables.getTableState(context, tableId) == TableState.OFFLINE)
             throw new TableOfflineException(Tables.getTableOfflineMsg(context, tableId));
 
@@ -380,7 +381,7 @@ public class TabletServerBatchReaderIterator implements Iterator<Entry<Key,Value
 
         Tables.clearCache(context);
         if (!Tables.exists(context, tableId))
-          fatalException = new TableDeletedException(tableId.canonicalID());
+          fatalException = new TableDeletedException(tableId.canonical());
         else
           fatalException = e;
       } catch (SampleNotPresentException e) {
@@ -763,7 +764,7 @@ public class TabletServerBatchReaderIterator implements Iterator<Entry<Key,Value
       log.debug("Server : " + server + " msg : " + e.getMessage(), e);
       String tableInfo = "?";
       if (e.getExtent() != null) {
-        Table.ID tableId = new KeyExtent(e.getExtent()).getTableId();
+        TableId tableId = new KeyExtent(e.getExtent()).getTableId();
         tableInfo = Tables.getPrintableTableInfoFromId(context, tableId);
       }
       String message = "Table " + tableInfo + " does not have sampling configured or built";
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/TabletServerBatchWriter.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/TabletServerBatchWriter.java
index 0cd9d97..453bb98 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/TabletServerBatchWriter.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/TabletServerBatchWriter.java
@@ -54,6 +54,7 @@ import org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException;
 import org.apache.accumulo.core.constraints.Violations;
 import org.apache.accumulo.core.data.ConstraintViolationSummary;
 import org.apache.accumulo.core.data.Mutation;
+import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.data.TabletId;
 import org.apache.accumulo.core.dataImpl.KeyExtent;
 import org.apache.accumulo.core.dataImpl.TabletIdImpl;
@@ -235,7 +236,7 @@ public class TabletServerBatchWriter {
     this.notifyAll();
   }
 
-  public synchronized void addMutation(Table.ID table, Mutation m)
+  public synchronized void addMutation(TableId table, Mutation m)
       throws MutationsRejectedException {
 
     if (closed)
@@ -285,7 +286,7 @@ public class TabletServerBatchWriter {
     }
   }
 
-  public void addMutation(Table.ID table, Iterator<Mutation> iterator)
+  public void addMutation(TableId table, Iterator<Mutation> iterator)
       throws MutationsRejectedException {
     while (iterator.hasNext()) {
       addMutation(table, iterator.next());
@@ -501,14 +502,14 @@ public class TabletServerBatchWriter {
     if (authorizationFailures.size() > 0) {
 
       // was a table deleted?
-      HashSet<Table.ID> tableIds = new HashSet<>();
+      HashSet<TableId> tableIds = new HashSet<>();
       for (KeyExtent ke : authorizationFailures.keySet())
         tableIds.add(ke.getTableId());
 
       Tables.clearCache(context);
-      for (Table.ID tableId : tableIds)
+      for (TableId tableId : tableIds)
         if (!Tables.exists(context, tableId))
-          throw new TableDeletedException(tableId.canonicalID());
+          throw new TableDeletedException(tableId.canonical());
 
       synchronized (this) {
         somethingFailed = true;
@@ -599,7 +600,7 @@ public class TabletServerBatchWriter {
       return recentFailures;
     }
 
-    synchronized void add(Table.ID table, ArrayList<Mutation> tableFailures) {
+    synchronized void add(TableId table, ArrayList<Mutation> tableFailures) {
       init().addAll(table, tableFailures);
     }
 
@@ -652,7 +653,7 @@ public class TabletServerBatchWriter {
     private final SimpleThreadPool binningThreadPool;
     private final Map<String,TabletServerMutations<Mutation>> serversMutations;
     private final Set<String> queued;
-    private final Map<Table.ID,TabletLocator> locators;
+    private final Map<TableId,TabletLocator> locators;
 
     public MutationWriter(int numSendThreads) {
       serversMutations = new HashMap<>();
@@ -663,7 +664,7 @@ public class TabletServerBatchWriter {
       binningThreadPool.setRejectedExecutionHandler(new ThreadPoolExecutor.CallerRunsPolicy());
     }
 
-    private synchronized TabletLocator getLocator(Table.ID tableId) {
+    private synchronized TabletLocator getLocator(TableId tableId) {
       TabletLocator ret = locators.get(tableId);
       if (ret == null) {
         ret = new TimeoutTabletLocator(timeout, context, tableId);
@@ -675,10 +676,10 @@ public class TabletServerBatchWriter {
 
     private void binMutations(MutationSet mutationsToProcess,
         Map<String,TabletServerMutations<Mutation>> binnedMutations) {
-      Table.ID tableId = null;
+      TableId tableId = null;
       try {
-        Set<Entry<Table.ID,List<Mutation>>> es = mutationsToProcess.getMutations().entrySet();
-        for (Entry<Table.ID,List<Mutation>> entry : es) {
+        Set<Entry<TableId,List<Mutation>>> es = mutationsToProcess.getMutations().entrySet();
+        for (Entry<TableId,List<Mutation>> entry : es) {
           tableId = entry.getKey();
           TabletLocator locator = getLocator(tableId);
           List<Mutation> tableMutations = entry.getValue();
@@ -692,7 +693,7 @@ public class TabletServerBatchWriter {
 
               if (tableFailures.size() == tableMutations.size())
                 if (!Tables.exists(context, entry.getKey()))
-                  throw new TableDeletedException(entry.getKey().canonicalID());
+                  throw new TableDeletedException(entry.getKey().canonical());
                 else if (Tables.getTableState(context, tableId) == TableState.OFFLINE)
                   throw new TableOfflineException(
                       Tables.getTableOfflineMsg(context, entry.getKey()));
@@ -836,7 +837,7 @@ public class TabletServerBatchWriter {
 
           long count = 0;
 
-          Set<Table.ID> tableIds = new TreeSet<>();
+          Set<TableId> tableIds = new TreeSet<>();
           for (Map.Entry<KeyExtent,List<Mutation>> entry : mutationBatch.entrySet()) {
             count += entry.getValue().size();
             tableIds.add(entry.getKey().getTableId());
@@ -886,11 +887,11 @@ public class TabletServerBatchWriter {
           if (log.isTraceEnabled())
             log.trace("failed to send mutations to {} : {}", location, e.getMessage());
 
-          HashSet<Table.ID> tables = new HashSet<>();
+          HashSet<TableId> tables = new HashSet<>();
           for (KeyExtent ke : mutationBatch.keySet())
             tables.add(ke.getTableId());
 
-          for (Table.ID table : tables)
+          for (TableId table : tables)
             getLocator(table).invalidateCache(context, location);
 
           failedMutations.add(tsm);
@@ -974,7 +975,7 @@ public class TabletServerBatchWriter {
               int numCommitted = (int) (long) entry.getValue();
               totalCommitted += numCommitted;
 
-              Table.ID tableId = failedExtent.getTableId();
+              TableId tableId = failedExtent.getTableId();
 
               getLocator(tableId).invalidateCache(failedExtent);
 
@@ -1013,14 +1014,14 @@ public class TabletServerBatchWriter {
 
   private static class MutationSet {
 
-    private final HashMap<Table.ID,List<Mutation>> mutations;
+    private final HashMap<TableId,List<Mutation>> mutations;
     private int memoryUsed = 0;
 
     MutationSet() {
       mutations = new HashMap<>();
     }
 
-    void addMutation(Table.ID table, Mutation mutation) {
+    void addMutation(TableId table, Mutation mutation) {
       List<Mutation> tabMutList = mutations.get(table);
       if (tabMutList == null) {
         tabMutList = new ArrayList<>();
@@ -1032,7 +1033,7 @@ public class TabletServerBatchWriter {
       memoryUsed += mutation.estimatedMemoryUsed();
     }
 
-    Map<Table.ID,List<Mutation>> getMutations() {
+    Map<TableId,List<Mutation>> getMutations() {
       return mutations;
     }
 
@@ -1045,10 +1046,10 @@ public class TabletServerBatchWriter {
     }
 
     public void addAll(MutationSet failures) {
-      Set<Entry<Table.ID,List<Mutation>>> es = failures.getMutations().entrySet();
+      Set<Entry<TableId,List<Mutation>>> es = failures.getMutations().entrySet();
 
-      for (Entry<Table.ID,List<Mutation>> entry : es) {
-        Table.ID table = entry.getKey();
+      for (Entry<TableId,List<Mutation>> entry : es) {
+        TableId table = entry.getKey();
 
         for (Mutation mutation : entry.getValue()) {
           addMutation(table, mutation);
@@ -1056,7 +1057,7 @@ public class TabletServerBatchWriter {
       }
     }
 
-    public void addAll(Table.ID table, List<Mutation> mutations) {
+    public void addAll(TableId table, List<Mutation> mutations) {
       for (Mutation mutation : mutations) {
         addMutation(table, mutation);
       }
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/ThriftScanner.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/ThriftScanner.java
index e7ccd7f..ac6ecd7 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/ThriftScanner.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/ThriftScanner.java
@@ -46,6 +46,7 @@ import org.apache.accumulo.core.data.Key;
 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.TableId;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.dataImpl.KeyExtent;
 import org.apache.accumulo.core.dataImpl.thrift.InitialScan;
@@ -146,7 +147,7 @@ public class ThriftScanner {
   public static class ScanState {
 
     boolean isolated;
-    Table.ID tableId;
+    TableId tableId;
     Text startRow;
     boolean skipStartRow;
     long readaheadThreshold;
@@ -174,7 +175,7 @@ public class ThriftScanner {
     SamplerConfiguration samplerConfig;
     Map<String,String> executionHints;
 
-    public ScanState(ClientContext context, Table.ID tableId, Authorizations authorizations,
+    public ScanState(ClientContext context, TableId tableId, Authorizations authorizations,
         Range range, SortedSet<Column> fetchedColumns, int size,
         List<IterInfo> serverSideIteratorList,
         Map<String,Map<String,String>> serverSideIteratorOptions, boolean isolated,
@@ -267,7 +268,7 @@ public class ThriftScanner {
 
             if (loc == null) {
               if (!Tables.exists(context, scanState.tableId))
-                throw new TableDeletedException(scanState.tableId.canonicalID());
+                throw new TableDeletedException(scanState.tableId.canonical());
               else if (Tables.getTableState(context, scanState.tableId) == TableState.OFFLINE)
                 throw new TableOfflineException(
                     Tables.getTableOfflineMsg(context, scanState.tableId));
@@ -322,7 +323,7 @@ public class ThriftScanner {
         } catch (AccumuloSecurityException e) {
           Tables.clearCache(context);
           if (!Tables.exists(context, scanState.tableId))
-            throw new TableDeletedException(scanState.tableId.canonicalID());
+            throw new TableDeletedException(scanState.tableId.canonical());
           e.setTableInfo(Tables.getPrintableTableInfoFromId(context, scanState.tableId));
           throw e;
         } catch (TApplicationException tae) {
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/TimeoutTabletLocator.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/TimeoutTabletLocator.java
index d2b749b..cc6fd20 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/TimeoutTabletLocator.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/TimeoutTabletLocator.java
@@ -25,6 +25,7 @@ import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.client.TimedOutException;
 import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.data.Range;
+import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.dataImpl.KeyExtent;
 import org.apache.hadoop.io.Text;
 
@@ -51,7 +52,7 @@ public class TimeoutTabletLocator extends SyncingTabletLocator {
     firstFailTime = null;
   }
 
-  public TimeoutTabletLocator(long timeout, final ClientContext context, final Table.ID table) {
+  public TimeoutTabletLocator(long timeout, final ClientContext context, final TableId table) {
     super(context, table);
     this.timeout = timeout;
   }
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/Writer.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/Writer.java
index 6e37942..28e5c71 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/Writer.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/Writer.java
@@ -28,6 +28,7 @@ import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.clientImpl.TabletLocator.TabletLocation;
 import org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException;
 import org.apache.accumulo.core.data.Mutation;
+import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.dataImpl.KeyExtent;
 import org.apache.accumulo.core.rpc.ThriftUtil;
 import org.apache.accumulo.core.tabletserver.thrift.ConstraintViolationException;
@@ -47,9 +48,9 @@ public class Writer {
   private static final Logger log = LoggerFactory.getLogger(Writer.class);
 
   private ClientContext context;
-  private Table.ID tableId;
+  private TableId tableId;
 
-  public Writer(ClientContext context, Table.ID tableId) {
+  public Writer(ClientContext context, TableId tableId) {
     checkArgument(context != null, "context is null");
     checkArgument(tableId != null, "tableId is null");
     this.context = context;
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/bulk/Bulk.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/bulk/Bulk.java
index aed3e02..28f3456 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/bulk/Bulk.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/bulk/Bulk.java
@@ -22,7 +22,7 @@ import java.util.Iterator;
 import java.util.Map;
 import java.util.Objects;
 
-import org.apache.accumulo.core.clientImpl.Table;
+import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.dataImpl.KeyExtent;
 import org.apache.accumulo.core.util.TextUtil;
 import org.apache.hadoop.fs.Path;
@@ -48,7 +48,7 @@ public class Bulk {
       return tablet;
     }
 
-    public KeyExtent getKeyExtent(Table.ID tableId) {
+    public KeyExtent getKeyExtent(TableId tableId) {
       return tablet.toKeyExtent(tableId);
     }
 
@@ -70,7 +70,7 @@ public class Bulk {
       this.prevEndRow = prevEndRow == null ? null : TextUtil.getBytes(prevEndRow);
     }
 
-    public KeyExtent toKeyExtent(Table.ID tableId) {
+    public KeyExtent toKeyExtent(TableId tableId) {
       return Bulk.toKeyExtent(tableId, this);
     }
 
@@ -222,7 +222,7 @@ public class Bulk {
     return new Tablet(keyExtent.getEndRow(), keyExtent.getPrevEndRow());
   }
 
-  public static KeyExtent toKeyExtent(Table.ID tableId, Tablet tablet) {
+  public static KeyExtent toKeyExtent(TableId tableId, Tablet tablet) {
     return new KeyExtent(tableId, tablet.getEndRow(), tablet.getPrevEndRow());
   }
 }
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/bulk/BulkImport.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/bulk/BulkImport.java
index f3ebe3d..1333878 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/bulk/BulkImport.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/bulk/BulkImport.java
@@ -50,8 +50,6 @@ import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.client.admin.TableOperations.ImportDestinationArguments;
 import org.apache.accumulo.core.client.admin.TableOperations.ImportMappingOptions;
 import org.apache.accumulo.core.clientImpl.ClientContext;
-import org.apache.accumulo.core.clientImpl.Table;
-import org.apache.accumulo.core.clientImpl.Table.ID;
 import org.apache.accumulo.core.clientImpl.TableOperationsImpl;
 import org.apache.accumulo.core.clientImpl.Tables;
 import org.apache.accumulo.core.clientImpl.bulk.Bulk.FileInfo;
@@ -67,6 +65,7 @@ import org.apache.accumulo.core.data.LoadPlan;
 import org.apache.accumulo.core.data.LoadPlan.Destination;
 import org.apache.accumulo.core.data.LoadPlan.RangeType;
 import org.apache.accumulo.core.data.Range;
+import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.dataImpl.KeyExtent;
 import org.apache.accumulo.core.file.FileOperations;
 import org.apache.accumulo.core.file.FileSKVIterator;
@@ -114,7 +113,7 @@ public class BulkImport implements ImportDestinationArguments, ImportMappingOpti
   public void load()
       throws TableNotFoundException, IOException, AccumuloException, AccumuloSecurityException {
 
-    Table.ID tableId = Tables.getTableId(context, tableName);
+    TableId tableId = Tables.getTableId(context, tableName);
 
     Map<String,String> props = context.instanceOperations().getSystemConfiguration();
     AccumuloConfiguration conf = new ConfigurationCopy(props);
@@ -133,7 +132,7 @@ public class BulkImport implements ImportDestinationArguments, ImportMappingOpti
 
     BulkSerialize.writeLoadMapping(mappings, srcPath.toString(), fs::create);
 
-    List<ByteBuffer> args = Arrays.asList(ByteBuffer.wrap(tableId.getUtf8()),
+    List<ByteBuffer> args = Arrays.asList(ByteBuffer.wrap(tableId.canonical().getBytes(UTF_8)),
         ByteBuffer.wrap(srcPath.toString().getBytes(UTF_8)),
         ByteBuffer.wrap((setTime + "").getBytes(UTF_8)));
     new TableOperationsImpl(context).doBulkFateOperation(args, tableName);
@@ -339,7 +338,8 @@ public class BulkImport implements ImportDestinationArguments, ImportMappingOpti
     return fileLenCache;
   }
 
-  private SortedMap<KeyExtent,Files> computeMappingFromPlan(FileSystem fs, ID tableId, Path srcPath)
+  private SortedMap<KeyExtent,Files> computeMappingFromPlan(FileSystem fs, TableId tableId,
+      Path srcPath)
       throws IOException, AccumuloException, AccumuloSecurityException, TableNotFoundException {
 
     Map<String,List<Destination>> fileDestinations = plan.getDestinations().stream()
@@ -393,7 +393,7 @@ public class BulkImport implements ImportDestinationArguments, ImportMappingOpti
     return row == null ? null : new Text(row);
   }
 
-  private Set<KeyExtent> mapDesitnationsToExtents(Table.ID tableId, KeyExtentCache kec,
+  private Set<KeyExtent> mapDesitnationsToExtents(TableId tableId, KeyExtentCache kec,
       List<Destination> destinations)
       throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
     Set<KeyExtent> extents = new HashSet<>();
@@ -423,7 +423,7 @@ public class BulkImport implements ImportDestinationArguments, ImportMappingOpti
     return extents;
   }
 
-  private SortedMap<KeyExtent,Bulk.Files> computeMappingFromFiles(FileSystem fs, Table.ID tableId,
+  private SortedMap<KeyExtent,Bulk.Files> computeMappingFromFiles(FileSystem fs, TableId tableId,
       Path dirPath) throws IOException {
 
     Executor executor;
@@ -484,7 +484,7 @@ public class BulkImport implements ImportDestinationArguments, ImportMappingOpti
   }
 
   public static SortedMap<KeyExtent,Bulk.Files> computeFileToTabletMappings(FileSystem fs,
-      Table.ID tableId, Path dirPath, Executor executor, ClientContext context) throws IOException {
+      TableId tableId, Path dirPath, Executor executor, ClientContext context) throws IOException {
 
     KeyExtentCache extentCache = new ConcurrentKeyExtentCache(tableId, context);
 
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/bulk/BulkSerialize.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/bulk/BulkSerialize.java
index 3eb8541..f106766 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/bulk/BulkSerialize.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/bulk/BulkSerialize.java
@@ -35,9 +35,9 @@ import java.util.Set;
 import java.util.SortedMap;
 
 import org.apache.accumulo.core.Constants;
-import org.apache.accumulo.core.clientImpl.Table;
 import org.apache.accumulo.core.clientImpl.bulk.Bulk.Files;
 import org.apache.accumulo.core.clientImpl.bulk.Bulk.Mapping;
+import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.dataImpl.KeyExtent;
 import org.apache.hadoop.fs.Path;
 
@@ -114,7 +114,7 @@ public class BulkSerialize {
   /**
    * Read Json array of Bulk.Mapping into LoadMappingIterator
    */
-  public static LoadMappingIterator readLoadMapping(String bulkDir, Table.ID tableId, Input input)
+  public static LoadMappingIterator readLoadMapping(String bulkDir, TableId tableId, Input input)
       throws IOException {
     final Path lmFile = new Path(bulkDir, Constants.BULK_LOAD_MAPPING);
     return new LoadMappingIterator(tableId, input.open(lmFile));
@@ -154,7 +154,7 @@ public class BulkSerialize {
    * because the load mapping file was written with the original file names before they were moved
    * by BulkImportMove
    */
-  public static LoadMappingIterator getUpdatedLoadMapping(String bulkDir, Table.ID tableId,
+  public static LoadMappingIterator getUpdatedLoadMapping(String bulkDir, TableId tableId,
       Input input) throws IOException {
     Map<String,String> renames = readRenameMap(bulkDir, input);
     LoadMappingIterator lmi = readLoadMapping(bulkDir, tableId, input);
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/bulk/ConcurrentKeyExtentCache.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/bulk/ConcurrentKeyExtentCache.java
index b6b147f..6c3ed42 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/bulk/ConcurrentKeyExtentCache.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/bulk/ConcurrentKeyExtentCache.java
@@ -31,8 +31,8 @@ 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.clientImpl.ClientContext;
-import org.apache.accumulo.core.clientImpl.Table.ID;
 import org.apache.accumulo.core.clientImpl.bulk.BulkImport.KeyExtentCache;
+import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.dataImpl.KeyExtent;
 import org.apache.accumulo.core.metadata.schema.TabletMetadata;
 import org.apache.accumulo.core.metadata.schema.TabletsMetadata;
@@ -51,10 +51,10 @@ class ConcurrentKeyExtentCache implements KeyExtentCache {
   private ConcurrentSkipListMap<Text,KeyExtent> extents = new ConcurrentSkipListMap<>((t1, t2) -> {
     return (t1 == t2) ? 0 : (t1 == MAX ? 1 : (t2 == MAX ? -1 : t1.compareTo(t2)));
   });
-  private ID tableId;
+  private TableId tableId;
   private ClientContext ctx;
 
-  ConcurrentKeyExtentCache(ID tableId, ClientContext ctx) {
+  ConcurrentKeyExtentCache(TableId tableId, ClientContext ctx) {
     this.tableId = tableId;
     this.ctx = ctx;
   }
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/bulk/LoadMappingIterator.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/bulk/LoadMappingIterator.java
index 8a3ea8b..e66712d 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/bulk/LoadMappingIterator.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/bulk/LoadMappingIterator.java
@@ -28,7 +28,7 @@ import java.util.AbstractMap;
 import java.util.Iterator;
 import java.util.Map;
 
-import org.apache.accumulo.core.clientImpl.Table;
+import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.dataImpl.KeyExtent;
 
 import com.google.gson.Gson;
@@ -39,12 +39,12 @@ import com.google.gson.stream.JsonReader;
  */
 public class LoadMappingIterator
     implements Iterator<Map.Entry<KeyExtent,Bulk.Files>>, AutoCloseable {
-  private Table.ID tableId;
+  private TableId tableId;
   private JsonReader reader;
   private Gson gson = createGson();
   private Map<String,String> renameMap;
 
-  LoadMappingIterator(Table.ID tableId, InputStream loadMapFile) throws IOException {
+  LoadMappingIterator(TableId tableId, InputStream loadMapFile) throws IOException {
     this.tableId = tableId;
     this.reader = new JsonReader(new BufferedReader(new InputStreamReader(loadMapFile, UTF_8)));
     this.reader.beginArray();
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/mapred/BatchInputSplit.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/mapred/BatchInputSplit.java
index a6eb553..59ef860 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/mapred/BatchInputSplit.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/mapred/BatchInputSplit.java
@@ -19,8 +19,8 @@ package org.apache.accumulo.core.clientImpl.mapred;
 import java.io.IOException;
 import java.util.Collection;
 
-import org.apache.accumulo.core.clientImpl.Table;
 import org.apache.accumulo.core.data.Range;
+import org.apache.accumulo.core.data.TableId;
 import org.apache.hadoop.mapred.InputSplit;
 
 import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
@@ -45,7 +45,7 @@ public class BatchInputSplit extends org.apache.accumulo.core.clientImpl.mapredu
     super(split);
   }
 
-  public BatchInputSplit(String table, Table.ID tableId, Collection<Range> ranges,
+  public BatchInputSplit(String table, TableId tableId, Collection<Range> ranges,
       String[] location) {
     super(table, tableId, ranges, location);
   }
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/mapreduce/BatchInputSplit.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/mapreduce/BatchInputSplit.java
index 5f929a9..f244099 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/mapreduce/BatchInputSplit.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/mapreduce/BatchInputSplit.java
@@ -24,10 +24,10 @@ import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
 
-import org.apache.accumulo.core.clientImpl.Table;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.PartialKey;
 import org.apache.accumulo.core.data.Range;
+import org.apache.accumulo.core.data.TableId;
 
 /**
  * The Class BatchInputSplit. Encapsulates a set of Accumulo ranges on a single tablet for use in
@@ -49,9 +49,9 @@ public class BatchInputSplit extends org.apache.accumulo.core.client.mapreduce.R
     this.setRanges(split.getRanges());
   }
 
-  public BatchInputSplit(String table, Table.ID tableId, Collection<Range> ranges,
+  public BatchInputSplit(String table, TableId tableId, Collection<Range> ranges,
       String[] locations) {
-    super(table, tableId.canonicalID(), new Range(), locations);
+    super(table, tableId.canonical(), new Range(), locations);
     this.ranges = ranges;
   }
 
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/mapreduce/lib/InputConfigurator.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/mapreduce/lib/InputConfigurator.java
index a15ae8a..68d856a 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/mapreduce/lib/InputConfigurator.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/mapreduce/lib/InputConfigurator.java
@@ -48,11 +48,11 @@ import org.apache.accumulo.core.client.Scanner;
 import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.client.sample.SamplerConfiguration;
 import org.apache.accumulo.core.clientImpl.ClientContext;
-import org.apache.accumulo.core.clientImpl.Table;
 import org.apache.accumulo.core.clientImpl.Tables;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.PartialKey;
 import org.apache.accumulo.core.data.Range;
+import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.dataImpl.KeyExtent;
 import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
@@ -798,7 +798,7 @@ public class InputConfigurator extends ConfiguratorBase {
     return null;
   }
 
-  public static Map<String,Map<KeyExtent,List<Range>>> binOffline(Table.ID tableId,
+  public static Map<String,Map<KeyExtent,List<Range>>> binOffline(TableId tableId,
       List<Range> ranges, ClientContext context) throws AccumuloException, TableNotFoundException {
 
     Map<String,Map<KeyExtent,List<Range>>> binnedRanges = new HashMap<>();
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/AbstractId.java b/core/src/main/java/org/apache/accumulo/core/data/AbstractId.java
similarity index 55%
rename from core/src/main/java/org/apache/accumulo/core/clientImpl/AbstractId.java
rename to core/src/main/java/org/apache/accumulo/core/data/AbstractId.java
index 890d1fa..269eb9e 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/AbstractId.java
+++ b/core/src/main/java/org/apache/accumulo/core/data/AbstractId.java
@@ -14,10 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.accumulo.core.clientImpl;
-
-import static java.nio.charset.StandardCharsets.UTF_8;
-import static java.util.Objects.requireNonNull;
+package org.apache.accumulo.core.data;
 
 import java.io.Serializable;
 import java.util.Objects;
@@ -25,46 +22,35 @@ import java.util.Objects;
 /**
  * An abstract identifier class for comparing equality of identifiers of the same type.
  */
-public abstract class AbstractId implements Comparable<AbstractId>, Serializable {
+public abstract class AbstractId<T extends AbstractId<T>> implements Comparable<T>, Serializable {
 
-  private static final long serialVersionUID = -155513612834787244L;
+  private static final long serialVersionUID = 1L;
   private final String canonical;
-  private Integer hashCode = null;
 
   protected AbstractId(final String canonical) {
-    requireNonNull(canonical, "canonical cannot be null");
-    this.canonical = canonical;
+    this.canonical = Objects.requireNonNull(canonical, "canonical cannot be null");
   }
 
   /**
-   * The canonical ID
+   * The canonical ID. This is guaranteed to be non-null.
    */
-  public final String canonicalID() {
+  public final String canonical() {
     return canonical;
   }
 
-  public boolean isEmpty() {
-    return canonical.isEmpty();
-  }
-
   /**
    * AbstractID objects are considered equal if, and only if, they are of the same type and have the
    * same canonical identifier.
    */
   @Override
   public boolean equals(final Object obj) {
-    if (this == obj)
-      return true;
-    return obj != null && Objects.equals(getClass(), obj.getClass())
-        && Objects.equals(canonicalID(), ((AbstractId) obj).canonicalID());
+    return this == obj || (obj != null && Objects.equals(getClass(), obj.getClass())
+        && Objects.equals(canonical(), ((AbstractId<?>) obj).canonical()));
   }
 
   @Override
   public int hashCode() {
-    if (hashCode == null) {
-      hashCode = Objects.hash(canonicalID());
-    }
-    return hashCode;
+    return canonical().hashCode();
   }
 
   /**
@@ -72,20 +58,12 @@ public abstract class AbstractId implements Comparable<AbstractId>, Serializable
    */
   @Override
   public String toString() {
-    return canonical;
-  }
-
-  /**
-   * Return a UTF_8 byte[] of the canonical ID.
-   */
-  public final byte[] getUtf8() {
-    return canonical.getBytes(UTF_8);
+    return canonical();
   }
 
   @Override
-  public int compareTo(AbstractId id) {
-    requireNonNull(id, "id cannot be null");
-    return this.canonicalID().compareTo(id.canonicalID());
+  public int compareTo(T other) {
+    return canonical().compareTo(Objects.requireNonNull(other, "other cannot be null").canonical());
   }
 
 }
diff --git a/core/src/main/java/org/apache/accumulo/core/data/NamespaceId.java b/core/src/main/java/org/apache/accumulo/core/data/NamespaceId.java
new file mode 100644
index 0000000..dc58a96
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/data/NamespaceId.java
@@ -0,0 +1,51 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.accumulo.core.data;
+
+import java.util.concurrent.ExecutionException;
+
+import com.google.common.cache.Cache;
+import com.google.common.cache.CacheBuilder;
+
+/**
+ * A strongly typed representation of a namespace ID. This class cannot be used to get a namespace
+ * ID from a namespace name, but does provide the namespace ID string wrapped with a stronger type.
+ */
+public class NamespaceId extends AbstractId<NamespaceId> {
+  private static final long serialVersionUID = 1L;
+  static final Cache<String,NamespaceId> cache = CacheBuilder.newBuilder().weakValues().build();
+
+  private NamespaceId(String canonical) {
+    super(canonical);
+  }
+
+  /**
+   * Get a Namespace.ID object for the provided canonical string.
+   *
+   * @param canonical
+   *          Namespace ID string
+   * @return Namespace.ID object
+   */
+  public static NamespaceId of(final String canonical) {
+    try {
+      return cache.get(canonical, () -> new NamespaceId(canonical));
+    } catch (ExecutionException e) {
+      throw new AssertionError(
+          "This should never happen: ID constructor should never return null.");
+    }
+  }
+}
diff --git a/core/src/main/java/org/apache/accumulo/core/data/TableId.java b/core/src/main/java/org/apache/accumulo/core/data/TableId.java
new file mode 100644
index 0000000..c191170
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/data/TableId.java
@@ -0,0 +1,51 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.accumulo.core.data;
+
+import java.util.concurrent.ExecutionException;
+
+import com.google.common.cache.Cache;
+import com.google.common.cache.CacheBuilder;
+
+/**
+ * A strongly typed representation of a table ID. This class cannot be used to get a table ID from a
+ * table name, but does provide the table ID string wrapped with a stronger type.
+ */
+public class TableId extends AbstractId<TableId> {
+  private static final long serialVersionUID = 1L;
+  static final Cache<String,TableId> cache = CacheBuilder.newBuilder().weakValues().build();
+
+  private TableId(final String canonical) {
+    super(canonical);
+  }
+
+  /**
+   * Get a Table.ID object for the provided canonical string.
+   *
+   * @param canonical
+   *          table ID string
+   * @return Table.ID object
+   */
+  public static TableId of(final String canonical) {
+    try {
+      return cache.get(canonical, () -> new TableId(canonical));
+    } catch (ExecutionException e) {
+      throw new AssertionError(
+          "This should never happen: ID constructor should never return null.");
+    }
+  }
+}
diff --git a/core/src/main/java/org/apache/accumulo/core/dataImpl/KeyExtent.java b/core/src/main/java/org/apache/accumulo/core/dataImpl/KeyExtent.java
index bc94b06..a9bfdaf 100644
--- a/core/src/main/java/org/apache/accumulo/core/dataImpl/KeyExtent.java
+++ b/core/src/main/java/org/apache/accumulo/core/dataImpl/KeyExtent.java
@@ -35,10 +35,10 @@ import java.util.SortedSet;
 import java.util.TreeSet;
 import java.util.UUID;
 
-import org.apache.accumulo.core.clientImpl.Table;
 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.TableId;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.dataImpl.thrift.TKeyExtent;
 import org.apache.accumulo.core.metadata.MetadataTable;
@@ -55,11 +55,11 @@ import org.apache.hadoop.io.WritableComparable;
  */
 public class KeyExtent implements WritableComparable<KeyExtent> {
 
-  private Table.ID tableId;
+  private TableId tableId;
   private Text textEndRow;
   private Text textPrevEndRow;
 
-  private static final Table.ID EMPTY_ID = Table.ID.of("");
+  private static final TableId EMPTY_ID = TableId.of("");
   private static final Text EMPTY_TEXT = new Text("");
 
   private void check() {
@@ -86,7 +86,7 @@ public class KeyExtent implements WritableComparable<KeyExtent> {
     this.setPrevEndRow(new Text(), false, false);
   }
 
-  public KeyExtent(Table.ID table, Text endRow, Text prevEndRow) {
+  public KeyExtent(TableId table, Text endRow, Text prevEndRow) {
     this.setTableId(table);
     this.setEndRow(endRow, false, true);
     this.setPrevEndRow(prevEndRow, false, true);
@@ -104,7 +104,7 @@ public class KeyExtent implements WritableComparable<KeyExtent> {
   }
 
   public KeyExtent(TKeyExtent tke) {
-    this.setTableId(Table.ID.of(new String(ByteBufferUtil.toBytes(tke.table), UTF_8)));
+    this.setTableId(TableId.of(new String(ByteBufferUtil.toBytes(tke.table), UTF_8)));
     this.setEndRow(tke.endRow == null ? null : new Text(ByteBufferUtil.toBytes(tke.endRow)), false,
         false);
     this.setPrevEndRow(
@@ -149,7 +149,7 @@ public class KeyExtent implements WritableComparable<KeyExtent> {
    * Sets the extents table id
    *
    */
-  public void setTableId(Table.ID tId) {
+  public void setTableId(TableId tId) {
     Objects.requireNonNull(tId, "null table id not allowed");
 
     this.tableId = tId;
@@ -161,7 +161,7 @@ public class KeyExtent implements WritableComparable<KeyExtent> {
    * Returns the extent's table id
    *
    */
-  public Table.ID getTableId() {
+  public TableId getTableId() {
     return tableId;
   }
 
@@ -229,7 +229,7 @@ public class KeyExtent implements WritableComparable<KeyExtent> {
   public void readFields(DataInput in) throws IOException {
     Text tid = new Text();
     tid.readFields(in);
-    setTableId(Table.ID.of(tid.toString()));
+    setTableId(TableId.of(tid.toString()));
     boolean hasRow = in.readBoolean();
     if (hasRow) {
       Text er = new Text();
@@ -253,7 +253,7 @@ public class KeyExtent implements WritableComparable<KeyExtent> {
 
   @Override
   public void write(DataOutput out) throws IOException {
-    new Text(getTableId().getUtf8()).write(out);
+    new Text(getTableId().canonical()).write(out);
     if (getEndRow() != null) {
       out.writeBoolean(true);
       getEndRow().write(out);
@@ -357,7 +357,7 @@ public class KeyExtent implements WritableComparable<KeyExtent> {
   public String toString() {
     String endRowString;
     String prevEndRowString;
-    String tableIdString = getTableId().canonicalID().replaceAll(";", "\\\\;").replaceAll("\\\\",
+    String tableIdString = getTableId().canonical().replaceAll(";", "\\\\;").replaceAll("\\\\",
         "\\\\\\\\");
 
     if (getEndRow() == null)
@@ -431,12 +431,12 @@ public class KeyExtent implements WritableComparable<KeyExtent> {
       String decodedString = new String(
           Arrays.copyOfRange(flattenedExtent.getBytes(), 0, flattenedExtent.getLength() - 1),
           UTF_8);
-      Table.ID tableId = Table.ID.of(decodedString);
+      TableId tableId = TableId.of(decodedString);
       this.setTableId(tableId);
       this.setEndRow(null, false, false);
     } else {
 
-      Table.ID tableId = Table.ID
+      TableId tableId = TableId
           .of(new String(Arrays.copyOfRange(flattenedExtent.getBytes(), 0, semiPos), UTF_8));
 
       Text endRow = new Text();
@@ -452,7 +452,7 @@ public class KeyExtent implements WritableComparable<KeyExtent> {
   public static byte[] tableOfMetadataRow(Text row) {
     KeyExtent ke = new KeyExtent();
     ke.decodeMetadataRow(row);
-    return ke.getTableId().getUtf8();
+    return ke.getTableId().canonical().getBytes(UTF_8);
   }
 
   public boolean contains(final ByteSequence bsrow) {
@@ -640,7 +640,7 @@ public class KeyExtent implements WritableComparable<KeyExtent> {
   }
 
   public TKeyExtent toThrift() {
-    return new TKeyExtent(ByteBuffer.wrap(tableId.getUtf8()),
+    return new TKeyExtent(ByteBuffer.wrap(tableId.canonical().getBytes(UTF_8)),
         textEndRow == null ? null : TextUtil.getByteBuffer(textEndRow),
         textPrevEndRow == null ? null : TextUtil.getByteBuffer(textPrevEndRow));
   }
diff --git a/core/src/main/java/org/apache/accumulo/core/dataImpl/TabletIdImpl.java b/core/src/main/java/org/apache/accumulo/core/dataImpl/TabletIdImpl.java
index 70abb20..04f7191 100644
--- a/core/src/main/java/org/apache/accumulo/core/dataImpl/TabletIdImpl.java
+++ b/core/src/main/java/org/apache/accumulo/core/dataImpl/TabletIdImpl.java
@@ -36,7 +36,7 @@ public class TabletIdImpl implements TabletId {
 
   @Override
   public Text getTableId() {
-    return new Text(ke.getTableId().getUtf8());
+    return new Text(ke.getTableId().canonical());
   }
 
   @Override
diff --git a/core/src/main/java/org/apache/accumulo/core/file/rfile/BlockIndex.java b/core/src/main/java/org/apache/accumulo/core/file/rfile/BlockIndex.java
index fe62461..fd081ba 100644
--- a/core/src/main/java/org/apache/accumulo/core/file/rfile/BlockIndex.java
+++ b/core/src/main/java/org/apache/accumulo/core/file/rfile/BlockIndex.java
@@ -16,8 +16,6 @@
  */
 package org.apache.accumulo.core.file.rfile;
 
-import static org.apache.accumulo.core.file.blockfile.impl.CachableBlockFile.CachedBlockRead;
-
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Arrays;
@@ -27,6 +25,7 @@ import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.file.blockfile.cache.impl.ClassSize;
 import org.apache.accumulo.core.file.blockfile.cache.impl.SizeConstants;
+import org.apache.accumulo.core.file.blockfile.impl.CachableBlockFile.CachedBlockRead;
 import org.apache.accumulo.core.file.rfile.MultiLevelIndex.IndexEntry;
 import org.apache.accumulo.core.spi.cache.CacheEntry.Weighable;
 
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 3f857e3..d45619d 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.clientImpl.ClientContext;
-import org.apache.accumulo.core.clientImpl.Table;
+import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.dataImpl.KeyExtent;
 
 /**
@@ -35,10 +35,10 @@ public abstract class MetadataServicer {
   public static MetadataServicer forTableName(ClientContext context, String tableName)
       throws AccumuloException, AccumuloSecurityException {
     checkArgument(tableName != null, "tableName is null");
-    return forTableId(context, Table.ID.of(context.tableOperations().tableIdMap().get(tableName)));
+    return forTableId(context, TableId.of(context.tableOperations().tableIdMap().get(tableName)));
   }
 
-  public static MetadataServicer forTableId(ClientContext context, Table.ID tableId) {
+  public static MetadataServicer forTableId(ClientContext context, TableId tableId) {
     checkArgument(tableId != null, "tableId is null");
     if (RootTable.ID.equals(tableId))
       return new ServicerForRootTable(context);
@@ -52,7 +52,7 @@ public abstract class MetadataServicer {
    *
    * @return the table id of the table currently being serviced
    */
-  public abstract Table.ID getServicedTableId();
+  public abstract TableId getServicedTableId();
 
   /**
    * Populate the provided data structure with the known tablets for the table being serviced
diff --git a/core/src/main/java/org/apache/accumulo/core/metadata/MetadataTable.java b/core/src/main/java/org/apache/accumulo/core/metadata/MetadataTable.java
index 043696b..b5208e3 100644
--- a/core/src/main/java/org/apache/accumulo/core/metadata/MetadataTable.java
+++ b/core/src/main/java/org/apache/accumulo/core/metadata/MetadataTable.java
@@ -17,13 +17,13 @@
 package org.apache.accumulo.core.metadata;
 
 import org.apache.accumulo.core.clientImpl.Namespace;
-import org.apache.accumulo.core.clientImpl.Table;
+import org.apache.accumulo.core.data.TableId;
 
 public class MetadataTable {
 
   public static final String OLD_NAME = "!METADATA";
 
-  public static final Table.ID ID = Table.ID.METADATA;
-  public static final String NAME = Namespace.ACCUMULO + ".metadata";
+  public static final TableId ID = TableId.of("!0");
+  public static final String NAME = Namespace.ACCUMULO.name() + ".metadata";
 
 }
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 90f3642..4c7f6db 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,14 +17,15 @@
 package org.apache.accumulo.core.metadata;
 
 import org.apache.accumulo.core.clientImpl.Namespace;
-import org.apache.accumulo.core.clientImpl.Table;
+import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.dataImpl.KeyExtent;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection;
 
 public class RootTable {
 
-  public static final Table.ID ID = Table.ID.ROOT;
-  public static final String NAME = Namespace.ACCUMULO + ".root";
+  public static final TableId ID = TableId.of("+r");
+
+  public static final String NAME = Namespace.ACCUMULO.name() + ".root";
 
   /**
    * DFS location relative to the Accumulo directory
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 15c701c..b8d1fd1 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
@@ -19,7 +19,7 @@ package org.apache.accumulo.core.metadata;
 import java.util.SortedMap;
 
 import org.apache.accumulo.core.clientImpl.ClientContext;
-import org.apache.accumulo.core.clientImpl.Table;
+import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.dataImpl.KeyExtent;
 
 /**
@@ -35,7 +35,7 @@ class ServicerForRootTable extends MetadataServicer {
   }
 
   @Override
-  public Table.ID getServicedTableId() {
+  public TableId getServicedTableId() {
     return RootTable.ID;
   }
 
diff --git a/core/src/main/java/org/apache/accumulo/core/metadata/ServicerForUserTables.java b/core/src/main/java/org/apache/accumulo/core/metadata/ServicerForUserTables.java
index 9a8a5d3..d1d180b 100644
--- a/core/src/main/java/org/apache/accumulo/core/metadata/ServicerForUserTables.java
+++ b/core/src/main/java/org/apache/accumulo/core/metadata/ServicerForUserTables.java
@@ -17,7 +17,7 @@
 package org.apache.accumulo.core.metadata;
 
 import org.apache.accumulo.core.clientImpl.ClientContext;
-import org.apache.accumulo.core.clientImpl.Table;
+import org.apache.accumulo.core.data.TableId;
 
 /**
  * A metadata servicer for user tables.<br>
@@ -25,7 +25,7 @@ import org.apache.accumulo.core.clientImpl.Table;
  */
 class ServicerForUserTables extends TableMetadataServicer {
 
-  public ServicerForUserTables(ClientContext context, Table.ID tableId) {
+  public ServicerForUserTables(ClientContext context, TableId tableId) {
     super(context, MetadataTable.NAME, tableId);
   }
 
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 9689f29..1e34006 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
@@ -25,8 +25,8 @@ import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.Scanner;
 import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.clientImpl.ClientContext;
-import org.apache.accumulo.core.clientImpl.Table;
 import org.apache.accumulo.core.data.Key;
+import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.dataImpl.KeyExtent;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection;
@@ -39,18 +39,18 @@ import org.apache.hadoop.io.Text;
 abstract class TableMetadataServicer extends MetadataServicer {
 
   private final ClientContext context;
-  private Table.ID tableIdBeingServiced;
+  private TableId tableIdBeingServiced;
   private String serviceTableName;
 
   public TableMetadataServicer(ClientContext context, String serviceTableName,
-      Table.ID tableIdBeingServiced) {
+      TableId tableIdBeingServiced) {
     this.context = context;
     this.serviceTableName = serviceTableName;
     this.tableIdBeingServiced = tableIdBeingServiced;
   }
 
   @Override
-  public Table.ID getServicedTableId() {
+  public TableId getServicedTableId() {
     return tableIdBeingServiced;
   }
 
diff --git a/core/src/main/java/org/apache/accumulo/core/metadata/schema/MetadataSchema.java b/core/src/main/java/org/apache/accumulo/core/metadata/schema/MetadataSchema.java
index 0cb8d49..c91506e 100644
--- a/core/src/main/java/org/apache/accumulo/core/metadata/schema/MetadataSchema.java
+++ b/core/src/main/java/org/apache/accumulo/core/metadata/schema/MetadataSchema.java
@@ -20,11 +20,11 @@ import static com.google.common.base.Preconditions.checkArgument;
 import static java.util.Objects.requireNonNull;
 
 import org.apache.accumulo.core.client.admin.TimeType;
-import org.apache.accumulo.core.clientImpl.Table;
 import org.apache.accumulo.core.data.ArrayByteSequence;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.PartialKey;
 import org.apache.accumulo.core.data.Range;
+import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.schema.Section;
 import org.apache.accumulo.core.util.ColumnFQ;
 import org.apache.hadoop.io.Text;
@@ -46,13 +46,13 @@ public class MetadataSchema {
       return section.getRange();
     }
 
-    public static Range getRange(Table.ID tableId) {
-      return new Range(new Key(tableId.canonicalID() + ';'), true,
-          new Key(tableId.canonicalID() + '<').followingKey(PartialKey.ROW), false);
+    public static Range getRange(TableId tableId) {
+      return new Range(new Key(tableId.canonical() + ';'), true,
+          new Key(tableId.canonical() + '<').followingKey(PartialKey.ROW), false);
     }
 
-    public static Text getRow(Table.ID tableId, Text endRow) {
-      Text entry = new Text(tableId.getUtf8());
+    public static Text getRow(TableId tableId, Text endRow) {
+      Text entry = new Text(tableId.canonical());
 
       if (endRow == null) {
         // append delimiter for default tablet
@@ -291,9 +291,9 @@ public class MetadataSchema {
      * @param k
      *          Key to extract from
      */
-    public static Table.ID getTableId(Key k) {
+    public static TableId getTableId(Key k) {
       requireNonNull(k);
-      return Table.ID.of(k.getColumnQualifier().toString());
+      return TableId.of(k.getColumnQualifier().toString());
     }
 
     /**
diff --git a/core/src/main/java/org/apache/accumulo/core/metadata/schema/TabletMetadata.java b/core/src/main/java/org/apache/accumulo/core/metadata/schema/TabletMetadata.java
index 55195ee..3929425 100644
--- a/core/src/main/java/org/apache/accumulo/core/metadata/schema/TabletMetadata.java
+++ b/core/src/main/java/org/apache/accumulo/core/metadata/schema/TabletMetadata.java
@@ -37,10 +37,10 @@ import java.util.function.Function;
 
 import org.apache.accumulo.core.client.RowIterator;
 import org.apache.accumulo.core.client.Scanner;
-import org.apache.accumulo.core.clientImpl.Table;
 import org.apache.accumulo.core.data.ByteSequence;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Range;
+import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.dataImpl.KeyExtent;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.BulkFileColumnFamily;
@@ -68,7 +68,7 @@ import com.google.common.collect.Iterators;
 
 public class TabletMetadata {
 
-  private Table.ID tableId;
+  private TableId tableId;
   private Text prevEndRow;
   private boolean sawPrevEndRow = false;
   private Text endRow;
@@ -131,7 +131,7 @@ public class TabletMetadata {
     }
   }
 
-  public Table.ID getTableId() {
+  public TableId getTableId() {
     return tableId;
   }
 
@@ -363,7 +363,7 @@ public class TabletMetadata {
   @VisibleForTesting
   static TabletMetadata create(String id, String prevEndRow, String endRow) {
     TabletMetadata te = new TabletMetadata();
-    te.tableId = Table.ID.of(id);
+    te.tableId = TableId.of(id);
     te.sawPrevEndRow = true;
     te.prevEndRow = prevEndRow == null ? null : new Text(prevEndRow);
     te.endRow = endRow == null ? null : new Text(endRow);
diff --git a/core/src/main/java/org/apache/accumulo/core/metadata/schema/TabletsMetadata.java b/core/src/main/java/org/apache/accumulo/core/metadata/schema/TabletsMetadata.java
index f5c7bf1..8c80617 100644
--- a/core/src/main/java/org/apache/accumulo/core/metadata/schema/TabletsMetadata.java
+++ b/core/src/main/java/org/apache/accumulo/core/metadata/schema/TabletsMetadata.java
@@ -35,9 +35,8 @@ import org.apache.accumulo.core.client.AccumuloClient;
 import org.apache.accumulo.core.client.IsolatedScanner;
 import org.apache.accumulo.core.client.Scanner;
 import org.apache.accumulo.core.client.TableNotFoundException;
-import org.apache.accumulo.core.clientImpl.Table;
-import org.apache.accumulo.core.clientImpl.Table.ID;
 import org.apache.accumulo.core.data.Range;
+import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.dataImpl.KeyExtent;
 import org.apache.accumulo.core.metadata.MetadataTable;
 import org.apache.accumulo.core.metadata.RootTable;
@@ -73,7 +72,7 @@ public class TabletsMetadata implements Iterable<TabletMetadata>, AutoCloseable
     private Text endRow;
     private boolean checkConsistency = false;
     private boolean saveKeyValues;
-    private ID tableId;
+    private TableId tableId;
 
     @Override
     public TabletsMetadata build(AccumuloClient client) {
@@ -204,7 +203,7 @@ public class TabletsMetadata implements Iterable<TabletMetadata>, AutoCloseable
     }
 
     @Override
-    public TableRangeOptions forTable(Table.ID tableId) {
+    public TableRangeOptions forTable(TableId tableId) {
       Preconditions.checkArgument(!tableId.equals(RootTable.ID),
           "Getting tablet metadata for " + RootTable.NAME + " not supported at this time.");
       if (tableId.equals(MetadataTable.ID)) {
@@ -309,7 +308,7 @@ public class TabletsMetadata implements Iterable<TabletMetadata>, AutoCloseable
      * metadata table ID is passed in then the root table is scanned. Defaults to returning all
      * tablets for the table ID.
      */
-    TableRangeOptions forTable(Table.ID tableId);
+    TableRangeOptions forTable(TableId tableId);
 
     /**
      * Obtain tablet metadata by scanning the metadata table. Defaults to the range
diff --git a/core/src/main/java/org/apache/accumulo/core/replication/ReplicationSchema.java b/core/src/main/java/org/apache/accumulo/core/replication/ReplicationSchema.java
index 71d941c..ff4b637 100644
--- a/core/src/main/java/org/apache/accumulo/core/replication/ReplicationSchema.java
+++ b/core/src/main/java/org/apache/accumulo/core/replication/ReplicationSchema.java
@@ -24,11 +24,11 @@ import java.nio.charset.CharacterCodingException;
 
 import org.apache.accumulo.core.client.ScannerBase;
 import org.apache.accumulo.core.client.lexicoder.ULongLexicoder;
-import org.apache.accumulo.core.clientImpl.Table;
 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.Mutation;
+import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.data.Value;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.Text;
@@ -103,9 +103,9 @@ public class ReplicationSchema {
      *          Key to extract from
      * @return The table ID
      */
-    public static Table.ID getTableId(Key k) {
+    public static TableId getTableId(Key k) {
       requireNonNull(k);
-      return Table.ID.of(k.getColumnQualifier().toString());
+      return TableId.of(k.getColumnQualifier().toString());
     }
 
     /**
@@ -132,8 +132,8 @@ public class ReplicationSchema {
       scanner.fetchColumnFamily(NAME);
     }
 
-    public static Mutation add(Mutation m, Table.ID tableId, Value v) {
-      m.put(NAME, new Text(tableId.getUtf8()), v);
+    public static Mutation add(Mutation m, TableId tableId, Value v) {
+      m.put(NAME, new Text(tableId.canonical()), v);
       return m;
     }
   }
@@ -219,8 +219,8 @@ public class ReplicationSchema {
      *          Serialized Status msg
      * @return The original Mutation
      */
-    public static Mutation add(Mutation m, Table.ID tableId, Value v) {
-      m.put(NAME, new Text(tableId.getUtf8()), v);
+    public static Mutation add(Mutation m, TableId tableId, Value v) {
+      m.put(NAME, new Text(tableId.canonical()), v);
       return m;
     }
 
diff --git a/core/src/main/java/org/apache/accumulo/core/replication/ReplicationTable.java b/core/src/main/java/org/apache/accumulo/core/replication/ReplicationTable.java
index 53fb248..d04c952 100644
--- a/core/src/main/java/org/apache/accumulo/core/replication/ReplicationTable.java
+++ b/core/src/main/java/org/apache/accumulo/core/replication/ReplicationTable.java
@@ -31,8 +31,8 @@ import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.client.TableOfflineException;
 import org.apache.accumulo.core.clientImpl.ClientContext;
 import org.apache.accumulo.core.clientImpl.Namespace;
-import org.apache.accumulo.core.clientImpl.Table;
 import org.apache.accumulo.core.clientImpl.Tables;
+import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.master.state.tables.TableState;
 import org.apache.accumulo.core.replication.ReplicationSchema.StatusSection;
 import org.apache.accumulo.core.replication.ReplicationSchema.WorkSection;
@@ -46,8 +46,8 @@ import com.google.common.collect.ImmutableMap;
 public class ReplicationTable {
   private static final Logger log = LoggerFactory.getLogger(ReplicationTable.class);
 
-  public static final Table.ID ID = Table.ID.of("+rep");
-  public static final String NAME = Namespace.ACCUMULO + ".replication";
+  public static final TableId ID = TableId.of("+rep");
+  public static final String NAME = Namespace.ACCUMULO.name() + ".replication";
 
   public static final String COMBINER_NAME = "statuscombiner";
 
diff --git a/core/src/main/java/org/apache/accumulo/core/replication/ReplicationTarget.java b/core/src/main/java/org/apache/accumulo/core/replication/ReplicationTarget.java
index ae133db..8f8ee46 100644
--- a/core/src/main/java/org/apache/accumulo/core/replication/ReplicationTarget.java
+++ b/core/src/main/java/org/apache/accumulo/core/replication/ReplicationTarget.java
@@ -22,7 +22,7 @@ import java.io.DataInput;
 import java.io.DataOutput;
 import java.io.IOException;
 
-import org.apache.accumulo.core.clientImpl.Table;
+import org.apache.accumulo.core.data.TableId;
 import org.apache.hadoop.io.DataInputBuffer;
 import org.apache.hadoop.io.DataOutputBuffer;
 import org.apache.hadoop.io.Text;
@@ -36,11 +36,11 @@ public class ReplicationTarget implements Writable {
 
   private String peerName;
   private String remoteIdentifier;
-  private Table.ID sourceTableId;
+  private TableId sourceTableId;
 
   public ReplicationTarget() {}
 
-  public ReplicationTarget(String peerName, String remoteIdentifier, Table.ID sourceTableId) {
+  public ReplicationTarget(String peerName, String remoteIdentifier, TableId sourceTableId) {
     this.peerName = peerName;
     this.remoteIdentifier = remoteIdentifier;
     this.sourceTableId = sourceTableId;
@@ -62,11 +62,11 @@ public class ReplicationTarget implements Writable {
     this.remoteIdentifier = remoteIdentifier;
   }
 
-  public Table.ID getSourceTableId() {
+  public TableId getSourceTableId() {
     return sourceTableId;
   }
 
-  public void setSourceTableId(Table.ID sourceTableId) {
+  public void setSourceTableId(TableId sourceTableId) {
     this.sourceTableId = sourceTableId;
   }
 
@@ -90,7 +90,7 @@ public class ReplicationTarget implements Writable {
       out.writeBoolean(false);
     } else {
       out.writeBoolean(true);
-      WritableUtils.writeString(out, sourceTableId.canonicalID());
+      WritableUtils.writeString(out, sourceTableId.canonical());
     }
   }
 
@@ -103,7 +103,7 @@ public class ReplicationTarget implements Writable {
       this.remoteIdentifier = WritableUtils.readString(in);
     }
     if (in.readBoolean()) {
-      this.sourceTableId = Table.ID.of(WritableUtils.readString(in));
+      this.sourceTableId = TableId.of(WritableUtils.readString(in));
     }
   }
 
diff --git a/core/src/main/java/org/apache/accumulo/core/summary/Gatherer.java b/core/src/main/java/org/apache/accumulo/core/summary/Gatherer.java
index 7fe5d0e..6461468 100644
--- a/core/src/main/java/org/apache/accumulo/core/summary/Gatherer.java
+++ b/core/src/main/java/org/apache/accumulo/core/summary/Gatherer.java
@@ -46,11 +46,11 @@ import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.summary.SummarizerConfiguration;
 import org.apache.accumulo.core.clientImpl.ClientContext;
 import org.apache.accumulo.core.clientImpl.ServerClient;
-import org.apache.accumulo.core.clientImpl.Table;
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
 import org.apache.accumulo.core.data.ByteSequence;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Range;
+import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.dataImpl.KeyExtent;
 import org.apache.accumulo.core.dataImpl.thrift.TRowRange;
 import org.apache.accumulo.core.dataImpl.thrift.TSummaries;
@@ -104,7 +104,7 @@ public class Gatherer {
   private static final Logger log = LoggerFactory.getLogger(Gatherer.class);
 
   private ClientContext ctx;
-  private Table.ID tableId;
+  private TableId tableId;
   private SummarizerFactory factory;
   private Text startRow = null;
   private Text endRow = null;
@@ -121,7 +121,7 @@ public class Gatherer {
   public Gatherer(ClientContext context, TSummaryRequest request, AccumuloConfiguration tableConfig,
       CryptoService cryptoService) {
     this.ctx = context;
-    this.tableId = Table.ID.of(request.tableId);
+    this.tableId = TableId.of(request.tableId);
     this.startRow = ByteBufferUtil.toText(request.bounds.startRow);
     this.endRow = ByteBufferUtil.toText(request.bounds.endRow);
     this.clipRange = new Range(startRow, false, endRow, true);
diff --git a/core/src/main/java/org/apache/accumulo/core/util/ByteBufferUtil.java b/core/src/main/java/org/apache/accumulo/core/util/ByteBufferUtil.java
index 3d73702..d0322b0 100644
--- a/core/src/main/java/org/apache/accumulo/core/util/ByteBufferUtil.java
+++ b/core/src/main/java/org/apache/accumulo/core/util/ByteBufferUtil.java
@@ -27,8 +27,8 @@ import java.util.Arrays;
 import java.util.Collection;
 import java.util.List;
 
-import org.apache.accumulo.core.clientImpl.Table;
 import org.apache.accumulo.core.data.ByteSequence;
+import org.apache.accumulo.core.data.TableId;
 import org.apache.hadoop.io.Text;
 
 public class ByteBufferUtil {
@@ -90,8 +90,8 @@ public class ByteBufferUtil {
     }
   }
 
-  public static Table.ID toTableId(ByteBuffer bytes) {
-    return Table.ID.of(toString(bytes));
+  public static TableId toTableId(ByteBuffer bytes) {
+    return TableId.of(toString(bytes));
   }
 
   public static ByteBuffer toByteBuffers(ByteSequence bs) {
diff --git a/core/src/main/java/org/apache/accumulo/core/util/LocalityGroupUtil.java b/core/src/main/java/org/apache/accumulo/core/util/LocalityGroupUtil.java
index 98568a0..8b4163c 100644
--- a/core/src/main/java/org/apache/accumulo/core/util/LocalityGroupUtil.java
+++ b/core/src/main/java/org/apache/accumulo/core/util/LocalityGroupUtil.java
@@ -30,7 +30,6 @@ import java.util.SortedSet;
 import java.util.TreeSet;
 
 import org.apache.accumulo.core.client.AccumuloException;
-import org.apache.accumulo.core.clientImpl.Table;
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
 import org.apache.accumulo.core.conf.ConfigurationCopy;
 import org.apache.accumulo.core.conf.Property;
@@ -40,6 +39,7 @@ import org.apache.accumulo.core.data.Column;
 import org.apache.accumulo.core.data.ColumnUpdate;
 import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.data.Range;
+import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.dataImpl.thrift.TMutation;
 import org.apache.accumulo.core.file.FileSKVIterator;
 import org.apache.accumulo.core.file.rfile.RFile.Reader;
@@ -97,7 +97,7 @@ public class LocalityGroupUtil {
   }
 
   public static Map<String,Set<ByteSequence>> getLocalityGroupsIgnoringErrors(
-      AccumuloConfiguration acuconf, Table.ID tableId) {
+      AccumuloConfiguration acuconf, TableId tableId) {
     try {
       return getLocalityGroups(acuconf);
     } catch (LocalityGroupConfigurationError | RuntimeException e) {
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 9786ccb..1fb6bc8 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
@@ -23,11 +23,11 @@ import java.util.List;
 import org.apache.accumulo.core.cli.ClientOnRequiredTable;
 import org.apache.accumulo.core.client.AccumuloClient;
 import org.apache.accumulo.core.clientImpl.ClientContext;
-import org.apache.accumulo.core.clientImpl.Table;
 import org.apache.accumulo.core.clientImpl.Tables;
 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.TableId;
 import org.apache.accumulo.core.dataImpl.KeyExtent;
 import org.apache.accumulo.core.metadata.MetadataTable;
 import org.apache.accumulo.core.metadata.schema.DataFileValue;
@@ -212,7 +212,7 @@ public class Merge {
       Text end) throws MergeException {
     // open up metadata, walk through the tablets.
 
-    Table.ID tableId;
+    TableId tableId;
     TabletsMetadata tablets;
     try {
       ClientContext context = (ClientContext) client;
diff --git a/core/src/test/java/org/apache/accumulo/core/clientImpl/ScannerImplTest.java b/core/src/test/java/org/apache/accumulo/core/clientImpl/ScannerImplTest.java
index 542faee..adba1f3 100644
--- a/core/src/test/java/org/apache/accumulo/core/clientImpl/ScannerImplTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/clientImpl/ScannerImplTest.java
@@ -19,6 +19,7 @@ package org.apache.accumulo.core.clientImpl;
 import static org.junit.Assert.assertEquals;
 
 import org.apache.accumulo.core.client.Scanner;
+import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.security.Authorizations;
 import org.easymock.EasyMock;
 import org.junit.Before;
@@ -35,7 +36,7 @@ public class ScannerImplTest {
 
   @Test
   public void testValidReadaheadValues() {
-    Scanner s = new ScannerImpl(context, Table.ID.of("foo"), Authorizations.EMPTY);
+    Scanner s = new ScannerImpl(context, TableId.of("foo"), Authorizations.EMPTY);
     s.setReadaheadThreshold(0);
     s.setReadaheadThreshold(10);
     s.setReadaheadThreshold(Long.MAX_VALUE);
@@ -46,7 +47,7 @@ public class ScannerImplTest {
 
   @Test(expected = IllegalArgumentException.class)
   public void testInValidReadaheadValues() {
-    Scanner s = new ScannerImpl(context, Table.ID.of("foo"), Authorizations.EMPTY);
+    Scanner s = new ScannerImpl(context, TableId.of("foo"), Authorizations.EMPTY);
     s.setReadaheadThreshold(-1);
     s.close();
   }
@@ -54,7 +55,7 @@ public class ScannerImplTest {
   @Test
   public void testGetAuthorizations() {
     Authorizations expected = new Authorizations("a,b");
-    Scanner s = new ScannerImpl(context, Table.ID.of("foo"), expected);
+    Scanner s = new ScannerImpl(context, TableId.of("foo"), expected);
     assertEquals(expected, s.getAuthorizations());
     s.close();
   }
@@ -62,7 +63,7 @@ public class ScannerImplTest {
   @SuppressWarnings("resource")
   @Test(expected = IllegalArgumentException.class)
   public void testNullAuthorizationsFails() {
-    new ScannerImpl(context, Table.ID.of("foo"), null);
+    new ScannerImpl(context, TableId.of("foo"), null);
   }
 
 }
diff --git a/core/src/test/java/org/apache/accumulo/core/clientImpl/TableTest.java b/core/src/test/java/org/apache/accumulo/core/clientImpl/TableTest.java
deleted file mode 100644
index eaa0435..0000000
--- a/core/src/test/java/org/apache/accumulo/core/clientImpl/TableTest.java
+++ /dev/null
@@ -1,92 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.accumulo.core.clientImpl;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertSame;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.TestName;
-
-/**
- * Tests the Table ID class, mainly the internal cache.
- */
-public class TableTest {
-  @Rule
-  public TestName name = new TestName();
-
-  @Test
-  public void testCacheIncreases() {
-    Long initialSize = Table.ID.cache.asMap().entrySet().stream().count();
-    String tableString = "table-" + name.getMethodName();
-    Table.ID table1 = Table.ID.of(tableString);
-    assertEquals(initialSize + 1, Table.ID.cache.asMap().entrySet().stream().count());
-    assertEquals(tableString, table1.canonicalID());
-  }
-
-  @Test
-  public void testCacheNoDuplicates() {
-    String tableString = "table-" + name.getMethodName();
-    Long initialSize = Table.ID.cache.asMap().entrySet().stream().count();
-    Table.ID table1 = Table.ID.of(tableString);
-    assertEquals(initialSize + 1, Table.ID.cache.asMap().entrySet().stream().count());
-    assertEquals(tableString, table1.canonicalID());
-
-    // ensure duplicates are not created
-    Table.ID builtInTableId = Table.ID.of("!0");
-    assertSame(Table.ID.METADATA, builtInTableId);
-    builtInTableId = Table.ID.of("+r");
-    assertSame(Table.ID.ROOT, builtInTableId);
-    builtInTableId = Table.ID.of("+rep");
-    assertSame(Table.ID.REPLICATION, builtInTableId);
-    table1 = Table.ID.of(tableString);
-    assertEquals(initialSize + 1, Table.ID.cache.asMap().entrySet().stream().count());
-    assertEquals(tableString, table1.canonicalID());
-    Table.ID table2 = Table.ID.of(tableString);
-    assertEquals(initialSize + 1, Table.ID.cache.asMap().entrySet().stream().count());
-    assertEquals(tableString, table2.canonicalID());
-    assertSame(table1, table2);
-  }
-
-  @Test(timeout = 60_000)
-  public void testCacheDecreasesAfterGC() {
-    Long initialSize = Table.ID.cache.asMap().entrySet().stream().count();
-    generateJunkCacheEntries();
-    Long postGCSize;
-    do {
-      System.gc();
-      try {
-        Thread.sleep(500);
-      } catch (InterruptedException e) {
-        fail("Thread interrupted while waiting for GC");
-      }
-      postGCSize = Table.ID.cache.asMap().entrySet().stream().count();
-    } while (postGCSize > initialSize);
-
-    assertTrue("Cache did not decrease with GC.",
-        Table.ID.cache.asMap().entrySet().stream().count() < initialSize);
-  }
-
-  private void generateJunkCacheEntries() {
-    for (int i = 0; i < 1000; i++)
-      Table.ID.of(new String("table" + i));
-  }
-
-}
diff --git a/core/src/test/java/org/apache/accumulo/core/clientImpl/TabletLocatorImplTest.java b/core/src/test/java/org/apache/accumulo/core/clientImpl/TabletLocatorImplTest.java
index 4ef8b88..eb46597 100644
--- a/core/src/test/java/org/apache/accumulo/core/clientImpl/TabletLocatorImplTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/clientImpl/TabletLocatorImplTest.java
@@ -43,6 +43,7 @@ import org.apache.accumulo.core.data.Key;
 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.TableId;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.dataImpl.KeyExtent;
 import org.apache.accumulo.core.metadata.MetadataLocationObtainer;
@@ -60,7 +61,7 @@ public class TabletLocatorImplTest {
   private static final KeyExtent MTE = new KeyExtent(MetadataTable.ID, null, RTE.getEndRow());
 
   static KeyExtent nke(String t, String er, String per) {
-    return new KeyExtent(Table.ID.of(t), er == null ? null : new Text(er),
+    return new KeyExtent(TableId.of(t), er == null ? null : new Text(er),
         per == null ? null : new Text(per));
   }
 
@@ -141,7 +142,7 @@ public class TabletLocatorImplTest {
     RootTabletLocator rtl = new TestRootTabletLocator();
     TabletLocatorImpl rootTabletCache = new TabletLocatorImpl(MetadataTable.ID, rtl, ttlo,
         new YesLockChecker());
-    TabletLocatorImpl tab1TabletCache = new TabletLocatorImpl(Table.ID.of(table), rootTabletCache,
+    TabletLocatorImpl tab1TabletCache = new TabletLocatorImpl(TableId.of(table), rootTabletCache,
         ttlo, tslc);
 
     setLocation(tservers, rootTabLoc, RTE, MTE, metaTabLoc);
@@ -221,8 +222,8 @@ public class TabletLocatorImplTest {
   static Mutation nm(String row, String... data) {
     Mutation mut = new Mutation(new Text(row));
 
-    for (int i = 0; i < data.length; i++) {
-      String[] cvp = data[i].split("=");
+    for (String element : data) {
+      String[] cvp = element.split("=");
       String[] cols = cvp[0].split(":");
 
       mut.put(new Text(cols[0]), new Text(cols[1]), new Value(cvp[1].getBytes()));
@@ -624,7 +625,7 @@ public class TabletLocatorImplTest {
     RootTabletLocator rtl = new TestRootTabletLocator();
     TabletLocatorImpl rootTabletCache = new TabletLocatorImpl(MetadataTable.ID, rtl, ttlo,
         new YesLockChecker());
-    TabletLocatorImpl tab1TabletCache = new TabletLocatorImpl(Table.ID.of("tab1"), rootTabletCache,
+    TabletLocatorImpl tab1TabletCache = new TabletLocatorImpl(TableId.of("tab1"), rootTabletCache,
         ttlo, new YesLockChecker());
 
     locateTabletTest(tab1TabletCache, "r1", null, null);
@@ -1208,15 +1209,15 @@ public class TabletLocatorImplTest {
     RootTabletLocator rtl = new TestRootTabletLocator();
     TabletLocatorImpl rootTabletCache = new TabletLocatorImpl(MetadataTable.ID, rtl, ttlo,
         new YesLockChecker());
-    TabletLocatorImpl tab0TabletCache = new TabletLocatorImpl(Table.ID.of("0"), rootTabletCache,
+    TabletLocatorImpl tab0TabletCache = new TabletLocatorImpl(TableId.of("0"), rootTabletCache,
         ttlo, new YesLockChecker());
 
     setLocation(tservers, "tserver1", RTE, mte1, "tserver2");
     setLocation(tservers, "tserver1", RTE, mte2, "tserver3");
 
     // create two tablets that straddle a metadata split point
-    KeyExtent ke1 = new KeyExtent(Table.ID.of("0"), new Text("0bbf20e"), null);
-    KeyExtent ke2 = new KeyExtent(Table.ID.of("0"), new Text("0bc0756"), new Text("0bbf20e"));
+    KeyExtent ke1 = new KeyExtent(TableId.of("0"), new Text("0bbf20e"), null);
+    KeyExtent ke2 = new KeyExtent(TableId.of("0"), new Text("0bc0756"), new Text("0bbf20e"));
 
     setLocation(tservers, "tserver2", mte1, ke1, "tserver4");
     setLocation(tservers, "tserver3", mte2, ke2, "tserver5");
@@ -1237,7 +1238,7 @@ public class TabletLocatorImplTest {
     RootTabletLocator rtl = new TestRootTabletLocator();
     TabletLocatorImpl rootTabletCache = new TabletLocatorImpl(MetadataTable.ID, rtl, ttlo,
         new YesLockChecker());
-    TabletLocatorImpl tab0TabletCache = new TabletLocatorImpl(Table.ID.of("0"), rootTabletCache,
+    TabletLocatorImpl tab0TabletCache = new TabletLocatorImpl(TableId.of("0"), rootTabletCache,
         ttlo, new YesLockChecker());
 
     setLocation(tservers, "tserver1", RTE, mte1, "tserver2");
@@ -1262,7 +1263,7 @@ public class TabletLocatorImplTest {
     KeyExtent mte4 = new KeyExtent(MetadataTable.ID, new Text("1;r"), new Text("1;j"));
     KeyExtent mte5 = new KeyExtent(MetadataTable.ID, null, new Text("1;r"));
 
-    KeyExtent ke1 = new KeyExtent(Table.ID.of("1"), null, null);
+    KeyExtent ke1 = new KeyExtent(TableId.of("1"), null, null);
 
     TServers tservers = new TServers();
     TestTabletLocationObtainer ttlo = new TestTabletLocationObtainer(tservers);
@@ -1271,7 +1272,7 @@ public class TabletLocatorImplTest {
 
     TabletLocatorImpl rootTabletCache = new TabletLocatorImpl(MetadataTable.ID, rtl, ttlo,
         new YesLockChecker());
-    TabletLocatorImpl tab0TabletCache = new TabletLocatorImpl(Table.ID.of("1"), rootTabletCache,
+    TabletLocatorImpl tab0TabletCache = new TabletLocatorImpl(TableId.of("1"), rootTabletCache,
         ttlo, new YesLockChecker());
 
     setLocation(tservers, "tserver1", RTE, mte1, "tserver2");
diff --git a/core/src/test/java/org/apache/accumulo/core/clientImpl/TabletServerBatchReaderTest.java b/core/src/test/java/org/apache/accumulo/core/clientImpl/TabletServerBatchReaderTest.java
index 035ec3e..cd0203c 100644
--- a/core/src/test/java/org/apache/accumulo/core/clientImpl/TabletServerBatchReaderTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/clientImpl/TabletServerBatchReaderTest.java
@@ -19,6 +19,7 @@ package org.apache.accumulo.core.clientImpl;
 import static org.junit.Assert.assertEquals;
 
 import org.apache.accumulo.core.client.BatchScanner;
+import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.security.Authorizations;
 import org.easymock.EasyMock;
 import org.junit.Before;
@@ -36,7 +37,7 @@ public class TabletServerBatchReaderTest {
   @Test
   public void testGetAuthorizations() {
     Authorizations expected = new Authorizations("a,b");
-    try (BatchScanner s = new TabletServerBatchReader(context, Table.ID.of("foo"), expected, 1)) {
+    try (BatchScanner s = new TabletServerBatchReader(context, TableId.of("foo"), expected, 1)) {
       assertEquals(expected, s.getAuthorizations());
     }
   }
@@ -44,6 +45,6 @@ public class TabletServerBatchReaderTest {
   @SuppressWarnings("resource")
   @Test(expected = IllegalArgumentException.class)
   public void testNullAuthorizationsFails() {
-    new TabletServerBatchReader(context, Table.ID.of("foo"), null, 1);
+    new TabletServerBatchReader(context, TableId.of("foo"), null, 1);
   }
 }
diff --git a/core/src/test/java/org/apache/accumulo/core/clientImpl/bulk/BulkSerializeTest.java b/core/src/test/java/org/apache/accumulo/core/clientImpl/bulk/BulkSerializeTest.java
index 25e1d9f..a8896f8 100644
--- a/core/src/test/java/org/apache/accumulo/core/clientImpl/bulk/BulkSerializeTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/clientImpl/bulk/BulkSerializeTest.java
@@ -26,10 +26,10 @@ import java.util.SortedMap;
 import java.util.TreeMap;
 
 import org.apache.accumulo.core.Constants;
-import org.apache.accumulo.core.clientImpl.Table;
 import org.apache.accumulo.core.clientImpl.bulk.Bulk.FileInfo;
 import org.apache.accumulo.core.clientImpl.bulk.Bulk.Files;
 import org.apache.accumulo.core.clientImpl.bulk.BulkSerialize.Input;
+import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.dataImpl.KeyExtent;
 import org.apache.hadoop.io.Text;
 import org.junit.Test;
@@ -38,7 +38,7 @@ public class BulkSerializeTest {
 
   @Test
   public void writeReadLoadMapping() throws Exception {
-    Table.ID tableId = Table.ID.of("3");
+    TableId tableId = TableId.of("3");
     SortedMap<KeyExtent,Bulk.Files> mapping = generateMapping(tableId);
 
     ByteArrayOutputStream baos = new ByteArrayOutputStream();
@@ -75,7 +75,7 @@ public class BulkSerializeTest {
 
   @Test
   public void testRemap() throws Exception {
-    Table.ID tableId = Table.ID.of("3");
+    TableId tableId = TableId.of("3");
     SortedMap<KeyExtent,Bulk.Files> mapping = generateMapping(tableId);
 
     SortedMap<KeyExtent,Bulk.Files> newNameMapping = new TreeMap<>();
@@ -117,7 +117,7 @@ public class BulkSerializeTest {
 
   }
 
-  public SortedMap<KeyExtent,Bulk.Files> generateMapping(Table.ID tableId) {
+  public SortedMap<KeyExtent,Bulk.Files> generateMapping(TableId tableId) {
     SortedMap<KeyExtent,Bulk.Files> mapping = new TreeMap<>();
     Bulk.Files testFiles = new Bulk.Files();
     Bulk.Files testFiles2 = new Bulk.Files();
diff --git a/core/src/test/java/org/apache/accumulo/core/clientImpl/bulk/ConcurrentKeyExtentCacheTest.java b/core/src/test/java/org/apache/accumulo/core/clientImpl/bulk/ConcurrentKeyExtentCacheTest.java
index 3da8b55..859448a 100644
--- a/core/src/test/java/org/apache/accumulo/core/clientImpl/bulk/ConcurrentKeyExtentCacheTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/clientImpl/bulk/ConcurrentKeyExtentCacheTest.java
@@ -32,7 +32,7 @@ import java.util.stream.Stream;
 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.clientImpl.Table;
+import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.dataImpl.KeyExtent;
 import org.apache.hadoop.io.Text;
 import org.junit.BeforeClass;
@@ -50,11 +50,11 @@ public class ConcurrentKeyExtentCacheTest {
     Text prev = null;
     for (int i = 1; i < 255; i++) {
       Text endRow = new Text(String.format("%02x", i));
-      extents.add(new KeyExtent(Table.ID.of("1"), endRow, prev));
+      extents.add(new KeyExtent(TableId.of("1"), endRow, prev));
       prev = endRow;
     }
 
-    extents.add(new KeyExtent(Table.ID.of("1"), null, prev));
+    extents.add(new KeyExtent(TableId.of("1"), null, prev));
 
     extentsSet.addAll(extents);
   }
diff --git a/core/src/test/java/org/apache/accumulo/core/clientImpl/mapreduce/BatchInputSplitTest.java b/core/src/test/java/org/apache/accumulo/core/clientImpl/mapreduce/BatchInputSplitTest.java
index 12ee754..8193732 100644
--- a/core/src/test/java/org/apache/accumulo/core/clientImpl/mapreduce/BatchInputSplitTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/clientImpl/mapreduce/BatchInputSplitTest.java
@@ -33,9 +33,9 @@ import java.util.List;
 import java.util.Set;
 
 import org.apache.accumulo.core.client.IteratorSetting;
-import org.apache.accumulo.core.clientImpl.Table;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Range;
+import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.iterators.user.SummingCombiner;
 import org.apache.accumulo.core.iterators.user.WholeRowIterator;
 import org.apache.accumulo.core.util.Pair;
@@ -52,7 +52,7 @@ public class BatchInputSplitTest {
   @Test
   public void testSimpleWritable() throws IOException {
     List<Range> ranges = Collections.singletonList(new Range(new Key("a"), new Key("b")));
-    BatchInputSplit split = new BatchInputSplit("table", Table.ID.of("1"), ranges,
+    BatchInputSplit split = new BatchInputSplit("table", TableId.of("1"), ranges,
         new String[] {"localhost"});
 
     ByteArrayOutputStream baos = new ByteArrayOutputStream();
@@ -74,7 +74,7 @@ public class BatchInputSplitTest {
   @Test
   public void testAllFieldsWritable() throws IOException {
     List<Range> ranges = Collections.singletonList(new Range(new Key("a"), new Key("b")));
-    BatchInputSplit split = new BatchInputSplit("table", Table.ID.of("1"), ranges,
+    BatchInputSplit split = new BatchInputSplit("table", TableId.of("1"), ranges,
         new String[] {"localhost"});
 
     Set<Pair<Text,Text>> fetchedColumns = new HashSet<>();
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 7b64205..bdeebd3 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
@@ -30,7 +30,6 @@ import java.util.SortedMap;
 import java.util.TreeMap;
 import java.util.TreeSet;
 
-import org.apache.accumulo.core.clientImpl.Table;
 import org.apache.accumulo.core.dataImpl.KeyExtent;
 import org.apache.hadoop.io.Text;
 import org.junit.Before;
@@ -38,7 +37,7 @@ import org.junit.Test;
 
 public class KeyExtentTest {
   KeyExtent nke(String t, String er, String per) {
-    return new KeyExtent(Table.ID.of(t), er == null ? null : new Text(er),
+    return new KeyExtent(TableId.of(t), er == null ? null : new Text(er),
         per == null ? null : new Text(per));
   }
 
@@ -57,7 +56,7 @@ public class KeyExtentTest {
     ke = new KeyExtent(flattenedExtent, (Text) null);
 
     assertEquals(new Text("bar"), ke.getEndRow());
-    assertEquals("foo", ke.getTableId().canonicalID());
+    assertEquals("foo", ke.getTableId().canonical());
     assertNull(ke.getPrevEndRow());
 
     flattenedExtent = new Text("foo<");
@@ -65,7 +64,7 @@ public class KeyExtentTest {
     ke = new KeyExtent(flattenedExtent, (Text) null);
 
     assertNull(ke.getEndRow());
-    assertEquals("foo", ke.getTableId().canonicalID());
+    assertEquals("foo", ke.getTableId().canonical());
     assertNull(ke.getPrevEndRow());
 
     flattenedExtent = new Text("foo;bar;");
@@ -73,7 +72,7 @@ public class KeyExtentTest {
     ke = new KeyExtent(flattenedExtent, (Text) null);
 
     assertEquals(new Text("bar;"), ke.getEndRow());
-    assertEquals("foo", ke.getTableId().canonicalID());
+    assertEquals("foo", ke.getTableId().canonical());
     assertNull(ke.getPrevEndRow());
 
   }
diff --git a/core/src/test/java/org/apache/accumulo/core/clientImpl/NamespaceTest.java b/core/src/test/java/org/apache/accumulo/core/data/NamespaceIdTest.java
similarity index 50%
rename from core/src/test/java/org/apache/accumulo/core/clientImpl/NamespaceTest.java
rename to core/src/test/java/org/apache/accumulo/core/data/NamespaceIdTest.java
index e86afed..3081299 100644
--- a/core/src/test/java/org/apache/accumulo/core/clientImpl/NamespaceTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/data/NamespaceIdTest.java
@@ -14,57 +14,63 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.accumulo.core.clientImpl;
+package org.apache.accumulo.core.data;
 
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotSame;
 import static org.junit.Assert.assertSame;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
+import org.apache.accumulo.core.clientImpl.Namespace;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.TestName;
 
 /**
- * Tests the Namespace ID class, mainly the internal cache.
+ * Tests the NamespaceId class, mainly the internal cache.
  */
-public class NamespaceTest {
+public class NamespaceIdTest {
   @Rule
   public TestName name = new TestName();
 
   @Test
   public void testCacheIncreases() {
     String namespaceString = "namespace-" + name.getMethodName();
-    Long initialSize = Namespace.ID.cache.asMap().entrySet().stream().count();
-    Namespace.ID nsId = Namespace.ID.of(namespaceString);
-    assertEquals(initialSize + 1, Namespace.ID.cache.asMap().entrySet().stream().count());
-    assertEquals(namespaceString, nsId.canonicalID());
+    Long initialSize = NamespaceId.cache.asMap().entrySet().stream().count();
+    NamespaceId nsId = NamespaceId.of(namespaceString);
+    assertEquals(initialSize + 1, NamespaceId.cache.asMap().entrySet().stream().count());
+    assertEquals(namespaceString, nsId.canonical());
   }
 
   @Test
   public void testCacheNoDuplicates() {
+    // the next line just preloads the built-ins, since they now exist in a separate class from
+    // NamespaceId, and aren't preloaded when the NamespaceId class is referenced
+    assertNotSame(Namespace.ACCUMULO.id(), Namespace.DEFAULT.id());
+
     String namespaceString = "namespace-" + name.getMethodName();
-    Long initialSize = Namespace.ID.cache.asMap().entrySet().stream().count();
-    Namespace.ID nsId = Namespace.ID.of(namespaceString);
-    assertEquals(initialSize + 1, Namespace.ID.cache.asMap().entrySet().stream().count());
-    assertEquals(namespaceString, nsId.canonicalID());
+    Long initialSize = NamespaceId.cache.asMap().entrySet().stream().count();
+    NamespaceId nsId = NamespaceId.of(namespaceString);
+    assertEquals(initialSize + 1, NamespaceId.cache.asMap().entrySet().stream().count());
+    assertEquals(namespaceString, nsId.canonical());
 
     // ensure duplicates are not created
-    Namespace.ID builtInNamespaceId = Namespace.ID.of("+accumulo");
-    assertSame(Namespace.ID.ACCUMULO, builtInNamespaceId);
-    builtInNamespaceId = Namespace.ID.of("+default");
-    assertSame(Namespace.ID.DEFAULT, builtInNamespaceId);
-    nsId = Namespace.ID.of(namespaceString);
-    assertEquals(initialSize + 1, Namespace.ID.cache.asMap().entrySet().stream().count());
-    assertEquals(namespaceString, nsId.canonicalID());
-    Namespace.ID nsId2 = Namespace.ID.of(namespaceString);
-    assertEquals(initialSize + 1, Namespace.ID.cache.asMap().entrySet().stream().count());
+    NamespaceId builtInNamespaceId = NamespaceId.of("+accumulo");
+    assertSame(Namespace.ACCUMULO.id(), builtInNamespaceId);
+    builtInNamespaceId = NamespaceId.of("+default");
+    assertSame(Namespace.DEFAULT.id(), builtInNamespaceId);
+    nsId = NamespaceId.of(namespaceString);
+    assertEquals(initialSize + 1, NamespaceId.cache.asMap().entrySet().stream().count());
+    assertEquals(namespaceString, nsId.canonical());
+    NamespaceId nsId2 = NamespaceId.of(namespaceString);
+    assertEquals(initialSize + 1, NamespaceId.cache.asMap().entrySet().stream().count());
     assertSame(nsId, nsId2);
   }
 
   @Test(timeout = 60_000)
   public void testCacheDecreasesAfterGC() {
-    Long initialSize = Namespace.ID.cache.asMap().entrySet().stream().count();
+    Long initialSize = NamespaceId.cache.asMap().entrySet().stream().count();
     generateJunkCacheEntries();
     Long postGCSize;
     do {
@@ -74,15 +80,15 @@ public class NamespaceTest {
       } catch (InterruptedException e) {
         fail("Thread interrupted while waiting for GC");
       }
-      postGCSize = Namespace.ID.cache.asMap().entrySet().stream().count();
+      postGCSize = NamespaceId.cache.asMap().entrySet().stream().count();
     } while (postGCSize > initialSize);
 
     assertTrue("Cache did not decrease with GC.",
-        Namespace.ID.cache.asMap().entrySet().stream().count() < initialSize);
+        NamespaceId.cache.asMap().entrySet().stream().count() < initialSize);
   }
 
   private void generateJunkCacheEntries() {
     for (int i = 0; i < 1000; i++)
-      Namespace.ID.of(new String("namespace" + i));
+      NamespaceId.of(new String("namespace" + i));
   }
 }
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 f6f34ec..307b8f7 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
@@ -31,7 +31,6 @@ import java.util.Arrays;
 import java.util.HashSet;
 import java.util.List;
 
-import org.apache.accumulo.core.clientImpl.Table;
 import org.apache.accumulo.core.dataImpl.KeyExtent;
 import org.apache.accumulo.core.dataImpl.thrift.TRange;
 import org.apache.hadoop.io.Text;
@@ -228,39 +227,39 @@ public class RangeTest {
   @Test
   public void testMergeOverlapping22() {
 
-    Range ke1 = new KeyExtent(Table.ID.of("tab1"), new Text("Bank"), null).toMetadataRange();
-    Range ke2 = new KeyExtent(Table.ID.of("tab1"), new Text("Fails"), new Text("Bank"))
+    Range ke1 = new KeyExtent(TableId.of("tab1"), new Text("Bank"), null).toMetadataRange();
+    Range ke2 = new KeyExtent(TableId.of("tab1"), new Text("Fails"), new Text("Bank"))
         .toMetadataRange();
-    Range ke3 = new KeyExtent(Table.ID.of("tab1"), new Text("Sam"), new Text("Fails"))
+    Range ke3 = new KeyExtent(TableId.of("tab1"), new Text("Sam"), new Text("Fails"))
         .toMetadataRange();
-    Range ke4 = new KeyExtent(Table.ID.of("tab1"), new Text("bails"), new Text("Sam"))
+    Range ke4 = new KeyExtent(TableId.of("tab1"), new Text("bails"), new Text("Sam"))
         .toMetadataRange();
-    Range ke5 = new KeyExtent(Table.ID.of("tab1"), null, new Text("bails")).toMetadataRange();
+    Range ke5 = new KeyExtent(TableId.of("tab1"), null, new Text("bails")).toMetadataRange();
 
     List<Range> rl = newRangeList(ke1, ke2, ke3, ke4, ke5);
     List<Range> expected = newRangeList(
-        new KeyExtent(Table.ID.of("tab1"), null, null).toMetadataRange());
+        new KeyExtent(TableId.of("tab1"), null, null).toMetadataRange());
     check(Range.mergeOverlapping(rl), expected);
 
     rl = newRangeList(ke1, ke2, ke4, ke5);
     expected = newRangeList(
-        new KeyExtent(Table.ID.of("tab1"), new Text("Fails"), null).toMetadataRange(),
-        new KeyExtent(Table.ID.of("tab1"), null, new Text("Sam")).toMetadataRange());
+        new KeyExtent(TableId.of("tab1"), new Text("Fails"), null).toMetadataRange(),
+        new KeyExtent(TableId.of("tab1"), null, new Text("Sam")).toMetadataRange());
     check(Range.mergeOverlapping(rl), expected);
 
     rl = newRangeList(ke2, ke3, ke4, ke5);
     expected = newRangeList(
-        new KeyExtent(Table.ID.of("tab1"), null, new Text("Bank")).toMetadataRange());
+        new KeyExtent(TableId.of("tab1"), null, new Text("Bank")).toMetadataRange());
     check(Range.mergeOverlapping(rl), expected);
 
     rl = newRangeList(ke1, ke2, ke3, ke4);
     expected = newRangeList(
-        new KeyExtent(Table.ID.of("tab1"), new Text("bails"), null).toMetadataRange());
+        new KeyExtent(TableId.of("tab1"), new Text("bails"), null).toMetadataRange());
     check(Range.mergeOverlapping(rl), expected);
 
     rl = newRangeList(ke2, ke3, ke4);
     expected = newRangeList(
-        new KeyExtent(Table.ID.of("tab1"), new Text("bails"), new Text("Bank")).toMetadataRange());
+        new KeyExtent(TableId.of("tab1"), new Text("bails"), new Text("Bank")).toMetadataRange());
 
     check(Range.mergeOverlapping(rl), expected);
   }
diff --git a/core/src/test/java/org/apache/accumulo/core/data/TableIdTest.java b/core/src/test/java/org/apache/accumulo/core/data/TableIdTest.java
new file mode 100644
index 0000000..b88c68d
--- /dev/null
+++ b/core/src/test/java/org/apache/accumulo/core/data/TableIdTest.java
@@ -0,0 +1,101 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.accumulo.core.data;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotSame;
+import static org.junit.Assert.assertSame;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import org.apache.accumulo.core.metadata.MetadataTable;
+import org.apache.accumulo.core.metadata.RootTable;
+import org.apache.accumulo.core.replication.ReplicationTable;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TestName;
+
+/**
+ * Tests the Table ID class, mainly the internal cache.
+ */
+public class TableIdTest {
+  @Rule
+  public TestName name = new TestName();
+
+  @Test
+  public void testCacheIncreases() {
+    Long initialSize = TableId.cache.asMap().entrySet().stream().count();
+    String tableString = "table-" + name.getMethodName();
+    TableId table1 = TableId.of(tableString);
+    assertEquals(initialSize + 1, TableId.cache.asMap().entrySet().stream().count());
+    assertEquals(tableString, table1.canonical());
+  }
+
+  @Test
+  public void testCacheNoDuplicates() {
+    // the next two lines just preloads the built-ins, since they now exist in a separate class from
+    // TableId, and aren't preloaded when the TableId class is referenced
+    assertNotSame(RootTable.ID, MetadataTable.ID);
+    assertNotSame(RootTable.ID, ReplicationTable.ID);
+
+    String tableString = "table-" + name.getMethodName();
+    Long initialSize = TableId.cache.asMap().entrySet().stream().count();
+    TableId table1 = TableId.of(tableString);
+    assertEquals(initialSize + 1, TableId.cache.asMap().entrySet().stream().count());
+    assertEquals(tableString, table1.canonical());
+
+    // ensure duplicates are not created
+    TableId builtInTableId = TableId.of("!0");
+    assertSame(MetadataTable.ID, builtInTableId);
+    builtInTableId = TableId.of("+r");
+    assertSame(RootTable.ID, builtInTableId);
+    builtInTableId = TableId.of("+rep");
+    assertSame(ReplicationTable.ID, builtInTableId);
+    table1 = TableId.of(tableString);
+    assertEquals(initialSize + 1, TableId.cache.asMap().entrySet().stream().count());
+    assertEquals(tableString, table1.canonical());
+    TableId table2 = TableId.of(tableString);
+    assertEquals(initialSize + 1, TableId.cache.asMap().entrySet().stream().count());
+    assertEquals(tableString, table2.canonical());
+    assertSame(table1, table2);
+  }
+
+  @Test(timeout = 60_000)
+  public void testCacheDecreasesAfterGC() {
+    Long initialSize = TableId.cache.asMap().entrySet().stream().count();
+    generateJunkCacheEntries();
+    Long postGCSize;
+    do {
+      System.gc();
+      try {
+        Thread.sleep(500);
+      } catch (InterruptedException e) {
+        fail("Thread interrupted while waiting for GC");
+      }
+      postGCSize = TableId.cache.asMap().entrySet().stream().count();
+    } while (postGCSize > initialSize);
+
+    assertTrue("Cache did not decrease with GC.",
+        TableId.cache.asMap().entrySet().stream().count() < initialSize);
+  }
+
+  private void generateJunkCacheEntries() {
+    for (int i = 0; i < 1000; i++)
+      TableId.of(new String("table" + i));
+  }
+
+}
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 407816b..497e5c2 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
@@ -28,13 +28,13 @@ import java.util.List;
 import java.util.Map;
 import java.util.TreeMap;
 
-import org.apache.accumulo.core.clientImpl.Table;
 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.ByteSequence;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Range;
+import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.dataImpl.KeyExtent;
 import org.apache.accumulo.core.dataImpl.thrift.IterInfo;
@@ -148,7 +148,7 @@ public class IteratorUtilTest {
     SortedMapIterator source = new SortedMapIterator(tm);
 
     SortedKeyValueIterator<Key,Value> iter = IteratorUtil.loadIterators(IteratorScope.minc, source,
-        new KeyExtent(Table.ID.of("tab"), null, null), conf, new DefaultIteratorEnvironment(conf));
+        new KeyExtent(TableId.of("tab"), null, null), conf, new DefaultIteratorEnvironment(conf));
     iter.seek(new Range(), EMPTY_COL_FAMS, false);
 
     assertTrue(iter.hasTop());
@@ -180,7 +180,7 @@ public class IteratorUtilTest {
     SortedMapIterator source = new SortedMapIterator(tm);
 
     SortedKeyValueIterator<Key,Value> iter = IteratorUtil.loadIterators(IteratorScope.majc, source,
-        new KeyExtent(Table.ID.of("tab"), null, null), conf, new DefaultIteratorEnvironment(conf));
+        new KeyExtent(TableId.of("tab"), null, null), conf, new DefaultIteratorEnvironment(conf));
     iter.seek(new Range(), EMPTY_COL_FAMS, false);
 
     assertTrue(iter.hasTop());
@@ -218,7 +218,7 @@ public class IteratorUtilTest {
         "1," + SquaringIter.class.getName());
 
     SortedKeyValueIterator<Key,Value> iter = IteratorUtil.loadIterators(IteratorScope.minc, source,
-        new KeyExtent(Table.ID.of("tab"), null, null), conf, new DefaultIteratorEnvironment(conf));
+        new KeyExtent(TableId.of("tab"), null, null), conf, new DefaultIteratorEnvironment(conf));
     iter.seek(new Range(), EMPTY_COL_FAMS, false);
 
     assertTrue(iter.hasTop());
@@ -257,7 +257,7 @@ public class IteratorUtilTest {
     SortedMapIterator source = new SortedMapIterator(tm);
 
     SortedKeyValueIterator<Key,Value> iter = IteratorUtil.loadIterators(IteratorScope.minc, source,
-        new KeyExtent(Table.ID.of("tab"), null, null), conf, new DefaultIteratorEnvironment(conf));
+        new KeyExtent(TableId.of("tab"), null, null), conf, new DefaultIteratorEnvironment(conf));
     iter.seek(new Range(), EMPTY_COL_FAMS, false);
 
     assertTrue(iter.hasTop());
@@ -295,7 +295,7 @@ public class IteratorUtilTest {
     SortedMapIterator source = new SortedMapIterator(tm);
 
     SortedKeyValueIterator<Key,Value> iter = IteratorUtil.loadIterators(IteratorScope.minc, source,
-        new KeyExtent(Table.ID.of("tab"), null, null), conf, new DefaultIteratorEnvironment(conf));
+        new KeyExtent(TableId.of("tab"), null, null), conf, new DefaultIteratorEnvironment(conf));
     iter.seek(new Range(), EMPTY_COL_FAMS, false);
 
     assertTrue(iter.hasTop());
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 ce17704..c8c01d4 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,10 +26,10 @@ import java.util.Collection;
 import java.util.List;
 import java.util.TreeMap;
 
-import org.apache.accumulo.core.clientImpl.Table;
 import org.apache.accumulo.core.data.ByteSequence;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Range;
+import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.dataImpl.KeyExtent;
 import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
@@ -361,7 +361,7 @@ public class MultiIteratorTest {
     List<SortedKeyValueIterator<Key,Value>> skvil = new ArrayList<>(1);
     skvil.add(new SortedMapIterator(tm1));
 
-    KeyExtent extent = new KeyExtent(Table.ID.of("tablename"), newRow(1), newRow(0));
+    KeyExtent extent = new KeyExtent(TableId.of("tablename"), newRow(1), newRow(0));
 
     MultiIterator mi = new MultiIterator(skvil, extent);
 
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 905ba49..d86618d 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
@@ -27,7 +27,7 @@ import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.admin.TableOperations;
 import org.apache.accumulo.core.clientImpl.ClientContext;
-import org.apache.accumulo.core.clientImpl.Table;
+import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.replication.ReplicationTable;
 import org.easymock.EasyMock;
 import org.junit.BeforeClass;
@@ -36,16 +36,16 @@ import org.junit.Test;
 public class MetadataServicerTest {
 
   private static final String userTableName = "tableName";
-  private static final Table.ID userTableId = Table.ID.of("tableId");
+  private static final TableId userTableId = TableId.of("tableId");
   private static ClientContext context;
 
   @BeforeClass
   public static void setupContext() {
     HashMap<String,String> tableNameToIdMap = new HashMap<>();
-    tableNameToIdMap.put(RootTable.NAME, RootTable.ID.canonicalID());
-    tableNameToIdMap.put(MetadataTable.NAME, MetadataTable.ID.canonicalID());
-    tableNameToIdMap.put(ReplicationTable.NAME, ReplicationTable.ID.canonicalID());
-    tableNameToIdMap.put(userTableName, userTableId.canonicalID());
+    tableNameToIdMap.put(RootTable.NAME, RootTable.ID.canonical());
+    tableNameToIdMap.put(MetadataTable.NAME, MetadataTable.ID.canonical());
+    tableNameToIdMap.put(ReplicationTable.NAME, ReplicationTable.ID.canonical());
+    tableNameToIdMap.put(userTableName, userTableId.canonical());
 
     context = EasyMock.createMock(ClientContext.class);
     TableOperations tableOps = EasyMock.createMock(TableOperations.class);
diff --git a/core/src/test/java/org/apache/accumulo/core/metadata/schema/TabletMetadataTest.java b/core/src/test/java/org/apache/accumulo/core/metadata/schema/TabletMetadataTest.java
index 1243d27..9232f0c 100644
--- a/core/src/test/java/org/apache/accumulo/core/metadata/schema/TabletMetadataTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/metadata/schema/TabletMetadataTest.java
@@ -30,9 +30,9 @@ import java.util.EnumSet;
 import java.util.SortedMap;
 import java.util.TreeMap;
 
-import org.apache.accumulo.core.clientImpl.Table.ID;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Mutation;
+import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.dataImpl.KeyExtent;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.BulkFileColumnFamily;
@@ -56,7 +56,7 @@ public class TabletMetadataTest {
 
   @Test
   public void testAllColumns() {
-    KeyExtent extent = new KeyExtent(ID.of("5"), new Text("df"), new Text("da"));
+    KeyExtent extent = new KeyExtent(TableId.of("5"), new Text("df"), new Text("da"));
 
     Mutation mutation = extent.getPrevRowUpdateMutation();
 
@@ -123,7 +123,7 @@ public class TabletMetadataTest {
 
   @Test
   public void testFuture() {
-    KeyExtent extent = new KeyExtent(ID.of("5"), new Text("df"), new Text("da"));
+    KeyExtent extent = new KeyExtent(TableId.of("5"), new Text("df"), new Text("da"));
 
     Mutation mutation = extent.getPrevRowUpdateMutation();
     mutation.at().family(FutureLocationColumnFamily.NAME).qualifier("s001").put("server1:8555");
@@ -142,7 +142,7 @@ public class TabletMetadataTest {
 
   @Test(expected = IllegalStateException.class)
   public void testFutureAndCurrent() {
-    KeyExtent extent = new KeyExtent(ID.of("5"), new Text("df"), new Text("da"));
+    KeyExtent extent = new KeyExtent(TableId.of("5"), new Text("df"), new Text("da"));
 
     Mutation mutation = extent.getPrevRowUpdateMutation();
     mutation.at().family(CurrentLocationColumnFamily.NAME).qualifier("s001").put("server1:8555");
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 c59d7e1..84daa2b 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,10 +22,10 @@ import static org.junit.Assert.assertTrue;
 import java.util.HashMap;
 import java.util.Map;
 
-import org.apache.accumulo.core.clientImpl.Table;
 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.TableId;
 import org.apache.accumulo.core.dataImpl.KeyExtent;
 import org.apache.accumulo.core.metadata.MetadataTable;
 import org.apache.accumulo.core.metadata.RootTable;
@@ -74,13 +74,13 @@ public class ReplicationConfigurationUtilTest {
 
   @Test
   public void regularTable() {
-    KeyExtent extent = new KeyExtent(Table.ID.of("1"), new Text("b"), new Text("a"));
+    KeyExtent extent = new KeyExtent(TableId.of("1"), new Text("b"), new Text("a"));
     assertTrue("Table should be replicated", ReplicationConfigurationUtil.isEnabled(extent, conf));
   }
 
   @Test
   public void regularNonEnabledTable() {
-    KeyExtent extent = new KeyExtent(Table.ID.of("1"), new Text("b"), new Text("a"));
+    KeyExtent extent = new KeyExtent(TableId.of("1"), new Text("b"), new Text("a"));
     assertFalse("Table should not be replicated",
         ReplicationConfigurationUtil.isEnabled(extent, new ConfigurationCopy(new HashMap<>())));
   }
diff --git a/core/src/test/java/org/apache/accumulo/core/replication/ReplicationSchemaTest.java b/core/src/test/java/org/apache/accumulo/core/replication/ReplicationSchemaTest.java
index 0a2ef86..87fdc06 100644
--- a/core/src/test/java/org/apache/accumulo/core/replication/ReplicationSchemaTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/replication/ReplicationSchemaTest.java
@@ -18,9 +18,9 @@ package org.apache.accumulo.core.replication;
 
 import static org.junit.Assert.assertEquals;
 
-import org.apache.accumulo.core.clientImpl.Table;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Mutation;
+import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.replication.ReplicationSchema.OrderSection;
 import org.apache.accumulo.core.replication.ReplicationSchema.StatusSection;
 import org.apache.accumulo.core.replication.ReplicationSchema.WorkSection;
@@ -63,15 +63,15 @@ public class ReplicationSchemaTest {
 
   @Test
   public void extractTableId() {
-    Table.ID tableId = Table.ID.of("1");
-    Key k = new Key(new Text("foo"), StatusSection.NAME, new Text(tableId.getUtf8()));
+    TableId tableId = TableId.of("1");
+    Key k = new Key(new Text("foo"), StatusSection.NAME, new Text(tableId.canonical()));
     assertEquals(tableId, StatusSection.getTableId(k));
   }
 
   @Test
   public void extractTableIdUsingText() {
-    Table.ID tableId = Table.ID.of("1");
-    Key k = new Key(new Text("foo"), StatusSection.NAME, new Text(tableId.getUtf8()));
+    TableId tableId = TableId.of("1");
+    Key k = new Key(new Text("foo"), StatusSection.NAME, new Text(tableId.canonical()));
     assertEquals(tableId, StatusSection.getTableId(k));
   }
 
diff --git a/core/src/test/java/org/apache/accumulo/core/replication/ReplicationTargetTest.java b/core/src/test/java/org/apache/accumulo/core/replication/ReplicationTargetTest.java
index 091e7ab..20dbf19 100644
--- a/core/src/test/java/org/apache/accumulo/core/replication/ReplicationTargetTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/replication/ReplicationTargetTest.java
@@ -19,7 +19,7 @@ package org.apache.accumulo.core.replication;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotEquals;
 
-import org.apache.accumulo.core.clientImpl.Table;
+import org.apache.accumulo.core.data.TableId;
 import org.apache.hadoop.io.DataInputBuffer;
 import org.apache.hadoop.io.DataOutputBuffer;
 import org.apache.hadoop.io.Text;
@@ -29,18 +29,18 @@ public class ReplicationTargetTest {
 
   @Test
   public void properEquality() {
-    ReplicationTarget expected1 = new ReplicationTarget("foo", "bar", Table.ID.of("1"));
+    ReplicationTarget expected1 = new ReplicationTarget("foo", "bar", TableId.of("1"));
 
-    assertEquals(expected1, new ReplicationTarget("foo", "bar", Table.ID.of("1")));
-    assertNotEquals(expected1, new ReplicationTarget("foo", "foo", Table.ID.of("1")));
-    assertNotEquals(expected1, new ReplicationTarget("bar", "bar", Table.ID.of("1")));
-    assertNotEquals(expected1, new ReplicationTarget(null, "bar", Table.ID.of("1")));
-    assertNotEquals(expected1, new ReplicationTarget("foo", null, Table.ID.of("1")));
+    assertEquals(expected1, new ReplicationTarget("foo", "bar", TableId.of("1")));
+    assertNotEquals(expected1, new ReplicationTarget("foo", "foo", TableId.of("1")));
+    assertNotEquals(expected1, new ReplicationTarget("bar", "bar", TableId.of("1")));
+    assertNotEquals(expected1, new ReplicationTarget(null, "bar", TableId.of("1")));
+    assertNotEquals(expected1, new ReplicationTarget("foo", null, TableId.of("1")));
   }
 
   @Test
   public void writableOut() throws Exception {
-    ReplicationTarget expected = new ReplicationTarget("foo", "bar", Table.ID.of("1"));
+    ReplicationTarget expected = new ReplicationTarget("foo", "bar", TableId.of("1"));
     DataOutputBuffer buffer = new DataOutputBuffer();
     expected.write(buffer);
 
@@ -64,7 +64,7 @@ public class ReplicationTargetTest {
 
   @Test
   public void staticFromTextHelper() throws Exception {
-    ReplicationTarget expected = new ReplicationTarget("foo", "bar", Table.ID.of("1"));
+    ReplicationTarget expected = new ReplicationTarget("foo", "bar", TableId.of("1"));
     DataOutputBuffer buffer = new DataOutputBuffer();
     expected.write(buffer);
     Text t = new Text();
@@ -75,7 +75,7 @@ public class ReplicationTargetTest {
 
   @Test
   public void staticToTextHelper() throws Exception {
-    ReplicationTarget expected = new ReplicationTarget("foo", "bar", Table.ID.of("1"));
+    ReplicationTarget expected = new ReplicationTarget("foo", "bar", TableId.of("1"));
     DataOutputBuffer buffer = new DataOutputBuffer();
     expected.write(buffer);
     Text t = new Text();
@@ -86,7 +86,7 @@ public class ReplicationTargetTest {
 
   @Test
   public void staticFromStringHelper() throws Exception {
-    ReplicationTarget expected = new ReplicationTarget("foo", "bar", Table.ID.of("1"));
+    ReplicationTarget expected = new ReplicationTarget("foo", "bar", TableId.of("1"));
     DataOutputBuffer buffer = new DataOutputBuffer();
     expected.write(buffer);
     Text t = new Text();
diff --git a/core/src/test/java/org/apache/accumulo/core/util/MergeTest.java b/core/src/test/java/org/apache/accumulo/core/util/MergeTest.java
index eb0dfd7..9218714 100644
--- a/core/src/test/java/org/apache/accumulo/core/util/MergeTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/util/MergeTest.java
@@ -24,7 +24,7 @@ import java.util.Iterator;
 import java.util.List;
 
 import org.apache.accumulo.core.client.AccumuloClient;
-import org.apache.accumulo.core.clientImpl.Table;
+import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.dataImpl.KeyExtent;
 import org.apache.accumulo.core.util.Merge.Size;
 import org.apache.hadoop.io.Text;
@@ -44,7 +44,7 @@ public class MergeTest {
           end = null;
         else
           end = new Text(String.format("%05d", tablets.size()));
-        KeyExtent extent = new KeyExtent(Table.ID.of("table"), end, start);
+        KeyExtent extent = new KeyExtent(TableId.of("table"), end, start);
         start = end;
         tablets.add(new Size(extent, size));
       }
diff --git a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapred/AccumuloRecordReader.java b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapred/AccumuloRecordReader.java
index 279e53c..00076bd 100644
--- a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapred/AccumuloRecordReader.java
+++ b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapred/AccumuloRecordReader.java
@@ -46,11 +46,11 @@ import org.apache.accumulo.core.client.sample.SamplerConfiguration;
 import org.apache.accumulo.core.clientImpl.ClientContext;
 import org.apache.accumulo.core.clientImpl.OfflineScanner;
 import org.apache.accumulo.core.clientImpl.ScannerImpl;
-import org.apache.accumulo.core.clientImpl.Table;
 import org.apache.accumulo.core.clientImpl.Tables;
 import org.apache.accumulo.core.clientImpl.TabletLocator;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Range;
+import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.dataImpl.KeyExtent;
 import org.apache.accumulo.core.master.state.tables.TableState;
@@ -185,10 +185,9 @@ public abstract class AccumuloRecordReader<K,V> implements RecordReader<K,V> {
 
       try {
         if (isOffline) {
-          scanner = new OfflineScanner(context, Table.ID.of(baseSplit.getTableId()),
-              authorizations);
+          scanner = new OfflineScanner(context, TableId.of(baseSplit.getTableId()), authorizations);
         } else {
-          scanner = new ScannerImpl(context, Table.ID.of(baseSplit.getTableId()), authorizations);
+          scanner = new ScannerImpl(context, TableId.of(baseSplit.getTableId()), authorizations);
         }
         if (isIsolated) {
           log.info("Creating isolated scanner");
@@ -272,7 +271,7 @@ public abstract class AccumuloRecordReader<K,V> implements RecordReader<K,V> {
   protected Key currentKey = null;
 
   private static Map<String,Map<KeyExtent,List<Range>>> binOfflineTable(JobConf job,
-      Table.ID tableId, List<Range> ranges, Class<?> callingClass)
+      TableId tableId, List<Range> ranges, Class<?> callingClass)
       throws TableNotFoundException, AccumuloException {
     try (AccumuloClient client = createClient(job, callingClass)) {
       return InputConfigurator.binOffline(tableId, ranges, (ClientContext) client);
@@ -296,7 +295,7 @@ public abstract class AccumuloRecordReader<K,V> implements RecordReader<K,V> {
         InputTableConfig tableConfig = tableConfigEntry.getValue();
 
         ClientContext context = (ClientContext) client;
-        Table.ID tableId;
+        TableId tableId;
         // resolve table name to id once, and use id from this point forward
         try {
           tableId = Tables.getTableId(context, tableName);
@@ -343,7 +342,7 @@ public abstract class AccumuloRecordReader<K,V> implements RecordReader<K,V> {
             tl.invalidateCache();
 
             while (!tl.binRanges(context, ranges, binnedRanges).isEmpty()) {
-              String tableIdStr = tableId.canonicalID();
+              String tableIdStr = tableId.canonical();
               if (!Tables.exists(context, tableId))
                 throw new TableDeletedException(tableIdStr);
               if (Tables.getTableState(context, tableId) == TableState.OFFLINE)
@@ -391,7 +390,7 @@ public abstract class AccumuloRecordReader<K,V> implements RecordReader<K,V> {
               for (Range r : extentRanges.getValue()) {
                 if (autoAdjust) {
                   // divide ranges into smaller ranges, based on the tablets
-                  RangeInputSplit split = new RangeInputSplit(tableName, tableId.canonicalID(),
+                  RangeInputSplit split = new RangeInputSplit(tableName, tableId.canonical(),
                       ke.clip(r), new String[] {location});
                   SplitUtils.updateSplit(split, tableConfig);
                   split.setOffline(tableConfig.isOfflineScan());
@@ -414,7 +413,7 @@ public abstract class AccumuloRecordReader<K,V> implements RecordReader<K,V> {
 
         if (!autoAdjust)
           for (Map.Entry<Range,ArrayList<String>> entry : splitsToAdd.entrySet()) {
-            RangeInputSplit split = new RangeInputSplit(tableName, tableId.canonicalID(),
+            RangeInputSplit split = new RangeInputSplit(tableName, tableId.canonical(),
                 entry.getKey(), entry.getValue().toArray(new String[0]));
             SplitUtils.updateSplit(split, tableConfig);
             split.setOffline(tableConfig.isOfflineScan());
diff --git a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapred/BatchInputSplit.java b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapred/BatchInputSplit.java
index d0a48d3..dfc8f64 100644
--- a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapred/BatchInputSplit.java
+++ b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapred/BatchInputSplit.java
@@ -19,8 +19,8 @@ package org.apache.accumulo.hadoopImpl.mapred;
 import java.io.IOException;
 import java.util.Collection;
 
-import org.apache.accumulo.core.clientImpl.Table;
 import org.apache.accumulo.core.data.Range;
+import org.apache.accumulo.core.data.TableId;
 import org.apache.hadoop.mapred.InputSplit;
 
 import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
@@ -42,7 +42,7 @@ public class BatchInputSplit extends org.apache.accumulo.hadoopImpl.mapreduce.Ba
     super(split);
   }
 
-  public BatchInputSplit(String table, Table.ID tableId, Collection<Range> ranges,
+  public BatchInputSplit(String table, TableId tableId, Collection<Range> ranges,
       String[] location) {
     super(table, tableId, ranges, location);
   }
diff --git a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/AccumuloRecordReader.java b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/AccumuloRecordReader.java
index 8e7d068..182d66b 100644
--- a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/AccumuloRecordReader.java
+++ b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/AccumuloRecordReader.java
@@ -46,11 +46,11 @@ import org.apache.accumulo.core.client.sample.SamplerConfiguration;
 import org.apache.accumulo.core.clientImpl.ClientContext;
 import org.apache.accumulo.core.clientImpl.OfflineScanner;
 import org.apache.accumulo.core.clientImpl.ScannerImpl;
-import org.apache.accumulo.core.clientImpl.Table;
 import org.apache.accumulo.core.clientImpl.Tables;
 import org.apache.accumulo.core.clientImpl.TabletLocator;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Range;
+import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.dataImpl.KeyExtent;
 import org.apache.accumulo.core.master.state.tables.TableState;
@@ -203,11 +203,11 @@ public abstract class AccumuloRecordReader<K,V> extends RecordReader<K,V> {
 
       try {
         if (isOffline) {
-          scanner = new OfflineScanner(context, Table.ID.of(split.getTableId()), authorizations);
+          scanner = new OfflineScanner(context, TableId.of(split.getTableId()), authorizations);
         } else {
           // Not using public API to create scanner so that we can use table ID
           // Table ID is used in case of renames during M/R job
-          scanner = new ScannerImpl(context, Table.ID.of(split.getTableId()), authorizations);
+          scanner = new ScannerImpl(context, TableId.of(split.getTableId()), authorizations);
         }
         if (isIsolated) {
           log.info("Creating isolated scanner");
@@ -306,7 +306,7 @@ public abstract class AccumuloRecordReader<K,V> extends RecordReader<K,V> {
   }
 
   private static Map<String,Map<KeyExtent,List<Range>>> binOfflineTable(JobContext context,
-      Table.ID tableId, List<Range> ranges, Class<?> callingClass)
+      TableId tableId, List<Range> ranges, Class<?> callingClass)
       throws TableNotFoundException, AccumuloException {
     try (AccumuloClient client = createClient(context, callingClass)) {
       return InputConfigurator.binOffline(tableId, ranges, (ClientContext) client);
@@ -327,7 +327,7 @@ public abstract class AccumuloRecordReader<K,V> extends RecordReader<K,V> {
         InputTableConfig tableConfig = tableConfigEntry.getValue();
 
         ClientContext clientContext = (ClientContext) client;
-        Table.ID tableId;
+        TableId tableId;
         // resolve table name to id once, and use id from this point forward
         try {
           tableId = Tables.getTableId(clientContext, tableName);
@@ -375,7 +375,7 @@ public abstract class AccumuloRecordReader<K,V> extends RecordReader<K,V> {
             tl.invalidateCache();
 
             while (!tl.binRanges(clientContext, ranges, binnedRanges).isEmpty()) {
-              String tableIdStr = tableId.canonicalID();
+              String tableIdStr = tableId.canonical();
               if (!Tables.exists(clientContext, tableId))
                 throw new TableDeletedException(tableIdStr);
               if (Tables.getTableState(clientContext, tableId) == TableState.OFFLINE)
@@ -425,7 +425,7 @@ public abstract class AccumuloRecordReader<K,V> extends RecordReader<K,V> {
               for (Range r : extentRanges.getValue()) {
                 if (autoAdjust) {
                   // divide ranges into smaller ranges, based on the tablets
-                  RangeInputSplit split = new RangeInputSplit(tableName, tableId.canonicalID(),
+                  RangeInputSplit split = new RangeInputSplit(tableName, tableId.canonical(),
                       ke.clip(r), new String[] {location});
                   SplitUtils.updateSplit(split, tableConfig);
                   split.setOffline(tableConfig.isOfflineScan());
@@ -447,7 +447,7 @@ public abstract class AccumuloRecordReader<K,V> extends RecordReader<K,V> {
 
         if (!autoAdjust)
           for (Map.Entry<Range,ArrayList<String>> entry : splitsToAdd.entrySet()) {
-            RangeInputSplit split = new RangeInputSplit(tableName, tableId.canonicalID(),
+            RangeInputSplit split = new RangeInputSplit(tableName, tableId.canonical(),
                 entry.getKey(), entry.getValue().toArray(new String[0]));
             SplitUtils.updateSplit(split, tableConfig);
             split.setOffline(tableConfig.isOfflineScan());
diff --git a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/BatchInputSplit.java b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/BatchInputSplit.java
index 89ec405..8cf12f3 100644
--- a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/BatchInputSplit.java
+++ b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/BatchInputSplit.java
@@ -24,10 +24,10 @@ import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
 
-import org.apache.accumulo.core.clientImpl.Table;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.PartialKey;
 import org.apache.accumulo.core.data.Range;
+import org.apache.accumulo.core.data.TableId;
 
 /**
  * The Class BatchInputSplit. Encapsulates a set of Accumulo ranges on a single tablet for use in
@@ -46,9 +46,9 @@ public class BatchInputSplit extends RangeInputSplit {
     this.setRanges(split.getRanges());
   }
 
-  public BatchInputSplit(String table, Table.ID tableId, Collection<Range> ranges,
+  public BatchInputSplit(String table, TableId tableId, Collection<Range> ranges,
       String[] locations) {
-    super(table, tableId.canonicalID(), new Range(), locations);
+    super(table, tableId.canonical(), new Range(), locations);
     this.ranges = ranges;
   }
 
diff --git a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/lib/InputConfigurator.java b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/lib/InputConfigurator.java
index 19e7f0c..938b769 100644
--- a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/lib/InputConfigurator.java
+++ b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/lib/InputConfigurator.java
@@ -50,13 +50,13 @@ import org.apache.accumulo.core.client.Scanner;
 import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.client.sample.SamplerConfiguration;
 import org.apache.accumulo.core.clientImpl.ClientContext;
-import org.apache.accumulo.core.clientImpl.Table;
 import org.apache.accumulo.core.clientImpl.Tables;
 import org.apache.accumulo.core.clientImpl.TabletLocator;
 import org.apache.accumulo.core.conf.ClientProperty;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.PartialKey;
 import org.apache.accumulo.core.data.Range;
+import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.dataImpl.KeyExtent;
 import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
@@ -711,7 +711,7 @@ public class InputConfigurator extends ConfiguratorBase {
    * @since 1.6.0
    */
   public static TabletLocator getTabletLocator(Class<?> implementingClass, Configuration conf,
-      Table.ID tableId) {
+      TableId tableId) {
     try (AccumuloClient client = createClient(implementingClass, conf)) {
       return TabletLocator.getLocator((ClientContext) client, tableId);
     }
@@ -808,7 +808,7 @@ public class InputConfigurator extends ConfiguratorBase {
     return null;
   }
 
-  public static Map<String,Map<KeyExtent,List<Range>>> binOffline(Table.ID tableId,
+  public static Map<String,Map<KeyExtent,List<Range>>> binOffline(TableId tableId,
       List<Range> ranges, ClientContext context) throws AccumuloException, TableNotFoundException {
     Map<String,Map<KeyExtent,List<Range>>> binnedRanges = new HashMap<>();
 
diff --git a/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoopImpl/mapreduce/BatchInputSplitTest.java b/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoopImpl/mapreduce/BatchInputSplitTest.java
index bcb62c9..58a0ad5 100644
--- a/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoopImpl/mapreduce/BatchInputSplitTest.java
+++ b/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoopImpl/mapreduce/BatchInputSplitTest.java
@@ -33,9 +33,9 @@ import java.util.List;
 import java.util.Set;
 
 import org.apache.accumulo.core.client.IteratorSetting;
-import org.apache.accumulo.core.clientImpl.Table;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Range;
+import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.iterators.user.SummingCombiner;
 import org.apache.accumulo.core.iterators.user.WholeRowIterator;
 import org.apache.hadoop.io.Text;
@@ -46,7 +46,7 @@ public class BatchInputSplitTest {
   @Test
   public void testSimpleWritable() throws IOException {
     List<Range> ranges = Collections.singletonList(new Range(new Key("a"), new Key("b")));
-    BatchInputSplit split = new BatchInputSplit("table", Table.ID.of("1"), ranges,
+    BatchInputSplit split = new BatchInputSplit("table", TableId.of("1"), ranges,
         new String[] {"localhost"});
 
     ByteArrayOutputStream baos = new ByteArrayOutputStream();
@@ -68,7 +68,7 @@ public class BatchInputSplitTest {
   @Test
   public void testAllFieldsWritable() throws IOException {
     List<Range> ranges = Collections.singletonList(new Range(new Key("a"), new Key("b")));
-    BatchInputSplit split = new BatchInputSplit("table", Table.ID.of("1"), ranges,
+    BatchInputSplit split = new BatchInputSplit("table", TableId.of("1"), ranges,
         new String[] {"localhost"});
 
     Set<IteratorSetting.Column> fetchedColumns = new HashSet<>();
diff --git a/minicluster/src/test/java/org/apache/accumulo/miniclusterImpl/MiniAccumuloClusterImplTest.java b/minicluster/src/test/java/org/apache/accumulo/miniclusterImpl/MiniAccumuloClusterImplTest.java
index 3c5f443..845102d 100644
--- a/minicluster/src/test/java/org/apache/accumulo/miniclusterImpl/MiniAccumuloClusterImplTest.java
+++ b/minicluster/src/test/java/org/apache/accumulo/miniclusterImpl/MiniAccumuloClusterImplTest.java
@@ -120,9 +120,9 @@ public class MiniAccumuloClusterImplTest {
         validGoals.contains(stats.goalState));
     assertNotNull("should have a table map.", stats.tableMap);
     assertTrue("root table should exist in " + stats.tableMap.keySet(),
-        stats.tableMap.keySet().contains(RootTable.ID.canonicalID()));
+        stats.tableMap.keySet().contains(RootTable.ID.canonical()));
     assertTrue("meta table should exist in " + stats.tableMap.keySet(),
-        stats.tableMap.keySet().contains(MetadataTable.ID.canonicalID()));
+        stats.tableMap.keySet().contains(MetadataTable.ID.canonical()));
     assertTrue("our test table should exist in " + stats.tableMap.keySet(),
         stats.tableMap.keySet().contains(testTableID));
     assertNotNull("there should be tservers.", stats.tServerInfo);
diff --git a/proxy/src/main/java/org/apache/accumulo/proxy/ProxyServer.java b/proxy/src/main/java/org/apache/accumulo/proxy/ProxyServer.java
index 6516623..bde9d26 100644
--- a/proxy/src/main/java/org/apache/accumulo/proxy/ProxyServer.java
+++ b/proxy/src/main/java/org/apache/accumulo/proxy/ProxyServer.java
@@ -1789,12 +1789,12 @@ public class ProxyServer implements AccumuloProxy.Iface {
 
   @Override
   public String systemNamespace() throws TException {
-    return Namespace.ACCUMULO;
+    return Namespace.ACCUMULO.name();
   }
 
   @Override
   public String defaultNamespace() throws TException {
-    return Namespace.DEFAULT;
+    return Namespace.DEFAULT.name();
   }
 
   @Override
diff --git a/server/base/src/main/java/org/apache/accumulo/server/client/BulkImporter.java b/server/base/src/main/java/org/apache/accumulo/server/client/BulkImporter.java
index 344dfd8..360fcbd 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/client/BulkImporter.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/client/BulkImporter.java
@@ -38,7 +38,6 @@ import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.clientImpl.ClientContext;
 import org.apache.accumulo.core.clientImpl.ServerClient;
-import org.apache.accumulo.core.clientImpl.Table;
 import org.apache.accumulo.core.clientImpl.TabletLocator;
 import org.apache.accumulo.core.clientImpl.TabletLocator.TabletLocation;
 import org.apache.accumulo.core.clientImpl.Translator;
@@ -48,6 +47,7 @@ import org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.data.ByteSequence;
 import org.apache.accumulo.core.data.Range;
+import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.dataImpl.KeyExtent;
 import org.apache.accumulo.core.dataImpl.thrift.TKeyExtent;
 import org.apache.accumulo.core.file.FileOperations;
@@ -124,7 +124,7 @@ public class BulkImporter {
         .synchronizedSortedMap(new TreeMap<>());
 
     ClientService.Client client = null;
-    final TabletLocator locator = TabletLocator.getLocator(context, Table.ID.of(tableId));
+    final TabletLocator locator = TabletLocator.getLocator(context, TableId.of(tableId));
 
     try {
       final Map<Path,List<TabletLocation>> assignments = Collections
diff --git a/server/base/src/main/java/org/apache/accumulo/server/client/ClientServiceHandler.java b/server/base/src/main/java/org/apache/accumulo/server/client/ClientServiceHandler.java
index e3b0498..9b8b5dd 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/client/ClientServiceHandler.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/client/ClientServiceHandler.java
@@ -35,9 +35,7 @@ import org.apache.accumulo.core.client.security.tokens.KerberosToken;
 import org.apache.accumulo.core.client.security.tokens.PasswordToken;
 import org.apache.accumulo.core.clientImpl.ClientContext;
 import org.apache.accumulo.core.clientImpl.Credentials;
-import org.apache.accumulo.core.clientImpl.Namespace;
 import org.apache.accumulo.core.clientImpl.Namespaces;
-import org.apache.accumulo.core.clientImpl.Table;
 import org.apache.accumulo.core.clientImpl.Tables;
 import org.apache.accumulo.core.clientImpl.thrift.ClientService;
 import org.apache.accumulo.core.clientImpl.thrift.ConfigurationType;
@@ -49,6 +47,8 @@ import org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException;
 import org.apache.accumulo.core.clientImpl.thrift.ThriftTableOperationException;
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
 import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.core.data.NamespaceId;
+import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.master.thrift.BulkImportState;
 import org.apache.accumulo.core.master.thrift.BulkImportStatus;
 import org.apache.accumulo.core.security.Authorizations;
@@ -86,7 +86,7 @@ public class ClientServiceHandler implements ClientService.Iface {
     this.security = AuditedSecurityOperation.getInstance(context);
   }
 
-  public static Table.ID checkTableId(ClientContext context, String tableName,
+  public static TableId checkTableId(ClientContext context, String tableName,
       TableOperation operation) throws ThriftTableOperationException {
     TableOperationExceptionType reason = null;
     try {
@@ -99,9 +99,9 @@ public class ClientServiceHandler implements ClientService.Iface {
     throw new ThriftTableOperationException(null, tableName, operation, reason, null);
   }
 
-  public static Namespace.ID checkNamespaceId(ClientContext context, String namespaceName,
+  public static NamespaceId checkNamespaceId(ClientContext context, String namespaceName,
       TableOperation operation) throws ThriftTableOperationException {
-    Namespace.ID namespaceId = Namespaces.lookupNamespaceId(context, namespaceName);
+    NamespaceId namespaceId = Namespaces.lookupNamespaceId(context, namespaceName);
     if (namespaceId == null) {
       // maybe the namespace exists, but the cache was not updated yet... so try to clear the cache
       // and check again
@@ -211,8 +211,8 @@ public class ClientServiceHandler implements ClientService.Iface {
   @Override
   public void grantTablePermission(TInfo tinfo, TCredentials credentials, String user,
       String tableName, byte permission) throws TException {
-    Table.ID tableId = checkTableId(context, tableName, TableOperation.PERMISSION);
-    Namespace.ID namespaceId;
+    TableId tableId = checkTableId(context, tableName, TableOperation.PERMISSION);
+    NamespaceId namespaceId;
     try {
       namespaceId = Tables.getNamespaceId(context, tableId);
     } catch (TableNotFoundException e) {
@@ -226,7 +226,7 @@ public class ClientServiceHandler implements ClientService.Iface {
   @Override
   public void grantNamespacePermission(TInfo tinfo, TCredentials credentials, String user,
       String ns, byte permission) throws ThriftSecurityException, ThriftTableOperationException {
-    Namespace.ID namespaceId = checkNamespaceId(context, ns, TableOperation.PERMISSION);
+    NamespaceId namespaceId = checkNamespaceId(context, ns, TableOperation.PERMISSION);
     security.grantNamespacePermission(credentials, user, namespaceId,
         NamespacePermission.getPermissionById(permission));
   }
@@ -241,8 +241,8 @@ public class ClientServiceHandler implements ClientService.Iface {
   @Override
   public void revokeTablePermission(TInfo tinfo, TCredentials credentials, String user,
       String tableName, byte permission) throws TException {
-    Table.ID tableId = checkTableId(context, tableName, TableOperation.PERMISSION);
-    Namespace.ID namespaceId;
+    TableId tableId = checkTableId(context, tableName, TableOperation.PERMISSION);
+    NamespaceId namespaceId;
     try {
       namespaceId = Tables.getNamespaceId(context, tableId);
     } catch (TableNotFoundException e) {
@@ -264,7 +264,7 @@ public class ClientServiceHandler implements ClientService.Iface {
   public boolean hasTablePermission(TInfo tinfo, TCredentials credentials, String user,
       String tableName, byte tblPerm)
       throws ThriftSecurityException, ThriftTableOperationException {
-    Table.ID tableId = checkTableId(context, tableName, TableOperation.PERMISSION);
+    TableId tableId = checkTableId(context, tableName, TableOperation.PERMISSION);
     return security.hasTablePermission(credentials, user, tableId,
         TablePermission.getPermissionById(tblPerm));
   }
@@ -272,7 +272,7 @@ public class ClientServiceHandler implements ClientService.Iface {
   @Override
   public boolean hasNamespacePermission(TInfo tinfo, TCredentials credentials, String user,
       String ns, byte perm) throws ThriftSecurityException, ThriftTableOperationException {
-    Namespace.ID namespaceId = checkNamespaceId(context, ns, TableOperation.PERMISSION);
+    NamespaceId namespaceId = checkNamespaceId(context, ns, TableOperation.PERMISSION);
     return security.hasNamespacePermission(credentials, user, namespaceId,
         NamespacePermission.getPermissionById(perm));
   }
@@ -280,7 +280,7 @@ public class ClientServiceHandler implements ClientService.Iface {
   @Override
   public void revokeNamespacePermission(TInfo tinfo, TCredentials credentials, String user,
       String ns, byte permission) throws ThriftSecurityException, ThriftTableOperationException {
-    Namespace.ID namespaceId = checkNamespaceId(context, ns, TableOperation.PERMISSION);
+    NamespaceId namespaceId = checkNamespaceId(context, ns, TableOperation.PERMISSION);
     security.revokeNamespacePermission(credentials, user, namespaceId,
         NamespacePermission.getPermissionById(permission));
   }
@@ -323,7 +323,7 @@ public class ClientServiceHandler implements ClientService.Iface {
   @Override
   public Map<String,String> getTableConfiguration(TInfo tinfo, TCredentials credentials,
       String tableName) throws TException, ThriftTableOperationException {
-    Table.ID tableId = checkTableId(context, tableName, null);
+    TableId tableId = checkTableId(context, tableName, null);
     AccumuloConfiguration config = context.getServerConfFactory().getTableConfiguration(tableId);
     return conf(credentials, config);
   }
@@ -384,7 +384,7 @@ public class ClientServiceHandler implements ClientService.Iface {
 
     security.authenticateUser(credentials, credentials);
 
-    Table.ID tableId = checkTableId(context, tableName, null);
+    TableId tableId = checkTableId(context, tableName, null);
 
     ClassLoader loader = getClass().getClassLoader();
     Class<?> shouldMatch;
@@ -419,7 +419,7 @@ public class ClientServiceHandler implements ClientService.Iface {
 
     security.authenticateUser(credentials, credentials);
 
-    Namespace.ID namespaceId = checkNamespaceId(context, ns, null);
+    NamespaceId namespaceId = checkNamespaceId(context, ns, null);
 
     ClassLoader loader = getClass().getClassLoader();
     Class<?> shouldMatch;
@@ -452,13 +452,13 @@ public class ClientServiceHandler implements ClientService.Iface {
   public List<TDiskUsage> getDiskUsage(Set<String> tables, TCredentials credentials)
       throws ThriftTableOperationException, ThriftSecurityException, TException {
     try {
-      HashSet<Table.ID> tableIds = new HashSet<>();
+      HashSet<TableId> tableIds = new HashSet<>();
 
       for (String table : tables) {
         // ensure that table table exists
-        Table.ID tableId = checkTableId(context, table, null);
+        TableId tableId = checkTableId(context, table, null);
         tableIds.add(tableId);
-        Namespace.ID namespaceId = Tables.getNamespaceId(context, tableId);
+        NamespaceId namespaceId = Tables.getNamespaceId(context, tableId);
         if (!security.canScan(credentials, tableId, namespaceId))
           throw new ThriftSecurityException(credentials.getPrincipal(),
               SecurityErrorCode.PERMISSION_DENIED);
@@ -480,7 +480,7 @@ public class ClientServiceHandler implements ClientService.Iface {
   @Override
   public Map<String,String> getNamespaceConfiguration(TInfo tinfo, TCredentials credentials,
       String ns) throws ThriftTableOperationException, TException {
-    Namespace.ID namespaceId;
+    NamespaceId namespaceId;
     try {
       namespaceId = Namespaces.getNamespaceId(context, ns);
     } catch (NamespaceNotFoundException e) {
diff --git a/server/base/src/main/java/org/apache/accumulo/server/conf/NamespaceConfWatcher.java b/server/base/src/main/java/org/apache/accumulo/server/conf/NamespaceConfWatcher.java
index 12d2055..cc0aefa 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/conf/NamespaceConfWatcher.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/conf/NamespaceConfWatcher.java
@@ -17,7 +17,7 @@
 package org.apache.accumulo.server.conf;
 
 import org.apache.accumulo.core.Constants;
-import org.apache.accumulo.core.clientImpl.Namespace;
+import org.apache.accumulo.core.data.NamespaceId;
 import org.apache.accumulo.server.ServerContext;
 import org.apache.log4j.Level;
 import org.apache.log4j.Logger;
@@ -72,7 +72,7 @@ class NamespaceConfWatcher implements Watcher {
         return;
       }
     }
-    Namespace.ID namespaceId = Namespace.ID.of(namespaceIdStr);
+    NamespaceId namespaceId = NamespaceId.of(namespaceIdStr);
 
     switch (event.getType()) {
       case NodeDataChanged:
diff --git a/server/base/src/main/java/org/apache/accumulo/server/conf/NamespaceConfiguration.java b/server/base/src/main/java/org/apache/accumulo/server/conf/NamespaceConfiguration.java
index f9aaad5..ff4b713 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/conf/NamespaceConfiguration.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/conf/NamespaceConfiguration.java
@@ -25,6 +25,7 @@ import org.apache.accumulo.core.conf.AccumuloConfiguration;
 import org.apache.accumulo.core.conf.ConfigurationObserver;
 import org.apache.accumulo.core.conf.ObservableConfiguration;
 import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.core.data.NamespaceId;
 import org.apache.accumulo.fate.zookeeper.ZooCache;
 import org.apache.accumulo.fate.zookeeper.ZooCacheFactory;
 import org.apache.accumulo.server.ServerContext;
@@ -39,12 +40,12 @@ public class NamespaceConfiguration extends ObservableConfiguration {
 
   private final AccumuloConfiguration parent;
   private ZooCachePropertyAccessor propCacheAccessor = null;
-  protected Namespace.ID namespaceId = null;
+  protected NamespaceId namespaceId = null;
   protected ServerContext context;
   private ZooCacheFactory zcf = new ZooCacheFactory();
   private final String path;
 
-  public NamespaceConfiguration(Namespace.ID namespaceId, ServerContext context,
+  public NamespaceConfiguration(NamespaceId namespaceId, ServerContext context,
       AccumuloConfiguration parent) {
     this.context = context;
     this.parent = parent;
@@ -69,7 +70,7 @@ public class NamespaceConfiguration extends ObservableConfiguration {
   private synchronized ZooCachePropertyAccessor getPropCacheAccessor() {
     if (propCacheAccessor == null) {
       synchronized (propCaches) {
-        PropCacheKey key = new PropCacheKey(context.getInstanceID(), namespaceId.canonicalID());
+        PropCacheKey key = new PropCacheKey(context.getInstanceID(), namespaceId.canonical());
         ZooCache propCache = propCaches.get(key);
         if (propCache == null) {
           propCache = zcf.getZooCache(context.getZooKeepers(),
@@ -90,7 +91,7 @@ public class NamespaceConfiguration extends ObservableConfiguration {
   public String get(Property property) {
     String key = property.getKey();
     AccumuloConfiguration getParent;
-    if (!(namespaceId.equals(Namespace.ID.ACCUMULO) && isIteratorOrConstraint(key))) {
+    if (!(namespaceId.equals(Namespace.ACCUMULO.id()) && isIteratorOrConstraint(key))) {
       getParent = parent;
     } else {
       // ignore iterators from parent if system namespace
@@ -104,13 +105,13 @@ public class NamespaceConfiguration extends ObservableConfiguration {
     Predicate<String> parentFilter = filter;
     // exclude system iterators/constraints from the system namespace
     // so they don't affect the metadata or root tables.
-    if (getNamespaceId().equals(Namespace.ID.ACCUMULO))
+    if (getNamespaceId().equals(Namespace.ACCUMULO.id()))
       parentFilter = key -> isIteratorOrConstraint(key) ? false : filter.test(key);
 
     getPropCacheAccessor().getProperties(props, getPath(), filter, parent, parentFilter);
   }
 
-  protected Namespace.ID getNamespaceId() {
+  protected NamespaceId getNamespaceId() {
     return namespaceId;
   }
 
diff --git a/server/base/src/main/java/org/apache/accumulo/server/conf/ServerConfiguration.java b/server/base/src/main/java/org/apache/accumulo/server/conf/ServerConfiguration.java
index 9a613f0..613eae0 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/conf/ServerConfiguration.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/conf/ServerConfiguration.java
@@ -16,15 +16,15 @@
  */
 package org.apache.accumulo.server.conf;
 
-import org.apache.accumulo.core.clientImpl.Namespace;
-import org.apache.accumulo.core.clientImpl.Table;
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
+import org.apache.accumulo.core.data.NamespaceId;
+import org.apache.accumulo.core.data.TableId;
 
 public abstract class ServerConfiguration {
 
-  public abstract TableConfiguration getTableConfiguration(Table.ID tableId);
+  public abstract TableConfiguration getTableConfiguration(TableId tableId);
 
-  public abstract NamespaceConfiguration getNamespaceConfiguration(Namespace.ID namespaceId);
+  public abstract NamespaceConfiguration getNamespaceConfiguration(NamespaceId namespaceId);
 
   public abstract AccumuloConfiguration getSystemConfiguration();
 
diff --git a/server/base/src/main/java/org/apache/accumulo/server/conf/ServerConfigurationFactory.java b/server/base/src/main/java/org/apache/accumulo/server/conf/ServerConfigurationFactory.java
index f655a4d..838ab30 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/conf/ServerConfigurationFactory.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/conf/ServerConfigurationFactory.java
@@ -20,13 +20,13 @@ import java.util.HashMap;
 import java.util.Map;
 
 import org.apache.accumulo.core.client.TableNotFoundException;
-import org.apache.accumulo.core.clientImpl.Namespace;
-import org.apache.accumulo.core.clientImpl.Table;
 import org.apache.accumulo.core.clientImpl.Tables;
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
 import org.apache.accumulo.core.conf.ConfigSanityCheck;
 import org.apache.accumulo.core.conf.DefaultConfiguration;
 import org.apache.accumulo.core.conf.SiteConfiguration;
+import org.apache.accumulo.core.data.NamespaceId;
+import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.fate.zookeeper.ZooCacheFactory;
 import org.apache.accumulo.server.ServerContext;
 
@@ -35,10 +35,10 @@ import org.apache.accumulo.server.ServerContext;
  */
 public class ServerConfigurationFactory extends ServerConfiguration {
 
-  private static final Map<String,Map<Table.ID,TableConfiguration>> tableConfigs = new HashMap<>(1);
-  private static final Map<String,Map<Namespace.ID,NamespaceConfiguration>> namespaceConfigs = new HashMap<>(
+  private static final Map<String,Map<TableId,TableConfiguration>> tableConfigs = new HashMap<>(1);
+  private static final Map<String,Map<NamespaceId,NamespaceConfiguration>> namespaceConfigs = new HashMap<>(
       1);
-  private static final Map<String,Map<Table.ID,NamespaceConfiguration>> tableParentConfigs = new HashMap<>(
+  private static final Map<String,Map<TableId,NamespaceConfiguration>> tableParentConfigs = new HashMap<>(
       1);
 
   private static void addInstanceToCaches(String iid) {
@@ -53,13 +53,13 @@ public class ServerConfigurationFactory extends ServerConfiguration {
     }
   }
 
-  static boolean removeCachedTableConfiguration(String instanceId, Table.ID tableId) {
+  static boolean removeCachedTableConfiguration(String instanceId, TableId tableId) {
     synchronized (tableConfigs) {
       return tableConfigs.get(instanceId).remove(tableId) != null;
     }
   }
 
-  static boolean removeCachedNamespaceConfiguration(String instanceId, Namespace.ID namespaceId) {
+  static boolean removeCachedNamespaceConfiguration(String instanceId, NamespaceId namespaceId) {
     synchronized (namespaceConfigs) {
       return namespaceConfigs.get(instanceId).remove(namespaceId) != null;
     }
@@ -79,7 +79,7 @@ public class ServerConfigurationFactory extends ServerConfiguration {
 
   static void expireAllTableObservers() {
     synchronized (tableConfigs) {
-      for (Map<Table.ID,TableConfiguration> instanceMap : tableConfigs.values()) {
+      for (Map<TableId,TableConfiguration> instanceMap : tableConfigs.values()) {
         for (TableConfiguration c : instanceMap.values()) {
           c.expireAllObservers();
         }
@@ -131,7 +131,7 @@ public class ServerConfigurationFactory extends ServerConfiguration {
   }
 
   @Override
-  public TableConfiguration getTableConfiguration(Table.ID tableId) {
+  public TableConfiguration getTableConfiguration(TableId tableId) {
     TableConfiguration conf;
     synchronized (tableConfigs) {
       conf = tableConfigs.get(instanceID).get(tableId);
@@ -151,7 +151,7 @@ public class ServerConfigurationFactory extends ServerConfiguration {
       conf = new TableConfiguration(context, tableId, getNamespaceConfigurationForTable(tableId));
       ConfigSanityCheck.validate(conf);
       synchronized (tableConfigs) {
-        Map<Table.ID,TableConfiguration> configs = tableConfigs.get(instanceID);
+        Map<TableId,TableConfiguration> configs = tableConfigs.get(instanceID);
         TableConfiguration existingConf = configs.get(tableId);
         if (existingConf == null) {
           // Configuration doesn't exist yet
@@ -165,7 +165,7 @@ public class ServerConfigurationFactory extends ServerConfiguration {
     return conf;
   }
 
-  public NamespaceConfiguration getNamespaceConfigurationForTable(Table.ID tableId) {
+  public NamespaceConfiguration getNamespaceConfigurationForTable(TableId tableId) {
     NamespaceConfiguration conf;
     synchronized (tableParentConfigs) {
       conf = tableParentConfigs.get(instanceID).get(tableId);
@@ -173,7 +173,7 @@ public class ServerConfigurationFactory extends ServerConfiguration {
     // can't hold the lock during the construction and validation of the config,
     // which may result in creating multiple objects for the same id, but that's ok.
     if (conf == null) {
-      Namespace.ID namespaceId;
+      NamespaceId namespaceId;
       try {
         namespaceId = Tables.getNamespaceId(context, tableId);
       } catch (TableNotFoundException e) {
@@ -189,7 +189,7 @@ public class ServerConfigurationFactory extends ServerConfiguration {
   }
 
   @Override
-  public NamespaceConfiguration getNamespaceConfiguration(Namespace.ID namespaceId) {
+  public NamespaceConfiguration getNamespaceConfiguration(NamespaceId namespaceId) {
     NamespaceConfiguration conf;
     // can't hold the lock during the construction and validation of the config,
     // which may result in creating multiple objects for the same id, but that's ok.
diff --git a/server/base/src/main/java/org/apache/accumulo/server/conf/TableConfWatcher.java b/server/base/src/main/java/org/apache/accumulo/server/conf/TableConfWatcher.java
index dbc8167..1230fd1 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/conf/TableConfWatcher.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/conf/TableConfWatcher.java
@@ -17,7 +17,7 @@
 package org.apache.accumulo.server.conf;
 
 import org.apache.accumulo.core.Constants;
-import org.apache.accumulo.core.clientImpl.Table;
+import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.server.ServerContext;
 import org.apache.log4j.Level;
 import org.apache.log4j.Logger;
@@ -72,7 +72,7 @@ class TableConfWatcher implements Watcher {
         return;
       }
     }
-    Table.ID tableId = Table.ID.of(tableIdString);
+    TableId tableId = TableId.of(tableIdString);
 
     switch (event.getType()) {
       case NodeDataChanged:
diff --git a/server/base/src/main/java/org/apache/accumulo/server/conf/TableConfiguration.java b/server/base/src/main/java/org/apache/accumulo/server/conf/TableConfiguration.java
index 3c433a1..749b04b 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/conf/TableConfiguration.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/conf/TableConfiguration.java
@@ -28,10 +28,10 @@ import java.util.concurrent.atomic.AtomicReference;
 import java.util.function.Predicate;
 
 import org.apache.accumulo.core.Constants;
-import org.apache.accumulo.core.clientImpl.Table;
 import org.apache.accumulo.core.conf.ConfigurationObserver;
 import org.apache.accumulo.core.conf.ObservableConfiguration;
 import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.dataImpl.thrift.IterInfo;
 import org.apache.accumulo.core.iterators.IteratorUtil;
 import org.apache.accumulo.core.iterators.IteratorUtil.IteratorScope;
@@ -57,12 +57,11 @@ public class TableConfiguration extends ObservableConfiguration {
   private final NamespaceConfiguration parent;
   private ZooCacheFactory zcf = new ZooCacheFactory();
 
-  private final Table.ID tableId;
+  private final TableId tableId;
 
   private EnumMap<IteratorScope,AtomicReference<ParsedIteratorConfig>> iteratorConfig;
 
-  public TableConfiguration(ServerContext context, Table.ID tableId,
-      NamespaceConfiguration parent) {
+  public TableConfiguration(ServerContext context, TableId tableId, NamespaceConfiguration parent) {
     this.context = requireNonNull(context);
     this.tableId = requireNonNull(tableId);
     this.parent = requireNonNull(parent);
@@ -80,7 +79,7 @@ public class TableConfiguration extends ObservableConfiguration {
   private synchronized ZooCachePropertyAccessor getPropCacheAccessor() {
     if (propCacheAccessor == null) {
       synchronized (propCaches) {
-        PropCacheKey key = new PropCacheKey(context.getInstanceID(), tableId.canonicalID());
+        PropCacheKey key = new PropCacheKey(context.getInstanceID(), tableId.canonical());
         ZooCache propCache = propCaches.get(key);
         if (propCache == null) {
           propCache = zcf.getZooCache(context.getZooKeepers(),
@@ -128,7 +127,7 @@ public class TableConfiguration extends ObservableConfiguration {
     getPropCacheAccessor().getProperties(props, getPath(), filter, parent, null);
   }
 
-  public Table.ID getTableId() {
+  public TableId getTableId() {
     return tableId;
   }
 
diff --git a/server/base/src/main/java/org/apache/accumulo/server/constraints/MetadataConstraints.java b/server/base/src/main/java/org/apache/accumulo/server/constraints/MetadataConstraints.java
index ec4ee20..7de61d1 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/constraints/MetadataConstraints.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/constraints/MetadataConstraints.java
@@ -156,7 +156,7 @@ public class MetadataConstraints implements Constraint {
     }
 
     // ensure row is not less than Constants.METADATA_TABLE_ID
-    if (new Text(row).compareTo(new Text(MetadataTable.ID.getUtf8())) < 0) {
+    if (new Text(row).compareTo(new Text(MetadataTable.ID.canonical())) < 0) {
       violations = addViolation(violations, 5);
     }
 
diff --git a/server/base/src/main/java/org/apache/accumulo/server/fs/PerTableVolumeChooser.java b/server/base/src/main/java/org/apache/accumulo/server/fs/PerTableVolumeChooser.java
index 5aab9d2..82268f0 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/fs/PerTableVolumeChooser.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/fs/PerTableVolumeChooser.java
@@ -19,10 +19,10 @@ package org.apache.accumulo.server.fs;
 import java.io.IOException;
 import java.util.concurrent.ConcurrentHashMap;
 
-import org.apache.accumulo.core.clientImpl.Table;
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
 import org.apache.accumulo.core.conf.ConfigurationTypeHelper;
 import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.server.conf.ServerConfigurationFactory;
 import org.apache.accumulo.server.conf.TableConfiguration;
 import org.apache.accumulo.server.fs.VolumeChooserEnvironment.ChooserScope;
@@ -41,7 +41,7 @@ public class PerTableVolumeChooser implements VolumeChooser {
   private static final Logger log = LoggerFactory.getLogger(PerTableVolumeChooser.class);
   // TODO Add hint of expected size to construction, see ACCUMULO-3410
   /* Track VolumeChooser instances so they can keep state. */
-  private final ConcurrentHashMap<Table.ID,VolumeChooser> tableSpecificChooserCache = new ConcurrentHashMap<>();
+  private final ConcurrentHashMap<TableId,VolumeChooser> tableSpecificChooserCache = new ConcurrentHashMap<>();
   private final ConcurrentHashMap<ChooserScope,VolumeChooser> scopeSpecificChooserCache = new ConcurrentHashMap<>();
   private final RandomVolumeChooser randomChooser = new RandomVolumeChooser();
 
diff --git a/server/base/src/main/java/org/apache/accumulo/server/fs/VolumeChooserEnvironment.java b/server/base/src/main/java/org/apache/accumulo/server/fs/VolumeChooserEnvironment.java
index a1a4f3e..37f5571 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/fs/VolumeChooserEnvironment.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/fs/VolumeChooserEnvironment.java
@@ -18,7 +18,7 @@ package org.apache.accumulo.server.fs;
 
 import java.util.Objects;
 
-import org.apache.accumulo.core.clientImpl.Table;
+import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.server.ServerContext;
 import org.apache.hadoop.io.Text;
 
@@ -35,7 +35,7 @@ public class VolumeChooserEnvironment {
 
   private final ServerContext context;
   private final ChooserScope scope;
-  private final Table.ID tableId;
+  private final TableId tableId;
   private final Text endRow;
 
   public VolumeChooserEnvironment(ChooserScope scope, ServerContext context) {
@@ -45,7 +45,7 @@ public class VolumeChooserEnvironment {
     this.endRow = null;
   }
 
-  public VolumeChooserEnvironment(Table.ID tableId, Text endRow, ServerContext context) {
+  public VolumeChooserEnvironment(TableId tableId, Text endRow, ServerContext context) {
     this.context = context;
     this.scope = ChooserScope.TABLE;
     this.tableId = Objects.requireNonNull(tableId);
@@ -65,7 +65,7 @@ public class VolumeChooserEnvironment {
     return endRow;
   }
 
-  public Table.ID getTableId() {
+  public TableId getTableId() {
     return tableId;
   }
 
diff --git a/server/base/src/main/java/org/apache/accumulo/server/fs/VolumeManager.java b/server/base/src/main/java/org/apache/accumulo/server/fs/VolumeManager.java
index 3e72829..9b5a0f8 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/fs/VolumeManager.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/fs/VolumeManager.java
@@ -19,8 +19,8 @@ package org.apache.accumulo.server.fs;
 import java.io.IOException;
 import java.util.Collection;
 
-import org.apache.accumulo.core.clientImpl.Table;
 import org.apache.accumulo.core.data.Key;
+import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.volume.Volume;
 import org.apache.accumulo.server.ServerConstants;
 import org.apache.hadoop.fs.ContentSummary;
@@ -155,7 +155,7 @@ public interface VolumeManager {
   // Convert a file or directory metadata reference into a path
   Path getFullPath(Key key);
 
-  Path getFullPath(Table.ID tableId, String path);
+  Path getFullPath(TableId tableId, String path);
 
   // Given a filename, figure out the qualified path given multiple namespaces
   Path getFullPath(FileType fileType, String fileName);
diff --git a/server/base/src/main/java/org/apache/accumulo/server/fs/VolumeManagerImpl.java b/server/base/src/main/java/org/apache/accumulo/server/fs/VolumeManagerImpl.java
index 0ba12b4..339f924 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/fs/VolumeManagerImpl.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/fs/VolumeManagerImpl.java
@@ -28,11 +28,11 @@ import java.util.HashSet;
 import java.util.Map;
 import java.util.Map.Entry;
 
-import org.apache.accumulo.core.clientImpl.Table;
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
 import org.apache.accumulo.core.conf.DefaultConfiguration;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.data.Key;
+import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.dataImpl.KeyExtent;
 import org.apache.accumulo.core.file.rfile.RFile;
 import org.apache.accumulo.core.volume.NonConfiguredVolume;
@@ -391,7 +391,7 @@ public class VolumeManagerImpl implements VolumeManager {
     // TODO sanity check col fam
     String relPath = key.getColumnQualifierData().toString();
     byte[] tableId = KeyExtent.tableOfMetadataRow(key.getRow());
-    return getFullPath(Table.ID.of(new String(tableId)), relPath);
+    return getFullPath(TableId.of(new String(tableId)), relPath);
   }
 
   @Override
@@ -418,14 +418,14 @@ public class VolumeManagerImpl implements VolumeManager {
   }
 
   @Override
-  public Path getFullPath(Table.ID tableId, String path) {
+  public Path getFullPath(TableId tableId, String path) {
     if (path.contains(":"))
       return new Path(path);
 
     if (path.startsWith("../"))
       path = path.substring(2);
     else if (path.startsWith("/"))
-      path = "/" + tableId.canonicalID() + path;
+      path = "/" + tableId.canonical() + path;
     else
       throw new IllegalArgumentException("Unexpected path prefix " + path);
 
diff --git a/server/base/src/main/java/org/apache/accumulo/server/init/Initialize.java b/server/base/src/main/java/org/apache/accumulo/server/init/Initialize.java
index e7cb605..c895f20 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/init/Initialize.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/init/Initialize.java
@@ -41,7 +41,6 @@ import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.IteratorSetting;
 import org.apache.accumulo.core.client.IteratorSetting.Column;
 import org.apache.accumulo.core.clientImpl.Namespace;
-import org.apache.accumulo.core.clientImpl.Table;
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
 import org.apache.accumulo.core.conf.DefaultConfiguration;
 import org.apache.accumulo.core.conf.Property;
@@ -49,6 +48,7 @@ import org.apache.accumulo.core.conf.SiteConfiguration;
 import org.apache.accumulo.core.crypto.CryptoServiceFactory;
 import org.apache.accumulo.core.crypto.CryptoServiceFactory.ClassloaderType;
 import org.apache.accumulo.core.data.Key;
+import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.dataImpl.KeyExtent;
 import org.apache.accumulo.core.file.FileOperations;
@@ -523,12 +523,12 @@ public class Initialize implements KeywordExecutable {
   }
 
   private static class Tablet {
-    Table.ID tableId;
+    TableId tableId;
     String dir;
     Text prevEndRow, endRow;
     String[] files;
 
-    Tablet(Table.ID tableId, String dir, Text prevEndRow, Text endRow, String... files) {
+    Tablet(TableId tableId, String dir, Text prevEndRow, Text endRow, String... files) {
       this.tableId = tableId;
       this.dir = dir;
       this.prevEndRow = prevEndRow;
@@ -617,15 +617,15 @@ public class Initialize implements KeywordExecutable {
         NodeExistsPolicy.FAIL);
     zoo.putPersistentData(zkInstanceRoot + Constants.ZNAMESPACES, new byte[0],
         NodeExistsPolicy.FAIL);
-    TableManager.prepareNewNamespaceState(zoo, uuid, Namespace.ID.DEFAULT, Namespace.DEFAULT,
-        NodeExistsPolicy.FAIL);
-    TableManager.prepareNewNamespaceState(zoo, uuid, Namespace.ID.ACCUMULO, Namespace.ACCUMULO,
-        NodeExistsPolicy.FAIL);
-    TableManager.prepareNewTableState(zoo, uuid, RootTable.ID, Namespace.ID.ACCUMULO,
+    TableManager.prepareNewNamespaceState(zoo, uuid, Namespace.DEFAULT.id(),
+        Namespace.DEFAULT.name(), NodeExistsPolicy.FAIL);
+    TableManager.prepareNewNamespaceState(zoo, uuid, Namespace.ACCUMULO.id(),
+        Namespace.ACCUMULO.name(), NodeExistsPolicy.FAIL);
+    TableManager.prepareNewTableState(zoo, uuid, RootTable.ID, Namespace.ACCUMULO.id(),
         RootTable.NAME, TableState.ONLINE, NodeExistsPolicy.FAIL);
-    TableManager.prepareNewTableState(zoo, uuid, MetadataTable.ID, Namespace.ID.ACCUMULO,
+    TableManager.prepareNewTableState(zoo, uuid, MetadataTable.ID, Namespace.ACCUMULO.id(),
         MetadataTable.NAME, TableState.ONLINE, NodeExistsPolicy.FAIL);
-    TableManager.prepareNewTableState(zoo, uuid, ReplicationTable.ID, Namespace.ID.ACCUMULO,
+    TableManager.prepareNewTableState(zoo, uuid, ReplicationTable.ID, Namespace.ACCUMULO.id(),
         ReplicationTable.NAME, TableState.OFFLINE, NodeExistsPolicy.FAIL);
     zoo.putPersistentData(zkInstanceRoot + Constants.ZTSERVERS, EMPTY_BYTE_ARRAY,
         NodeExistsPolicy.FAIL);
diff --git a/server/base/src/main/java/org/apache/accumulo/server/master/LiveTServerSet.java b/server/base/src/main/java/org/apache/accumulo/server/master/LiveTServerSet.java
index 9cf178e..a59831c 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/master/LiveTServerSet.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/master/LiveTServerSet.java
@@ -27,8 +27,8 @@ import java.util.Map.Entry;
 import java.util.Set;
 
 import org.apache.accumulo.core.Constants;
-import org.apache.accumulo.core.clientImpl.Table;
 import org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException;
+import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.dataImpl.KeyExtent;
 import org.apache.accumulo.core.master.thrift.TabletServerStatus;
 import org.apache.accumulo.core.rpc.ThriftUtil;
@@ -162,13 +162,13 @@ public class LiveTServerSet implements Watcher {
       }
     }
 
-    public void flush(ZooLock lock, Table.ID tableId, byte[] startRow, byte[] endRow)
+    public void flush(ZooLock lock, TableId tableId, byte[] startRow, byte[] endRow)
         throws TException {
       TabletClientService.Client client = ThriftUtil
           .getClient(new TabletClientService.Client.Factory(), address, context);
       try {
-        client.flush(Tracer.traceInfo(), context.rpcCreds(), lockString(lock),
-            tableId.canonicalID(), startRow == null ? null : ByteBuffer.wrap(startRow),
+        client.flush(Tracer.traceInfo(), context.rpcCreds(), lockString(lock), tableId.canonical(),
+            startRow == null ? null : ByteBuffer.wrap(startRow),
             endRow == null ? null : ByteBuffer.wrap(endRow));
       } finally {
         ThriftUtil.returnClient(client);
diff --git a/server/base/src/main/java/org/apache/accumulo/server/master/balancer/ChaoticLoadBalancer.java b/server/base/src/main/java/org/apache/accumulo/server/master/balancer/ChaoticLoadBalancer.java
index c5521c8..bd0f284 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/master/balancer/ChaoticLoadBalancer.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/master/balancer/ChaoticLoadBalancer.java
@@ -26,8 +26,8 @@ import java.util.Random;
 import java.util.Set;
 import java.util.SortedMap;
 
-import org.apache.accumulo.core.clientImpl.Table;
 import org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException;
+import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.dataImpl.KeyExtent;
 import org.apache.accumulo.core.master.thrift.TableInfo;
 import org.apache.accumulo.core.master.thrift.TabletServerStatus;
@@ -133,7 +133,7 @@ public class ChaoticLoadBalancer extends TabletBalancer {
 
     for (Entry<TServerInstance,TabletServerStatus> e : current.entrySet()) {
       for (String tableId : e.getValue().getTableMap().keySet()) {
-        Table.ID id = Table.ID.of(tableId);
+        TableId id = TableId.of(tableId);
         if (!moveMetadata && MetadataTable.ID.equals(id))
           continue;
         try {
diff --git a/server/base/src/main/java/org/apache/accumulo/server/master/balancer/DefaultLoadBalancer.java b/server/base/src/main/java/org/apache/accumulo/server/master/balancer/DefaultLoadBalancer.java
index ec071f3..021b5ab 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/master/balancer/DefaultLoadBalancer.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/master/balancer/DefaultLoadBalancer.java
@@ -27,7 +27,7 @@ import java.util.Objects;
 import java.util.Set;
 import java.util.SortedMap;
 
-import org.apache.accumulo.core.clientImpl.Table;
+import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.dataImpl.KeyExtent;
 import org.apache.accumulo.core.master.thrift.TableInfo;
 import org.apache.accumulo.core.master.thrift.TabletServerStatus;
@@ -43,13 +43,13 @@ public class DefaultLoadBalancer extends TabletBalancer {
 
   Iterator<TServerInstance> assignments;
   // if tableToBalance is set, then only balance the given table
-  Table.ID tableToBalance = null;
+  TableId tableToBalance = null;
 
   public DefaultLoadBalancer() {
 
   }
 
-  public DefaultLoadBalancer(Table.ID table) {
+  public DefaultLoadBalancer(TableId table) {
     tableToBalance = table;
   }
 
@@ -128,7 +128,7 @@ public class DefaultLoadBalancer extends TabletBalancer {
       if (current.size() < 2) {
         return false;
       }
-      final Map<Table.ID,Map<KeyExtent,TabletStats>> donerTabletStats = new HashMap<>();
+      final Map<TableId,Map<KeyExtent,TabletStats>> donerTabletStats = new HashMap<>();
 
       // Sort by total number of online tablets, per server
       int total = 0;
@@ -141,7 +141,7 @@ public class DefaultLoadBalancer extends TabletBalancer {
              * The check below was on entry.getKey(), but that resolves to a tabletserver not a
              * tablename. Believe it should be e.getKey() which is a tablename
              */
-            if (tableToBalance == null || tableToBalance.canonicalID().equals(e.getKey()))
+            if (tableToBalance == null || tableToBalance.canonical().equals(e.getKey()))
               serverTotal += e.getValue().onlineTablets;
           }
         }
@@ -203,26 +203,26 @@ public class DefaultLoadBalancer extends TabletBalancer {
    * busiest table
    */
   List<TabletMigration> move(ServerCounts tooMuch, ServerCounts tooLittle, int count,
-      Map<Table.ID,Map<KeyExtent,TabletStats>> donerTabletStats) {
+      Map<TableId,Map<KeyExtent,TabletStats>> donerTabletStats) {
 
     List<TabletMigration> result = new ArrayList<>();
     if (count == 0)
       return result;
 
     // Copy counts so we can update them as we propose migrations
-    Map<Table.ID,Integer> tooMuchMap = tabletCountsPerTable(tooMuch.status);
-    Map<Table.ID,Integer> tooLittleMap = tabletCountsPerTable(tooLittle.status);
+    Map<TableId,Integer> tooMuchMap = tabletCountsPerTable(tooMuch.status);
+    Map<TableId,Integer> tooLittleMap = tabletCountsPerTable(tooLittle.status);
 
     for (int i = 0; i < count; i++) {
-      Table.ID table;
+      TableId table;
       Integer tooLittleCount;
       if (tableToBalance == null) {
         // find a table to migrate
         // look for an uneven table count
         int biggestDifference = 0;
-        Table.ID biggestDifferenceTable = null;
-        for (Entry<Table.ID,Integer> tableEntry : tooMuchMap.entrySet()) {
-          Table.ID tableID = tableEntry.getKey();
+        TableId biggestDifferenceTable = null;
+        for (Entry<TableId,Integer> tableEntry : tooMuchMap.entrySet()) {
+          TableId tableID = tableEntry.getKey();
           if (tooLittleMap.get(tableID) == null)
             tooLittleMap.put(tableID, 0);
           int diff = tableEntry.getValue() - tooLittleMap.get(tableID);
@@ -279,12 +279,12 @@ public class DefaultLoadBalancer extends TabletBalancer {
     return result;
   }
 
-  static Map<Table.ID,Integer> tabletCountsPerTable(TabletServerStatus status) {
-    Map<Table.ID,Integer> result = new HashMap<>();
+  static Map<TableId,Integer> tabletCountsPerTable(TabletServerStatus status) {
+    Map<TableId,Integer> result = new HashMap<>();
     if (status != null && status.tableMap != null) {
       Map<String,TableInfo> tableMap = status.tableMap;
       for (Entry<String,TableInfo> entry : tableMap.entrySet()) {
-        result.put(Table.ID.of(entry.getKey()), entry.getValue().onlineTablets);
+        result.put(TableId.of(entry.getKey()), entry.getValue().onlineTablets);
       }
     }
     return result;
@@ -304,15 +304,15 @@ public class DefaultLoadBalancer extends TabletBalancer {
   }
 
   // define what it means for a tablet to be busy
-  private static Table.ID busiest(Map<String,TableInfo> tables) {
-    Table.ID result = null;
+  private static TableId busiest(Map<String,TableInfo> tables) {
+    TableId result = null;
     double busiest = Double.NEGATIVE_INFINITY;
     for (Entry<String,TableInfo> entry : tables.entrySet()) {
       TableInfo info = entry.getValue();
       double busy = info.ingestRate + info.queryRate;
       if (busy > busiest) {
         busiest = busy;
-        result = Table.ID.of(entry.getKey());
+        result = TableId.of(entry.getKey());
       }
     }
     return result;
diff --git a/server/base/src/main/java/org/apache/accumulo/server/master/balancer/GroupBalancer.java b/server/base/src/main/java/org/apache/accumulo/server/master/balancer/GroupBalancer.java
index 533fbb9..f0d1d85 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/master/balancer/GroupBalancer.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/master/balancer/GroupBalancer.java
@@ -33,7 +33,7 @@ import java.util.Set;
 import java.util.SortedMap;
 import java.util.function.Function;
 
-import org.apache.accumulo.core.clientImpl.Table;
+import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.dataImpl.KeyExtent;
 import org.apache.accumulo.core.master.thrift.TabletServerStatus;
 import org.apache.accumulo.core.metadata.schema.TabletsMetadata;
@@ -64,7 +64,7 @@ import com.google.common.collect.Multimap;
  */
 public abstract class GroupBalancer extends TabletBalancer {
 
-  private final Table.ID tableId;
+  private final TableId tableId;
   private long lastRun = 0;
 
   /**
@@ -72,7 +72,7 @@ public abstract class GroupBalancer extends TabletBalancer {
    */
   protected abstract Function<KeyExtent,String> getPartitioner();
 
-  public GroupBalancer(Table.ID tableId) {
+  public GroupBalancer(TableId tableId) {
     this.tableId = tableId;
   }
 
diff --git a/server/base/src/main/java/org/apache/accumulo/server/master/balancer/HostRegexTableLoadBalancer.java b/server/base/src/main/java/org/apache/accumulo/server/master/balancer/HostRegexTableLoadBalancer.java
index 26f5aec..2f0cd8b 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/master/balancer/HostRegexTableLoadBalancer.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/master/balancer/HostRegexTableLoadBalancer.java
@@ -34,10 +34,10 @@ import java.util.TreeMap;
 import java.util.regex.Pattern;
 
 import org.apache.accumulo.core.client.admin.TableOperations;
-import org.apache.accumulo.core.clientImpl.Table;
 import org.apache.accumulo.core.conf.ConfigurationObserver;
 import org.apache.accumulo.core.conf.ConfigurationTypeHelper;
 import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.dataImpl.KeyExtent;
 import org.apache.accumulo.core.master.thrift.TableInfo;
 import org.apache.accumulo.core.master.thrift.TabletServerStatus;
@@ -106,7 +106,7 @@ public class HostRegexTableLoadBalancer extends TableLoadBalancer implements Con
   private static final long ONE_HOUR = 60 * 60 * 1000;
   private static final Set<KeyExtent> EMPTY_MIGRATIONS = Collections.emptySet();
 
-  private Map<Table.ID,String> tableIdToTableName = null;
+  private Map<TableId,String> tableIdToTableName = null;
   private Map<String,Pattern> poolNameToRegexPattern = null;
   private volatile long lastOOBCheck = System.currentTimeMillis();
   private volatile boolean isIpBasedRegex = false;
@@ -224,7 +224,7 @@ public class HostRegexTableLoadBalancer extends TableLoadBalancer implements Con
     tableIdToTableName = new HashMap<>();
     poolNameToRegexPattern = new HashMap<>();
     for (Entry<String,String> table : t.tableIdMap().entrySet()) {
-      Table.ID tableId = Table.ID.of(table.getValue());
+      TableId tableId = TableId.of(table.getValue());
       tableIdToTableName.put(tableId, table.getKey());
       conf.getTableConfiguration(tableId).addObserver(this);
       Map<String,String> customProps = conf.getTableConfiguration(tableId)
@@ -275,7 +275,7 @@ public class HostRegexTableLoadBalancer extends TableLoadBalancer implements Con
     return buf.toString();
   }
 
-  public Map<Table.ID,String> getTableIdToTableName() {
+  public Map<TableId,String> getTableIdToTableName() {
     return tableIdToTableName;
   }
 
@@ -311,7 +311,7 @@ public class HostRegexTableLoadBalancer extends TableLoadBalancer implements Con
 
     Map<String,SortedMap<TServerInstance,TabletServerStatus>> pools = splitCurrentByRegex(current);
     // group the unassigned into tables
-    Map<Table.ID,Map<KeyExtent,TServerInstance>> groupedUnassigned = new HashMap<>();
+    Map<TableId,Map<KeyExtent,TServerInstance>> groupedUnassigned = new HashMap<>();
     for (Entry<KeyExtent,TServerInstance> e : unassigned.entrySet()) {
       Map<KeyExtent,TServerInstance> tableUnassigned = groupedUnassigned
           .get(e.getKey().getTableId());
@@ -322,7 +322,7 @@ public class HostRegexTableLoadBalancer extends TableLoadBalancer implements Con
       tableUnassigned.put(e.getKey(), e.getValue());
     }
     // Send a view of the current servers to the tables tablet balancer
-    for (Entry<Table.ID,Map<KeyExtent,TServerInstance>> e : groupedUnassigned.entrySet()) {
+    for (Entry<TableId,Map<KeyExtent,TServerInstance>> e : groupedUnassigned.entrySet()) {
       Map<KeyExtent,TServerInstance> newAssignments = new HashMap<>();
       String tableName = tableIdToTableName.get(e.getKey());
       String poolName = getPoolNameForTable(tableName);
@@ -379,7 +379,7 @@ public class HostRegexTableLoadBalancer extends TableLoadBalancer implements Con
             }
             try {
               List<TabletStats> outOfBoundsTablets = getOnlineTabletsForTable(e.getKey(),
-                  Table.ID.of(tid));
+                  TableId.of(tid));
               if (outOfBoundsTablets == null) {
                 continue;
               }
@@ -463,7 +463,7 @@ public class HostRegexTableLoadBalancer extends TableLoadBalancer implements Con
     }
 
     for (String s : tableIdMap.values()) {
-      Table.ID tableId = Table.ID.of(s);
+      TableId tableId = TableId.of(s);
       String tableName = tableIdToTableName.get(tableId);
       String regexTableName = getPoolNameForTable(tableName);
       SortedMap<TServerInstance,TabletServerStatus> currentView = currentGrouped
diff --git a/server/base/src/main/java/org/apache/accumulo/server/master/balancer/RegexGroupBalancer.java b/server/base/src/main/java/org/apache/accumulo/server/master/balancer/RegexGroupBalancer.java
index 912c24a..6e1b6e1 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/master/balancer/RegexGroupBalancer.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/master/balancer/RegexGroupBalancer.java
@@ -22,9 +22,9 @@ import java.util.function.Function;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
-import org.apache.accumulo.core.clientImpl.Table;
 import org.apache.accumulo.core.conf.ConfigurationTypeHelper;
 import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.dataImpl.KeyExtent;
 import org.apache.hadoop.io.Text;
 
@@ -53,9 +53,9 @@ public class RegexGroupBalancer extends GroupBalancer {
   public static final String WAIT_TIME_PROPERTY = Property.TABLE_ARBITRARY_PROP_PREFIX.getKey()
       + "balancer.group.regex.wait.time";
 
-  private final Table.ID tableId;
+  private final TableId tableId;
 
-  public RegexGroupBalancer(Table.ID tableId) {
+  public RegexGroupBalancer(TableId tableId) {
     super(tableId);
     this.tableId = tableId;
   }
diff --git a/server/base/src/main/java/org/apache/accumulo/server/master/balancer/TableLoadBalancer.java b/server/base/src/main/java/org/apache/accumulo/server/master/balancer/TableLoadBalancer.java
index ec220d6..511d634 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/master/balancer/TableLoadBalancer.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/master/balancer/TableLoadBalancer.java
@@ -26,8 +26,8 @@ import java.util.Set;
 import java.util.SortedMap;
 
 import org.apache.accumulo.core.client.admin.TableOperations;
-import org.apache.accumulo.core.clientImpl.Table;
 import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.dataImpl.KeyExtent;
 import org.apache.accumulo.core.master.state.tables.TableState;
 import org.apache.accumulo.core.master.thrift.TabletServerStatus;
@@ -41,9 +41,9 @@ public class TableLoadBalancer extends TabletBalancer {
 
   private static final Logger log = LoggerFactory.getLogger(TableLoadBalancer.class);
 
-  Map<Table.ID,TabletBalancer> perTableBalancers = new HashMap<>();
+  Map<TableId,TabletBalancer> perTableBalancers = new HashMap<>();
 
-  private TabletBalancer constructNewBalancerForTable(String clazzName, Table.ID tableId)
+  private TabletBalancer constructNewBalancerForTable(String clazzName, TableId tableId)
       throws Exception {
     String context = null;
     context = this.context.getServerConfFactory().getTableConfiguration(tableId)
@@ -54,11 +54,11 @@ public class TableLoadBalancer extends TabletBalancer {
           TabletBalancer.class);
     else
       clazz = AccumuloVFSClassLoader.loadClass(clazzName, TabletBalancer.class);
-    Constructor<? extends TabletBalancer> constructor = clazz.getConstructor(Table.ID.class);
+    Constructor<? extends TabletBalancer> constructor = clazz.getConstructor(TableId.class);
     return constructor.newInstance(tableId);
   }
 
-  protected String getLoadBalancerClassNameForTable(Table.ID table) {
+  protected String getLoadBalancerClassNameForTable(TableId table) {
     TableState tableState = context.getTableManager().getTableState(table);
     if (tableState == null)
       return null;
@@ -68,7 +68,7 @@ public class TableLoadBalancer extends TabletBalancer {
     return null;
   }
 
-  protected TabletBalancer getBalancerForTable(Table.ID tableId) {
+  protected TabletBalancer getBalancerForTable(TableId tableId) {
     TabletBalancer balancer = perTableBalancers.get(tableId);
 
     String clazzName = getLoadBalancerClassNameForTable(tableId);
@@ -115,7 +115,7 @@ public class TableLoadBalancer extends TabletBalancer {
   public void getAssignments(SortedMap<TServerInstance,TabletServerStatus> current,
       Map<KeyExtent,TServerInstance> unassigned, Map<KeyExtent,TServerInstance> assignments) {
     // separate the unassigned into tables
-    Map<Table.ID,Map<KeyExtent,TServerInstance>> groupedUnassigned = new HashMap<>();
+    Map<TableId,Map<KeyExtent,TServerInstance>> groupedUnassigned = new HashMap<>();
     for (Entry<KeyExtent,TServerInstance> e : unassigned.entrySet()) {
       Map<KeyExtent,TServerInstance> tableUnassigned = groupedUnassigned
           .get(e.getKey().getTableId());
@@ -125,7 +125,7 @@ public class TableLoadBalancer extends TabletBalancer {
       }
       tableUnassigned.put(e.getKey(), e.getValue());
     }
-    for (Entry<Table.ID,Map<KeyExtent,TServerInstance>> e : groupedUnassigned.entrySet()) {
+    for (Entry<TableId,Map<KeyExtent,TServerInstance>> e : groupedUnassigned.entrySet()) {
       Map<KeyExtent,TServerInstance> newAssignments = new HashMap<>();
       getBalancerForTable(e.getKey()).getAssignments(current, e.getValue(), newAssignments);
       assignments.putAll(newAssignments);
@@ -150,7 +150,7 @@ public class TableLoadBalancer extends TabletBalancer {
       return minBalanceTime;
     for (String s : t.tableIdMap().values()) {
       ArrayList<TabletMigration> newMigrations = new ArrayList<>();
-      long tableBalanceTime = getBalancerForTable(Table.ID.of(s)).balance(current, migrations,
+      long tableBalanceTime = getBalancerForTable(TableId.of(s)).balance(current, migrations,
           newMigrations);
       if (tableBalanceTime < minBalanceTime)
         minBalanceTime = tableBalanceTime;
diff --git a/server/base/src/main/java/org/apache/accumulo/server/master/balancer/TabletBalancer.java b/server/base/src/main/java/org/apache/accumulo/server/master/balancer/TabletBalancer.java
index 25dcbc0..8a5594e 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/master/balancer/TabletBalancer.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/master/balancer/TabletBalancer.java
@@ -23,9 +23,9 @@ import java.util.Map;
 import java.util.Set;
 import java.util.SortedMap;
 
-import org.apache.accumulo.core.clientImpl.Table;
 import org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException;
 import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.dataImpl.KeyExtent;
 import org.apache.accumulo.core.master.thrift.TabletServerStatus;
 import org.apache.accumulo.core.rpc.ThriftUtil;
@@ -223,13 +223,13 @@ public abstract class TabletBalancer {
    * @throws TException
    *           any other problem
    */
-  public List<TabletStats> getOnlineTabletsForTable(TServerInstance tserver, Table.ID tableId)
+  public List<TabletStats> getOnlineTabletsForTable(TServerInstance tserver, TableId tableId)
       throws ThriftSecurityException, TException {
     log.debug("Scanning tablet server {} for table {}", tserver, tableId);
     Client client = ThriftUtil.getClient(new TabletClientService.Client.Factory(),
         tserver.getLocation(), context);
     try {
-      return client.getTabletStats(Tracer.traceInfo(), context.rpcCreds(), tableId.canonicalID());
+      return client.getTabletStats(Tracer.traceInfo(), context.rpcCreds(), tableId.canonical());
     } catch (TTransportException e) {
       log.error("Unable to connect to {}: ", tserver, e);
     } finally {
diff --git a/server/base/src/main/java/org/apache/accumulo/server/master/state/CurrentState.java b/server/base/src/main/java/org/apache/accumulo/server/master/state/CurrentState.java
index 1311f74..c1b496d 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/master/state/CurrentState.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/master/state/CurrentState.java
@@ -19,13 +19,13 @@ package org.apache.accumulo.server.master.state;
 import java.util.Collection;
 import java.util.Set;
 
-import org.apache.accumulo.core.clientImpl.Table;
+import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.dataImpl.KeyExtent;
 import org.apache.accumulo.core.master.thrift.MasterState;
 
 public interface CurrentState {
 
-  Set<Table.ID> onlineTables();
+  Set<TableId> onlineTables();
 
   Set<TServerInstance> onlineTabletServers();
 
diff --git a/server/base/src/main/java/org/apache/accumulo/server/master/state/TabletStateChangeIterator.java b/server/base/src/main/java/org/apache/accumulo/server/master/state/TabletStateChangeIterator.java
index 1d81862..89cdcfa 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/master/state/TabletStateChangeIterator.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/master/state/TabletStateChangeIterator.java
@@ -29,8 +29,8 @@ import java.util.Map;
 import java.util.Set;
 
 import org.apache.accumulo.core.client.IteratorSetting;
-import org.apache.accumulo.core.clientImpl.Table;
 import org.apache.accumulo.core.data.Key;
+import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.dataImpl.KeyExtent;
 import org.apache.accumulo.core.iterators.IteratorEnvironment;
@@ -58,8 +58,8 @@ public class TabletStateChangeIterator extends SkippingIterator {
   private static final Logger log = LoggerFactory.getLogger(TabletStateChangeIterator.class);
 
   private Set<TServerInstance> current;
-  private Set<Table.ID> onlineTables;
-  private Map<Table.ID,MergeInfo> merges;
+  private Set<TableId> onlineTables;
+  private Map<TableId,MergeInfo> merges;
   private boolean debug = false;
   private Set<KeyExtent> migrations;
   private MasterState masterState = MasterState.NORMAL;
@@ -105,12 +105,12 @@ public class TabletStateChangeIterator extends SkippingIterator {
     }
   }
 
-  private Set<Table.ID> parseTableIDs(String tableIDs) {
+  private Set<TableId> parseTableIDs(String tableIDs) {
     if (tableIDs == null)
       return null;
-    Set<Table.ID> result = new HashSet<>();
+    Set<TableId> result = new HashSet<>();
     for (String tableID : tableIDs.split(","))
-      result.add(Table.ID.of(tableID));
+      result.add(TableId.of(tableID));
     return result;
   }
 
@@ -132,11 +132,11 @@ public class TabletStateChangeIterator extends SkippingIterator {
     return result;
   }
 
-  private Map<Table.ID,MergeInfo> parseMerges(String merges) {
+  private Map<TableId,MergeInfo> parseMerges(String merges) {
     if (merges == null)
       return null;
     try {
-      Map<Table.ID,MergeInfo> result = new HashMap<>();
+      Map<TableId,MergeInfo> result = new HashMap<>();
       DataInputBuffer buffer = new DataInputBuffer();
       byte[] data = Base64.getDecoder().decode(merges);
       buffer.reset(data, data.length);
@@ -225,7 +225,7 @@ public class TabletStateChangeIterator extends SkippingIterator {
     }
   }
 
-  public static void setOnlineTables(IteratorSetting cfg, Set<Table.ID> onlineTables) {
+  public static void setOnlineTables(IteratorSetting cfg, Set<TableId> onlineTables) {
     if (onlineTables != null)
       cfg.addOption(TABLES_OPTION, Joiner.on(",").join(onlineTables));
   }
diff --git a/server/base/src/main/java/org/apache/accumulo/server/problems/ProblemReport.java b/server/base/src/main/java/org/apache/accumulo/server/problems/ProblemReport.java
index a9eaa4d..2110551 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/problems/ProblemReport.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/problems/ProblemReport.java
@@ -28,9 +28,9 @@ import java.net.UnknownHostException;
 import java.util.Map.Entry;
 
 import org.apache.accumulo.core.Constants;
-import org.apache.accumulo.core.clientImpl.Table;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Mutation;
+import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.util.Encoding;
 import org.apache.accumulo.fate.zookeeper.ZooReaderWriter;
@@ -42,14 +42,14 @@ import org.apache.hadoop.io.Text;
 import org.apache.zookeeper.KeeperException;
 
 public class ProblemReport {
-  private Table.ID tableId;
+  private TableId tableId;
   private ProblemType problemType;
   private String resource;
   private String exception;
   private String server;
   private long creationTime;
 
-  public ProblemReport(Table.ID tableId, ProblemType problemType, String resource, String server,
+  public ProblemReport(TableId tableId, ProblemType problemType, String resource, String server,
       Throwable e, long creationTime) {
     requireNonNull(tableId, "tableId is null");
     requireNonNull(problemType, "problemType is null");
@@ -75,16 +75,16 @@ public class ProblemReport {
     this.creationTime = creationTime;
   }
 
-  public ProblemReport(Table.ID tableId, ProblemType problemType, String resource, String server,
+  public ProblemReport(TableId tableId, ProblemType problemType, String resource, String server,
       Throwable e) {
     this(tableId, problemType, resource, server, e, System.currentTimeMillis());
   }
 
-  public ProblemReport(Table.ID tableId, ProblemType problemType, String resource, Throwable e) {
+  public ProblemReport(TableId tableId, ProblemType problemType, String resource, Throwable e) {
     this(tableId, problemType, resource, null, e);
   }
 
-  private ProblemReport(Table.ID table, ProblemType problemType, String resource, byte enc[])
+  private ProblemReport(TableId table, ProblemType problemType, String resource, byte enc[])
       throws IOException {
     requireNonNull(table, "table is null");
     requireNonNull(problemType, "problemType is null");
@@ -168,7 +168,7 @@ public class ProblemReport {
   private String getZPath(String zkRoot) throws IOException {
     ByteArrayOutputStream baos = new ByteArrayOutputStream();
     DataOutputStream dos = new DataOutputStream(baos);
-    dos.writeUTF(getTableId().canonicalID());
+    dos.writeUTF(getTableId().canonical());
     dos.writeUTF(getProblemType().name());
     dos.writeUTF(getResource());
     dos.close();
@@ -185,7 +185,7 @@ public class ProblemReport {
     ByteArrayInputStream bais = new ByteArrayInputStream(bytes);
     DataInputStream dis = new DataInputStream(bais);
 
-    Table.ID tableId = Table.ID.of(dis.readUTF());
+    TableId tableId = TableId.of(dis.readUTF());
     String problemType = dis.readUTF();
     String resource = dis.readUTF();
 
@@ -197,7 +197,7 @@ public class ProblemReport {
   }
 
   public static ProblemReport decodeMetadataEntry(Entry<Key,Value> entry) throws IOException {
-    Table.ID tableId = Table.ID.of(entry.getKey().getRow().toString().substring("~err_".length()));
+    TableId tableId = TableId.of(entry.getKey().getRow().toString().substring("~err_".length()));
     String problemType = entry.getKey().getColumnFamily().toString();
     String resource = entry.getKey().getColumnQualifier().toString();
 
@@ -205,7 +205,7 @@ public class ProblemReport {
         entry.getValue().get());
   }
 
-  public Table.ID getTableId() {
+  public TableId getTableId() {
     return tableId;
   }
 
diff --git a/server/base/src/main/java/org/apache/accumulo/server/problems/ProblemReportingIterator.java b/server/base/src/main/java/org/apache/accumulo/server/problems/ProblemReportingIterator.java
index 6e7e72f..6dba085 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/problems/ProblemReportingIterator.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/problems/ProblemReportingIterator.java
@@ -21,10 +21,10 @@ import java.util.Collection;
 import java.util.Map;
 import java.util.concurrent.atomic.AtomicBoolean;
 
-import org.apache.accumulo.core.clientImpl.Table;
 import org.apache.accumulo.core.data.ByteSequence;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Range;
+import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.iterators.IteratorEnvironment;
 import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
@@ -36,10 +36,10 @@ public class ProblemReportingIterator implements InterruptibleIterator {
   private boolean sawError = false;
   private final boolean continueOnError;
   private String resource;
-  private Table.ID tableId;
+  private TableId tableId;
   private final ServerContext context;
 
-  public ProblemReportingIterator(ServerContext context, Table.ID tableId, String resource,
+  public ProblemReportingIterator(ServerContext context, TableId tableId, String resource,
       boolean continueOnError, SortedKeyValueIterator<Key,Value> source) {
     this.context = context;
     this.tableId = tableId;
diff --git a/server/base/src/main/java/org/apache/accumulo/server/problems/ProblemReports.java b/server/base/src/main/java/org/apache/accumulo/server/problems/ProblemReports.java
index d9f683f..98114c9 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/problems/ProblemReports.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/problems/ProblemReports.java
@@ -33,11 +33,11 @@ import java.util.concurrent.TimeUnit;
 import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.IteratorSetting;
 import org.apache.accumulo.core.client.Scanner;
-import org.apache.accumulo.core.clientImpl.Table;
 import org.apache.accumulo.core.conf.SiteConfiguration;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.data.Range;
+import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.iterators.SortedKeyIterator;
 import org.apache.accumulo.core.metadata.MetadataTable;
@@ -125,7 +125,7 @@ public class ProblemReports implements Iterable<ProblemReport> {
     }
   }
 
-  public void deleteProblemReport(Table.ID table, ProblemType pType, String resource) {
+  public void deleteProblemReport(TableId table, ProblemType pType, String resource) {
     final ProblemReport pr = new ProblemReport(table, pType, resource, null);
 
     Runnable r = new Runnable() {
@@ -157,7 +157,7 @@ public class ProblemReports implements Iterable<ProblemReport> {
 
   private static ProblemReports instance;
 
-  public void deleteProblemReports(Table.ID table) throws Exception {
+  public void deleteProblemReports(TableId table) throws Exception {
 
     if (isMeta(table)) {
       Iterator<ProblemReport> pri = iterator(table);
@@ -184,11 +184,11 @@ public class ProblemReports implements Iterable<ProblemReport> {
       MetadataTableUtil.getMetadataTable(context).update(delMut);
   }
 
-  private static boolean isMeta(Table.ID tableId) {
+  private static boolean isMeta(TableId tableId) {
     return tableId.equals(MetadataTable.ID) || tableId.equals(RootTable.ID);
   }
 
-  public Iterator<ProblemReport> iterator(final Table.ID table) {
+  public Iterator<ProblemReport> iterator(final TableId table) {
     try {
 
       return new Iterator<ProblemReport>() {
@@ -300,9 +300,9 @@ public class ProblemReports implements Iterable<ProblemReport> {
     getInstance(context).printProblems();
   }
 
-  public Map<Table.ID,Map<ProblemType,Integer>> summarize() {
+  public Map<TableId,Map<ProblemType,Integer>> summarize() {
 
-    TreeMap<Table.ID,Map<ProblemType,Integer>> summary = new TreeMap<>();
+    TreeMap<TableId,Map<ProblemType,Integer>> summary = new TreeMap<>();
 
     for (ProblemReport pr : this) {
       Map<ProblemType,Integer> tableProblems = summary.get(pr.getTableId());
diff --git a/server/base/src/main/java/org/apache/accumulo/server/replication/DistributedWorkQueueWorkAssignerHelper.java b/server/base/src/main/java/org/apache/accumulo/server/replication/DistributedWorkQueueWorkAssignerHelper.java
index 4877ff1..47293e0 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/replication/DistributedWorkQueueWorkAssignerHelper.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/replication/DistributedWorkQueueWorkAssignerHelper.java
@@ -20,7 +20,7 @@ import static java.util.Objects.requireNonNull;
 
 import java.util.Map.Entry;
 
-import org.apache.accumulo.core.clientImpl.Table;
+import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.replication.ReplicationTarget;
 import org.apache.accumulo.server.zookeeper.DistributedWorkQueue;
 
@@ -76,6 +76,6 @@ public class DistributedWorkQueueWorkAssignerHelper {
     return Maps.immutableEntry(filename,
         new ReplicationTarget(queueKey.substring(index + 1, secondIndex),
             queueKey.substring(secondIndex + 1, thirdIndex),
-            Table.ID.of(queueKey.substring(thirdIndex + 1))));
+            TableId.of(queueKey.substring(thirdIndex + 1))));
   }
 }
diff --git a/server/base/src/main/java/org/apache/accumulo/server/replication/ReplicationUtil.java b/server/base/src/main/java/org/apache/accumulo/server/replication/ReplicationUtil.java
index 0c4fb44..0791082 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/replication/ReplicationUtil.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/replication/ReplicationUtil.java
@@ -25,11 +25,11 @@ import java.util.Set;
 
 import org.apache.accumulo.core.client.BatchScanner;
 import org.apache.accumulo.core.client.TableNotFoundException;
-import org.apache.accumulo.core.clientImpl.Table;
 import org.apache.accumulo.core.clientImpl.Tables;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Range;
+import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.master.thrift.MasterMonitorInfo;
 import org.apache.accumulo.core.metadata.MetadataTable;
@@ -109,14 +109,14 @@ public class ReplicationUtil {
   public Set<ReplicationTarget> getReplicationTargets() {
     // The total set of configured targets
     final Set<ReplicationTarget> allConfiguredTargets = new HashSet<>();
-    final Map<String,Table.ID> tableNameToId = Tables.getNameToIdMap(context);
+    final Map<String,TableId> tableNameToId = Tables.getNameToIdMap(context);
 
     for (String table : tableNameToId.keySet()) {
       if (MetadataTable.NAME.equals(table) || RootTable.NAME.equals(table)) {
         continue;
       }
 
-      Table.ID localId = tableNameToId.get(table);
+      TableId localId = tableNameToId.get(table);
       if (localId == null) {
         log.trace("Could not determine ID for {}", table);
         continue;
diff --git a/server/base/src/main/java/org/apache/accumulo/server/security/AuditedSecurityOperation.java b/server/base/src/main/java/org/apache/accumulo/server/security/AuditedSecurityOperation.java
index 5298c72..d1b6eaa 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/security/AuditedSecurityOperation.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/security/AuditedSecurityOperation.java
@@ -26,13 +26,13 @@ import java.util.Map.Entry;
 
 import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.clientImpl.Credentials;
-import org.apache.accumulo.core.clientImpl.Namespace;
-import org.apache.accumulo.core.clientImpl.Table;
 import org.apache.accumulo.core.clientImpl.Tables;
 import org.apache.accumulo.core.clientImpl.Translator;
 import org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException;
 import org.apache.accumulo.core.data.Column;
+import org.apache.accumulo.core.data.NamespaceId;
 import org.apache.accumulo.core.data.Range;
+import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.dataImpl.KeyExtent;
 import org.apache.accumulo.core.dataImpl.thrift.IterInfo;
 import org.apache.accumulo.core.dataImpl.thrift.TColumn;
@@ -77,7 +77,7 @@ public class AuditedSecurityOperation extends SecurityOperation {
     return instance;
   }
 
-  private String getTableName(Table.ID tableId) {
+  private String getTableName(TableId tableId) {
     try {
       return Tables.getTableName(context, tableId);
     } catch (TableNotFoundException e) {
@@ -93,7 +93,7 @@ public class AuditedSecurityOperation extends SecurityOperation {
     return auths;
   }
 
-  private boolean shouldAudit(TCredentials credentials, Table.ID tableId) {
+  private boolean shouldAudit(TCredentials credentials, TableId tableId) {
     return (audit.isInfoEnabled() || audit.isWarnEnabled()) && !tableId.equals(MetadataTable.ID)
         && shouldAudit(credentials);
   }
@@ -152,7 +152,7 @@ public class AuditedSecurityOperation extends SecurityOperation {
   }
 
   @Override
-  public boolean canScan(TCredentials credentials, Table.ID tableId, Namespace.ID namespaceId,
+  public boolean canScan(TCredentials credentials, TableId tableId, NamespaceId namespaceId,
       TRange range, List<TColumn> columns, List<IterInfo> ssiList,
       Map<String,Map<String,String>> ssio, List<ByteBuffer> authorizations)
       throws ThriftSecurityException {
@@ -183,7 +183,7 @@ public class AuditedSecurityOperation extends SecurityOperation {
       + " iterators: %s; iteratorOptions: %s;";
 
   @Override
-  public boolean canScan(TCredentials credentials, Table.ID tableId, Namespace.ID namespaceId,
+  public boolean canScan(TCredentials credentials, TableId tableId, NamespaceId namespaceId,
       Map<TKeyExtent,List<TRange>> tbatch, List<TColumn> tcolumns, List<IterInfo> ssiList,
       Map<String,Map<String,String>> ssio, List<ByteBuffer> authorizations)
       throws ThriftSecurityException {
@@ -265,7 +265,7 @@ public class AuditedSecurityOperation extends SecurityOperation {
       + " createTable; targetTable: %s;";
 
   @Override
-  public boolean canCreateTable(TCredentials c, String tableName, Namespace.ID namespaceId)
+  public boolean canCreateTable(TCredentials c, String tableName, NamespaceId namespaceId)
       throws ThriftSecurityException {
     try {
       boolean result = super.canCreateTable(c, tableName, namespaceId);
@@ -281,7 +281,7 @@ public class AuditedSecurityOperation extends SecurityOperation {
       + " deleteTable; targetTable: %s;";
 
   @Override
-  public boolean canDeleteTable(TCredentials c, Table.ID tableId, Namespace.ID namespaceId)
+  public boolean canDeleteTable(TCredentials c, TableId tableId, NamespaceId namespaceId)
       throws ThriftSecurityException {
     String tableName = getTableName(tableId);
     try {
@@ -298,8 +298,8 @@ public class AuditedSecurityOperation extends SecurityOperation {
       + " renameTable; targetTable: %s; newTableName: %s;";
 
   @Override
-  public boolean canRenameTable(TCredentials c, Table.ID tableId, String oldTableName,
-      String newTableName, Namespace.ID namespaceId) throws ThriftSecurityException {
+  public boolean canRenameTable(TCredentials c, TableId tableId, String oldTableName,
+      String newTableName, NamespaceId namespaceId) throws ThriftSecurityException {
     try {
       boolean result = super.canRenameTable(c, tableId, oldTableName, newTableName, namespaceId);
       audit(c, result, CAN_RENAME_TABLE_AUDIT_TEMPLATE, oldTableName, newTableName);
@@ -314,7 +314,7 @@ public class AuditedSecurityOperation extends SecurityOperation {
       + " splitTable; targetTable: %s; targetNamespace: %s;";
 
   @Override
-  public boolean canSplitTablet(TCredentials credentials, Table.ID table, Namespace.ID namespaceId)
+  public boolean canSplitTablet(TCredentials credentials, TableId table, NamespaceId namespaceId)
       throws ThriftSecurityException {
     try {
       boolean result = super.canSplitTablet(credentials, table, namespaceId);
@@ -346,7 +346,7 @@ public class AuditedSecurityOperation extends SecurityOperation {
       + " flushTable; targetTable: %s; targetNamespace: %s;";
 
   @Override
-  public boolean canFlush(TCredentials c, Table.ID tableId, Namespace.ID namespaceId)
+  public boolean canFlush(TCredentials c, TableId tableId, NamespaceId namespaceId)
       throws ThriftSecurityException {
     try {
       boolean result = super.canFlush(c, tableId, namespaceId);
@@ -362,7 +362,7 @@ public class AuditedSecurityOperation extends SecurityOperation {
       + " alterTable; targetTable: %s; targetNamespace: %s;";
 
   @Override
-  public boolean canAlterTable(TCredentials c, Table.ID tableId, Namespace.ID namespaceId)
+  public boolean canAlterTable(TCredentials c, TableId tableId, NamespaceId namespaceId)
       throws ThriftSecurityException {
     try {
       boolean result = super.canAlterTable(c, tableId, namespaceId);
@@ -378,8 +378,8 @@ public class AuditedSecurityOperation extends SecurityOperation {
       + " cloneTable; targetTable: %s; newTableName: %s";
 
   @Override
-  public boolean canCloneTable(TCredentials c, Table.ID tableId, String tableName,
-      Namespace.ID destinationNamespaceId, Namespace.ID sourceNamespaceId)
+  public boolean canCloneTable(TCredentials c, TableId tableId, String tableName,
+      NamespaceId destinationNamespaceId, NamespaceId sourceNamespaceId)
       throws ThriftSecurityException {
     String oldTableName = getTableName(tableId);
     try {
@@ -397,8 +397,8 @@ public class AuditedSecurityOperation extends SecurityOperation {
       + " deleteData; targetTable: %s; startRange: %s; endRange: %s;";
 
   @Override
-  public boolean canDeleteRange(TCredentials c, Table.ID tableId, String tableName, Text startRow,
-      Text endRow, Namespace.ID namespaceId) throws ThriftSecurityException {
+  public boolean canDeleteRange(TCredentials c, TableId tableId, String tableName, Text startRow,
+      Text endRow, NamespaceId namespaceId) throws ThriftSecurityException {
     try {
       boolean result = super.canDeleteRange(c, tableId, tableName, startRow, endRow, namespaceId);
       audit(c, result, CAN_DELETE_RANGE_AUDIT_TEMPLATE, tableName, startRow.toString(),
@@ -415,8 +415,8 @@ public class AuditedSecurityOperation extends SecurityOperation {
       + " bulkImport; targetTable: %s; dataDir: %s; failDir: %s;";
 
   @Override
-  public boolean canBulkImport(TCredentials c, Table.ID tableId, String tableName, String dir,
-      String failDir, Namespace.ID namespaceId) throws ThriftSecurityException {
+  public boolean canBulkImport(TCredentials c, TableId tableId, String tableName, String dir,
+      String failDir, NamespaceId namespaceId) throws ThriftSecurityException {
     try {
       boolean result = super.canBulkImport(c, tableId, tableName, dir, failDir, namespaceId);
       audit(c, result, CAN_BULK_IMPORT_AUDIT_TEMPLATE, tableName, dir, failDir);
@@ -431,7 +431,7 @@ public class AuditedSecurityOperation extends SecurityOperation {
       + " compactTable; targetTable: %s; targetNamespace: %s;";
 
   @Override
-  public boolean canCompact(TCredentials c, Table.ID tableId, Namespace.ID namespaceId)
+  public boolean canCompact(TCredentials c, TableId tableId, NamespaceId namespaceId)
       throws ThriftSecurityException {
     try {
       boolean result = super.canCompact(c, tableId, namespaceId);
@@ -525,8 +525,8 @@ public class AuditedSecurityOperation extends SecurityOperation {
       + " grantTable; targetUser: %s; targetTable: %s; targetNamespace: %s;";
 
   @Override
-  public boolean canGrantTable(TCredentials c, String user, Table.ID table,
-      Namespace.ID namespaceId) throws ThriftSecurityException {
+  public boolean canGrantTable(TCredentials c, String user, TableId table, NamespaceId namespaceId)
+      throws ThriftSecurityException {
     try {
       boolean result = super.canGrantTable(c, user, table, namespaceId);
       audit(c, result, CAN_GRANT_TABLE_AUDIT_TEMPLATE, user, table, namespaceId);
@@ -557,8 +557,8 @@ public class AuditedSecurityOperation extends SecurityOperation {
       + " revokeTable; targetUser: %s; targetTable %s; targetNamespace: %s;";
 
   @Override
-  public boolean canRevokeTable(TCredentials c, String user, Table.ID table,
-      Namespace.ID namespaceId) throws ThriftSecurityException {
+  public boolean canRevokeTable(TCredentials c, String user, TableId table, NamespaceId namespaceId)
+      throws ThriftSecurityException {
     try {
       boolean result = super.canRevokeTable(c, user, table, namespaceId);
       audit(c, result, CAN_REVOKE_TABLE_AUDIT_TEMPLATE, user, table, namespaceId);
@@ -574,7 +574,7 @@ public class AuditedSecurityOperation extends SecurityOperation {
 
   @Override
   public boolean canImport(TCredentials credentials, String tableName, String importDir,
-      Namespace.ID namespaceId) throws ThriftSecurityException {
+      NamespaceId namespaceId) throws ThriftSecurityException {
 
     try {
       boolean result = super.canImport(credentials, tableName, importDir, namespaceId);
@@ -590,8 +590,8 @@ public class AuditedSecurityOperation extends SecurityOperation {
       + " targetTable: %s; dataDir: %s;";
 
   @Override
-  public boolean canExport(TCredentials credentials, Table.ID tableId, String tableName,
-      String exportDir, Namespace.ID namespaceId) throws ThriftSecurityException {
+  public boolean canExport(TCredentials credentials, TableId tableId, String tableName,
+      String exportDir, NamespaceId namespaceId) throws ThriftSecurityException {
 
     try {
       boolean result = super.canExport(credentials, tableId, tableName, exportDir, namespaceId);
@@ -635,8 +635,8 @@ public class AuditedSecurityOperation extends SecurityOperation {
       + " grantTablePermission; permission: %s; targetTable: %s; targetUser: %s;";
 
   @Override
-  public void grantTablePermission(TCredentials credentials, String user, Table.ID tableId,
-      TablePermission permission, Namespace.ID namespaceId) throws ThriftSecurityException {
+  public void grantTablePermission(TCredentials credentials, String user, TableId tableId,
+      TablePermission permission, NamespaceId namespaceId) throws ThriftSecurityException {
     String tableName = getTableName(tableId);
     try {
       super.grantTablePermission(credentials, user, tableId, permission, namespaceId);
@@ -667,8 +667,8 @@ public class AuditedSecurityOperation extends SecurityOperation {
       + " revokeTablePermission; permission: %s; targetTable: %s; targetUser: %s;";
 
   @Override
-  public void revokeTablePermission(TCredentials credentials, String user, Table.ID tableId,
-      TablePermission permission, Namespace.ID namespaceId) throws ThriftSecurityException {
+  public void revokeTablePermission(TCredentials credentials, String user, TableId tableId,
+      TablePermission permission, NamespaceId namespaceId) throws ThriftSecurityException {
     String tableName = getTableName(tableId);
     try {
       super.revokeTablePermission(credentials, user, tableId, permission, namespaceId);
@@ -699,8 +699,8 @@ public class AuditedSecurityOperation extends SecurityOperation {
       + " %s; targetTable: %s;";
 
   @Override
-  public boolean canOnlineOfflineTable(TCredentials credentials, Table.ID tableId, FateOperation op,
-      Namespace.ID namespaceId) throws ThriftSecurityException {
+  public boolean canOnlineOfflineTable(TCredentials credentials, TableId tableId, FateOperation op,
+      NamespaceId namespaceId) throws ThriftSecurityException {
     String tableName = getTableName(tableId);
     String operation = null;
     if (op == FateOperation.TABLE_ONLINE)
@@ -723,7 +723,7 @@ public class AuditedSecurityOperation extends SecurityOperation {
       + " mergeTable; targetTable: %s; targetNamespace: %s;";
 
   @Override
-  public boolean canMerge(TCredentials c, Table.ID tableId, Namespace.ID namespaceId)
+  public boolean canMerge(TCredentials c, TableId tableId, NamespaceId namespaceId)
       throws ThriftSecurityException {
     try {
       boolean result = super.canMerge(c, tableId, namespaceId);
diff --git a/server/base/src/main/java/org/apache/accumulo/server/security/SecurityOperation.java b/server/base/src/main/java/org/apache/accumulo/server/security/SecurityOperation.java
index 86f60c9..6564623 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/security/SecurityOperation.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/security/SecurityOperation.java
@@ -30,10 +30,11 @@ import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
 import org.apache.accumulo.core.clientImpl.Credentials;
 import org.apache.accumulo.core.clientImpl.Namespace;
-import org.apache.accumulo.core.clientImpl.Table;
 import org.apache.accumulo.core.clientImpl.thrift.SecurityErrorCode;
 import org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException;
 import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.core.data.NamespaceId;
+import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.dataImpl.thrift.IterInfo;
 import org.apache.accumulo.core.dataImpl.thrift.TColumn;
 import org.apache.accumulo.core.dataImpl.thrift.TKeyExtent;
@@ -143,7 +144,7 @@ public class SecurityOperation {
     authorizor.initializeSecurity(credentials, rootPrincipal);
     permHandle.initializeSecurity(credentials, rootPrincipal);
     try {
-      permHandle.grantTablePermission(rootPrincipal, MetadataTable.ID.canonicalID(),
+      permHandle.grantTablePermission(rootPrincipal, MetadataTable.ID.canonical(),
           TablePermission.ALTER_TABLE);
     } catch (TableNotFoundException e) {
       // Shouldn't happen
@@ -309,7 +310,7 @@ public class SecurityOperation {
    * @return true if a user exists and has permission; false otherwise
    */
   private boolean hasSystemPermissionWithNamespaceId(TCredentials credentials,
-      SystemPermission permission, Namespace.ID namespaceId, boolean useCached)
+      SystemPermission permission, NamespaceId namespaceId, boolean useCached)
       throws ThriftSecurityException {
     if (isSystemUser(credentials))
       return true;
@@ -347,8 +348,8 @@ public class SecurityOperation {
    *
    * @return true if a user exists and has permission; false otherwise
    */
-  protected boolean hasTablePermission(TCredentials credentials, Table.ID tableId,
-      Namespace.ID namespaceId, TablePermission permission, boolean useCached)
+  protected boolean hasTablePermission(TCredentials credentials, TableId tableId,
+      NamespaceId namespaceId, TablePermission permission, boolean useCached)
       throws ThriftSecurityException {
     if (isSystemUser(credentials))
       return true;
@@ -363,7 +364,7 @@ public class SecurityOperation {
    *
    * @return true if a user exists and has permission; false otherwise
    */
-  protected boolean _hasTablePermission(String user, Table.ID table, TablePermission permission,
+  protected boolean _hasTablePermission(String user, TableId table, TablePermission permission,
       boolean useCached) throws ThriftSecurityException {
     targetUserExists(user);
 
@@ -373,8 +374,8 @@ public class SecurityOperation {
 
     try {
       if (useCached)
-        return permHandle.hasCachedTablePermission(user, table.canonicalID(), permission);
-      return permHandle.hasTablePermission(user, table.canonicalID(), permission);
+        return permHandle.hasCachedTablePermission(user, table.canonical(), permission);
+      return permHandle.hasTablePermission(user, table.canonical(), permission);
     } catch (TableNotFoundException e) {
       throw new ThriftSecurityException(user, SecurityErrorCode.TABLE_DOESNT_EXIST);
     }
@@ -386,20 +387,20 @@ public class SecurityOperation {
    *
    * @return true if a user exists and has permission; false otherwise
    */
-  protected boolean _hasNamespacePermission(String user, Namespace.ID namespace,
+  protected boolean _hasNamespacePermission(String user, NamespaceId namespace,
       NamespacePermission permission, boolean useCached) throws ThriftSecurityException {
     if (permission == null)
       return false;
 
     targetUserExists(user);
 
-    if (namespace.equals(Namespace.ID.ACCUMULO) && permission.equals(NamespacePermission.READ))
+    if (namespace.equals(Namespace.ACCUMULO.id()) && permission.equals(NamespacePermission.READ))
       return true;
 
     try {
       if (useCached)
-        return permHandle.hasCachedNamespacePermission(user, namespace.canonicalID(), permission);
-      return permHandle.hasNamespacePermission(user, namespace.canonicalID(), permission);
+        return permHandle.hasCachedNamespacePermission(user, namespace.canonical(), permission);
+      return permHandle.hasNamespacePermission(user, namespace.canonical(), permission);
     } catch (NamespaceNotFoundException e) {
       throw new ThriftSecurityException(user, SecurityErrorCode.NAMESPACE_DOESNT_EXIST);
     }
@@ -423,34 +424,34 @@ public class SecurityOperation {
       throw new ThriftSecurityException(user, SecurityErrorCode.USER_DOESNT_EXIST);
   }
 
-  public boolean canScan(TCredentials credentials, Table.ID tableId, Namespace.ID namespaceId)
+  public boolean canScan(TCredentials credentials, TableId tableId, NamespaceId namespaceId)
       throws ThriftSecurityException {
     authenticate(credentials);
     return hasTablePermission(credentials, tableId, namespaceId, TablePermission.READ, true);
   }
 
-  public boolean canScan(TCredentials credentials, Table.ID tableId, Namespace.ID namespaceId,
+  public boolean canScan(TCredentials credentials, TableId tableId, NamespaceId namespaceId,
       TRange range, List<TColumn> columns, List<IterInfo> ssiList,
       Map<String,Map<String,String>> ssio, List<ByteBuffer> authorizations)
       throws ThriftSecurityException {
     return canScan(credentials, tableId, namespaceId);
   }
 
-  public boolean canScan(TCredentials credentials, Table.ID table, Namespace.ID namespaceId,
+  public boolean canScan(TCredentials credentials, TableId table, NamespaceId namespaceId,
       Map<TKeyExtent,List<TRange>> tbatch, List<TColumn> tcolumns, List<IterInfo> ssiList,
       Map<String,Map<String,String>> ssio, List<ByteBuffer> authorizations)
       throws ThriftSecurityException {
     return canScan(credentials, table, namespaceId);
   }
 
-  public boolean canWrite(TCredentials credentials, Table.ID tableId, Namespace.ID namespaceId)
+  public boolean canWrite(TCredentials credentials, TableId tableId, NamespaceId namespaceId)
       throws ThriftSecurityException {
     authenticate(credentials);
     return hasTablePermission(credentials, tableId, namespaceId, TablePermission.WRITE, true);
   }
 
-  public boolean canConditionallyUpdate(TCredentials credentials, Table.ID tableID,
-      Namespace.ID namespaceId) throws ThriftSecurityException {
+  public boolean canConditionallyUpdate(TCredentials credentials, TableId tableID,
+      NamespaceId namespaceId) throws ThriftSecurityException {
 
     authenticate(credentials);
 
@@ -458,8 +459,8 @@ public class SecurityOperation {
         && hasTablePermission(credentials, tableID, namespaceId, TablePermission.READ, true);
   }
 
-  public boolean canSplitTablet(TCredentials credentials, Table.ID tableId,
-      Namespace.ID namespaceId) throws ThriftSecurityException {
+  public boolean canSplitTablet(TCredentials credentials, TableId tableId, NamespaceId namespaceId)
+      throws ThriftSecurityException {
     authenticate(credentials);
     return hasSystemPermissionWithNamespaceId(credentials, SystemPermission.ALTER_TABLE,
         namespaceId, false)
@@ -478,35 +479,35 @@ public class SecurityOperation {
     return hasSystemPermission(credentials, SystemPermission.SYSTEM, false);
   }
 
-  public boolean canFlush(TCredentials c, Table.ID tableId, Namespace.ID namespaceId)
+  public boolean canFlush(TCredentials c, TableId tableId, NamespaceId namespaceId)
       throws ThriftSecurityException {
     authenticate(c);
     return hasTablePermission(c, tableId, namespaceId, TablePermission.WRITE, false)
         || hasTablePermission(c, tableId, namespaceId, TablePermission.ALTER_TABLE, false);
   }
 
-  public boolean canAlterTable(TCredentials c, Table.ID tableId, Namespace.ID namespaceId)
+  public boolean canAlterTable(TCredentials c, TableId tableId, NamespaceId namespaceId)
       throws ThriftSecurityException {
     authenticate(c);
     return hasTablePermission(c, tableId, namespaceId, TablePermission.ALTER_TABLE, false)
         || hasSystemPermissionWithNamespaceId(c, SystemPermission.ALTER_TABLE, namespaceId, false);
   }
 
-  public boolean canCreateTable(TCredentials c, String tableName, Namespace.ID namespaceId)
+  public boolean canCreateTable(TCredentials c, String tableName, NamespaceId namespaceId)
       throws ThriftSecurityException {
     authenticate(c);
     return hasSystemPermissionWithNamespaceId(c, SystemPermission.CREATE_TABLE, namespaceId, false);
   }
 
-  public boolean canRenameTable(TCredentials c, Table.ID tableId, String oldTableName,
-      String newTableName, Namespace.ID namespaceId) throws ThriftSecurityException {
+  public boolean canRenameTable(TCredentials c, TableId tableId, String oldTableName,
+      String newTableName, NamespaceId namespaceId) throws ThriftSecurityException {
     authenticate(c);
     return hasSystemPermissionWithNamespaceId(c, SystemPermission.ALTER_TABLE, namespaceId, false)
         || hasTablePermission(c, tableId, namespaceId, TablePermission.ALTER_TABLE, false);
   }
 
-  public boolean canCloneTable(TCredentials c, Table.ID tableId, String tableName,
-      Namespace.ID destinationNamespaceId, Namespace.ID srcNamespaceId)
+  public boolean canCloneTable(TCredentials c, TableId tableId, String tableName,
+      NamespaceId destinationNamespaceId, NamespaceId srcNamespaceId)
       throws ThriftSecurityException {
     authenticate(c);
     return hasSystemPermissionWithNamespaceId(c, SystemPermission.CREATE_TABLE,
@@ -514,22 +515,22 @@ public class SecurityOperation {
         && hasTablePermission(c, tableId, srcNamespaceId, TablePermission.READ, false);
   }
 
-  public boolean canDeleteTable(TCredentials c, Table.ID tableId, Namespace.ID namespaceId)
+  public boolean canDeleteTable(TCredentials c, TableId tableId, NamespaceId namespaceId)
       throws ThriftSecurityException {
     authenticate(c);
     return hasSystemPermissionWithNamespaceId(c, SystemPermission.DROP_TABLE, namespaceId, false)
         || hasTablePermission(c, tableId, namespaceId, TablePermission.DROP_TABLE, false);
   }
 
-  public boolean canOnlineOfflineTable(TCredentials c, Table.ID tableId, FateOperation op,
-      Namespace.ID namespaceId) throws ThriftSecurityException {
+  public boolean canOnlineOfflineTable(TCredentials c, TableId tableId, FateOperation op,
+      NamespaceId namespaceId) throws ThriftSecurityException {
     authenticate(c);
     return hasSystemPermissionWithNamespaceId(c, SystemPermission.SYSTEM, namespaceId, false)
         || hasSystemPermissionWithNamespaceId(c, SystemPermission.ALTER_TABLE, namespaceId, false)
         || hasTablePermission(c, tableId, namespaceId, TablePermission.ALTER_TABLE, false);
   }
 
-  public boolean canMerge(TCredentials c, Table.ID tableId, Namespace.ID namespaceId)
+  public boolean canMerge(TCredentials c, TableId tableId, NamespaceId namespaceId)
       throws ThriftSecurityException {
     authenticate(c);
     return hasSystemPermissionWithNamespaceId(c, SystemPermission.SYSTEM, namespaceId, false)
@@ -537,20 +538,20 @@ public class SecurityOperation {
         || hasTablePermission(c, tableId, namespaceId, TablePermission.ALTER_TABLE, false);
   }
 
-  public boolean canDeleteRange(TCredentials c, Table.ID tableId, String tableName, Text startRow,
-      Text endRow, Namespace.ID namespaceId) throws ThriftSecurityException {
+  public boolean canDeleteRange(TCredentials c, TableId tableId, String tableName, Text startRow,
+      Text endRow, NamespaceId namespaceId) throws ThriftSecurityException {
     authenticate(c);
     return hasSystemPermissionWithNamespaceId(c, SystemPermission.SYSTEM, namespaceId, false)
         || hasTablePermission(c, tableId, namespaceId, TablePermission.WRITE, false);
   }
 
-  public boolean canBulkImport(TCredentials c, Table.ID tableId, String tableName, String dir,
-      String failDir, Namespace.ID namespaceId) throws ThriftSecurityException {
+  public boolean canBulkImport(TCredentials c, TableId tableId, String tableName, String dir,
+      String failDir, NamespaceId namespaceId) throws ThriftSecurityException {
     authenticate(c);
     return hasTablePermission(c, tableId, namespaceId, TablePermission.BULK_IMPORT, false);
   }
 
-  public boolean canCompact(TCredentials c, Table.ID tableId, Namespace.ID namespaceId)
+  public boolean canCompact(TCredentials c, TableId tableId, NamespaceId namespaceId)
       throws ThriftSecurityException {
     authenticate(c);
     return hasSystemPermissionWithNamespaceId(c, SystemPermission.ALTER_TABLE, namespaceId, false)
@@ -588,19 +589,19 @@ public class SecurityOperation {
     return hasSystemPermission(c, SystemPermission.GRANT, false);
   }
 
-  public boolean canGrantTable(TCredentials c, String user, Table.ID tableId,
-      Namespace.ID namespaceId) throws ThriftSecurityException {
+  public boolean canGrantTable(TCredentials c, String user, TableId tableId,
+      NamespaceId namespaceId) throws ThriftSecurityException {
     authenticate(c);
     return hasSystemPermissionWithNamespaceId(c, SystemPermission.ALTER_TABLE, namespaceId, false)
         || hasTablePermission(c, tableId, namespaceId, TablePermission.GRANT, false);
   }
 
-  public boolean canGrantNamespace(TCredentials c, Namespace.ID namespace)
+  public boolean canGrantNamespace(TCredentials c, NamespaceId namespace)
       throws ThriftSecurityException {
     return canModifyNamespacePermission(c, namespace);
   }
 
-  private boolean canModifyNamespacePermission(TCredentials c, Namespace.ID namespace)
+  private boolean canModifyNamespacePermission(TCredentials c, NamespaceId namespace)
       throws ThriftSecurityException {
     authenticate(c);
     // The one case where Table/SystemPermission -> NamespacePermission breaks down. The alternative
@@ -624,14 +625,14 @@ public class SecurityOperation {
     return hasSystemPermission(c, SystemPermission.GRANT, false);
   }
 
-  public boolean canRevokeTable(TCredentials c, String user, Table.ID tableId,
-      Namespace.ID namespaceId) throws ThriftSecurityException {
+  public boolean canRevokeTable(TCredentials c, String user, TableId tableId,
+      NamespaceId namespaceId) throws ThriftSecurityException {
     authenticate(c);
     return hasSystemPermissionWithNamespaceId(c, SystemPermission.ALTER_TABLE, namespaceId, false)
         || hasTablePermission(c, tableId, namespaceId, TablePermission.GRANT, false);
   }
 
-  public boolean canRevokeNamespace(TCredentials c, Namespace.ID namespace)
+  public boolean canRevokeNamespace(TCredentials c, NamespaceId namespace)
       throws ThriftSecurityException {
     return canModifyNamespacePermission(c, namespace);
   }
@@ -728,15 +729,15 @@ public class SecurityOperation {
     }
   }
 
-  public void grantTablePermission(TCredentials c, String user, Table.ID tableId,
-      TablePermission permission, Namespace.ID namespaceId) throws ThriftSecurityException {
+  public void grantTablePermission(TCredentials c, String user, TableId tableId,
+      TablePermission permission, NamespaceId namespaceId) throws ThriftSecurityException {
     if (!canGrantTable(c, user, tableId, namespaceId))
       throw new ThriftSecurityException(c.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
 
     targetUserExists(user);
 
     try {
-      permHandle.grantTablePermission(user, tableId.canonicalID(), permission);
+      permHandle.grantTablePermission(user, tableId.canonical(), permission);
       log.info("Granted table permission {} for user {} on the table {} at the request of user {}",
           permission, user, tableId, c.getPrincipal());
     } catch (AccumuloSecurityException e) {
@@ -746,7 +747,7 @@ public class SecurityOperation {
     }
   }
 
-  public void grantNamespacePermission(TCredentials c, String user, Namespace.ID namespace,
+  public void grantNamespacePermission(TCredentials c, String user, NamespaceId namespace,
       NamespacePermission permission) throws ThriftSecurityException {
     if (!canGrantNamespace(c, namespace))
       throw new ThriftSecurityException(c.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
@@ -754,7 +755,7 @@ public class SecurityOperation {
     targetUserExists(user);
 
     try {
-      permHandle.grantNamespacePermission(user, namespace.canonicalID(), permission);
+      permHandle.grantNamespacePermission(user, namespace.canonical(), permission);
       log.info("Granted namespace permission {} for user {} on the namespace {}"
           + " at the request of user {}", permission, user, namespace, c.getPrincipal());
     } catch (AccumuloSecurityException e) {
@@ -782,15 +783,15 @@ public class SecurityOperation {
     }
   }
 
-  public void revokeTablePermission(TCredentials c, String user, Table.ID tableId,
-      TablePermission permission, Namespace.ID namespaceId) throws ThriftSecurityException {
+  public void revokeTablePermission(TCredentials c, String user, TableId tableId,
+      TablePermission permission, NamespaceId namespaceId) throws ThriftSecurityException {
     if (!canRevokeTable(c, user, tableId, namespaceId))
       throw new ThriftSecurityException(c.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
 
     targetUserExists(user);
 
     try {
-      permHandle.revokeTablePermission(user, tableId.canonicalID(), permission);
+      permHandle.revokeTablePermission(user, tableId.canonical(), permission);
       log.info("Revoked table permission {} for user {} on the table {} at the request of user {}",
           permission, user, tableId, c.getPrincipal());
 
@@ -801,7 +802,7 @@ public class SecurityOperation {
     }
   }
 
-  public void revokeNamespacePermission(TCredentials c, String user, Namespace.ID namespace,
+  public void revokeNamespacePermission(TCredentials c, String user, NamespaceId namespace,
       NamespacePermission permission) throws ThriftSecurityException {
     if (!canRevokeNamespace(c, namespace))
       throw new ThriftSecurityException(c.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
@@ -809,7 +810,7 @@ public class SecurityOperation {
     targetUserExists(user);
 
     try {
-      permHandle.revokeNamespacePermission(user, namespace.canonicalID(), permission);
+      permHandle.revokeNamespacePermission(user, namespace.canonical(), permission);
       log.info("Revoked namespace permission {} for user {} on the namespace {}"
           + " at the request of user {}", permission, user, namespace, c.getPrincipal());
 
@@ -828,7 +829,7 @@ public class SecurityOperation {
     return _hasSystemPermission(user, permissionById, false);
   }
 
-  public boolean hasTablePermission(TCredentials credentials, String user, Table.ID tableId,
+  public boolean hasTablePermission(TCredentials credentials, String user, TableId tableId,
       TablePermission permissionById) throws ThriftSecurityException {
     if (!canAskAboutOtherUsers(credentials, user))
       throw new ThriftSecurityException(credentials.getPrincipal(),
@@ -837,7 +838,7 @@ public class SecurityOperation {
   }
 
   public boolean hasNamespacePermission(TCredentials credentials, String user,
-      Namespace.ID namespace, NamespacePermission permissionById) throws ThriftSecurityException {
+      NamespaceId namespace, NamespacePermission permissionById) throws ThriftSecurityException {
     if (!canAskAboutOtherUsers(credentials, user))
       throw new ThriftSecurityException(credentials.getPrincipal(),
           SecurityErrorCode.PERMISSION_DENIED);
@@ -849,13 +850,13 @@ public class SecurityOperation {
     return authenticator.listUsers();
   }
 
-  public void deleteTable(TCredentials credentials, Table.ID tableId, Namespace.ID namespaceId)
+  public void deleteTable(TCredentials credentials, TableId tableId, NamespaceId namespaceId)
       throws ThriftSecurityException {
     if (!canDeleteTable(credentials, tableId, namespaceId))
       throw new ThriftSecurityException(credentials.getPrincipal(),
           SecurityErrorCode.PERMISSION_DENIED);
     try {
-      permHandle.cleanTablePermissions(tableId.canonicalID());
+      permHandle.cleanTablePermissions(tableId.canonical());
     } catch (AccumuloSecurityException e) {
       e.setUser(credentials.getPrincipal());
       throw e.asThriftException();
@@ -865,13 +866,13 @@ public class SecurityOperation {
     }
   }
 
-  public void deleteNamespace(TCredentials credentials, Namespace.ID namespace)
+  public void deleteNamespace(TCredentials credentials, NamespaceId namespace)
       throws ThriftSecurityException {
     if (!canDeleteNamespace(credentials, namespace))
       throw new ThriftSecurityException(credentials.getPrincipal(),
           SecurityErrorCode.PERMISSION_DENIED);
     try {
-      permHandle.cleanNamespacePermissions(namespace.canonicalID());
+      permHandle.cleanNamespacePermissions(namespace.canonical());
     } catch (AccumuloSecurityException e) {
       e.setUser(credentials.getPrincipal());
       throw e.asThriftException();
@@ -881,20 +882,20 @@ public class SecurityOperation {
     }
   }
 
-  public boolean canExport(TCredentials credentials, Table.ID tableId, String tableName,
-      String exportDir, Namespace.ID namespaceId) throws ThriftSecurityException {
+  public boolean canExport(TCredentials credentials, TableId tableId, String tableName,
+      String exportDir, NamespaceId namespaceId) throws ThriftSecurityException {
     authenticate(credentials);
     return hasTablePermission(credentials, tableId, namespaceId, TablePermission.READ, false);
   }
 
   public boolean canImport(TCredentials credentials, String tableName, String importDir,
-      Namespace.ID namespaceId) throws ThriftSecurityException {
+      NamespaceId namespaceId) throws ThriftSecurityException {
     authenticate(credentials);
     return hasSystemPermissionWithNamespaceId(credentials, SystemPermission.CREATE_TABLE,
         namespaceId, false);
   }
 
-  public boolean canAlterNamespace(TCredentials credentials, Namespace.ID namespaceId)
+  public boolean canAlterNamespace(TCredentials credentials, NamespaceId namespaceId)
       throws ThriftSecurityException {
     authenticate(credentials);
     return hasSystemPermissionWithNamespaceId(credentials, SystemPermission.ALTER_NAMESPACE,
@@ -906,14 +907,14 @@ public class SecurityOperation {
     return hasSystemPermission(credentials, SystemPermission.CREATE_NAMESPACE, false);
   }
 
-  public boolean canDeleteNamespace(TCredentials credentials, Namespace.ID namespaceId)
+  public boolean canDeleteNamespace(TCredentials credentials, NamespaceId namespaceId)
       throws ThriftSecurityException {
     authenticate(credentials);
     return hasSystemPermissionWithNamespaceId(credentials, SystemPermission.DROP_NAMESPACE,
         namespaceId, false);
   }
 
-  public boolean canRenameNamespace(TCredentials credentials, Namespace.ID namespaceId)
+  public boolean canRenameNamespace(TCredentials credentials, NamespaceId namespaceId)
       throws ThriftSecurityException {
     authenticate(credentials);
     return hasSystemPermissionWithNamespaceId(credentials, SystemPermission.ALTER_NAMESPACE,
@@ -925,8 +926,8 @@ public class SecurityOperation {
     return hasSystemPermission(credentials, SystemPermission.OBTAIN_DELEGATION_TOKEN, false);
   }
 
-  public boolean canGetSummaries(TCredentials credentials, Table.ID tableId,
-      Namespace.ID namespaceId) throws ThriftSecurityException {
+  public boolean canGetSummaries(TCredentials credentials, TableId tableId, NamespaceId namespaceId)
+      throws ThriftSecurityException {
     authenticate(credentials);
     return hasTablePermission(credentials, tableId, namespaceId, TablePermission.GET_SUMMARIES,
         false);
diff --git a/server/base/src/main/java/org/apache/accumulo/server/security/handler/ZKAuthorizor.java b/server/base/src/main/java/org/apache/accumulo/server/security/handler/ZKAuthorizor.java
index 914ae6a..0ff17b9 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/security/handler/ZKAuthorizor.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/security/handler/ZKAuthorizor.java
@@ -27,8 +27,8 @@ import java.util.Set;
 import java.util.TreeSet;
 
 import org.apache.accumulo.core.client.AccumuloSecurityException;
-import org.apache.accumulo.core.clientImpl.Table;
 import org.apache.accumulo.core.clientImpl.thrift.SecurityErrorCode;
+import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.metadata.MetadataTable;
 import org.apache.accumulo.core.metadata.RootTable;
 import org.apache.accumulo.core.security.Authorizations;
@@ -92,7 +92,7 @@ public class ZKAuthorizor implements Authorizor {
     Set<SystemPermission> rootPerms = new TreeSet<>();
     for (SystemPermission p : SystemPermission.values())
       rootPerms.add(p);
-    Map<Table.ID,Set<TablePermission>> tablePerms = new HashMap<>();
+    Map<TableId,Set<TablePermission>> tablePerms = new HashMap<>();
     // Allow the root user to flush the metadata tables
     tablePerms.put(MetadataTable.ID, Collections.singleton(TablePermission.ALTER_TABLE));
     tablePerms.put(RootTable.ID, Collections.singleton(TablePermission.ALTER_TABLE));
diff --git a/server/base/src/main/java/org/apache/accumulo/server/security/handler/ZKPermHandler.java b/server/base/src/main/java/org/apache/accumulo/server/security/handler/ZKPermHandler.java
index 2183e8e..9bb05ff 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/security/handler/ZKPermHandler.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/security/handler/ZKPermHandler.java
@@ -29,8 +29,9 @@ import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.NamespaceNotFoundException;
 import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.clientImpl.Namespace;
-import org.apache.accumulo.core.clientImpl.Table;
 import org.apache.accumulo.core.clientImpl.thrift.SecurityErrorCode;
+import org.apache.accumulo.core.data.NamespaceId;
+import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.metadata.MetadataTable;
 import org.apache.accumulo.core.metadata.RootTable;
 import org.apache.accumulo.core.security.NamespacePermission;
@@ -395,15 +396,15 @@ public class ZKPermHandler implements PermissionHandler {
     Set<SystemPermission> rootPerms = new TreeSet<>();
     for (SystemPermission p : SystemPermission.values())
       rootPerms.add(p);
-    Map<Table.ID,Set<TablePermission>> tablePerms = new HashMap<>();
+    Map<TableId,Set<TablePermission>> tablePerms = new HashMap<>();
     // Allow the root user to flush the system tables
     tablePerms.put(RootTable.ID, Collections.singleton(TablePermission.ALTER_TABLE));
     tablePerms.put(MetadataTable.ID, Collections.singleton(TablePermission.ALTER_TABLE));
     // essentially the same but on the system namespace, the ALTER_TABLE permission is now redundant
-    Map<Namespace.ID,Set<NamespacePermission>> namespacePerms = new HashMap<>();
-    namespacePerms.put(Namespace.ID.ACCUMULO,
+    Map<NamespaceId,Set<NamespacePermission>> namespacePerms = new HashMap<>();
+    namespacePerms.put(Namespace.ACCUMULO.id(),
         Collections.singleton(NamespacePermission.ALTER_NAMESPACE));
-    namespacePerms.put(Namespace.ID.ACCUMULO,
+    namespacePerms.put(Namespace.ACCUMULO.id(),
         Collections.singleton(NamespacePermission.ALTER_TABLE));
 
     try {
@@ -414,9 +415,9 @@ public class ZKPermHandler implements PermissionHandler {
       initUser(rootuser);
       zoo.putPersistentData(ZKUserPath + "/" + rootuser + ZKUserSysPerms,
           ZKSecurityTool.convertSystemPermissions(rootPerms), NodeExistsPolicy.FAIL);
-      for (Entry<Table.ID,Set<TablePermission>> entry : tablePerms.entrySet())
+      for (Entry<TableId,Set<TablePermission>> entry : tablePerms.entrySet())
         createTablePerm(rootuser, entry.getKey(), entry.getValue());
-      for (Entry<Namespace.ID,Set<NamespacePermission>> entry : namespacePerms.entrySet())
+      for (Entry<NamespaceId,Set<NamespacePermission>> entry : namespacePerms.entrySet())
         createNamespacePerm(rootuser, entry.getKey(), entry.getValue());
     } catch (KeeperException | InterruptedException e) {
       log.error("{}", e.getMessage(), e);
@@ -445,7 +446,7 @@ public class ZKPermHandler implements PermissionHandler {
    * Sets up a new table configuration for the provided user/table. No checking for existence is
    * done here, it should be done before calling.
    */
-  private void createTablePerm(String user, Table.ID table, Set<TablePermission> perms)
+  private void createTablePerm(String user, TableId table, Set<TablePermission> perms)
       throws KeeperException, InterruptedException {
     synchronized (zooCache) {
       zooCache.clear();
@@ -458,7 +459,7 @@ public class ZKPermHandler implements PermissionHandler {
    * Sets up a new namespace configuration for the provided user/table. No checking for existence is
    * done here, it should be done before calling.
    */
-  private void createNamespacePerm(String user, Namespace.ID namespace,
+  private void createNamespacePerm(String user, NamespaceId namespace,
       Set<NamespacePermission> perms) throws KeeperException, InterruptedException {
     synchronized (zooCache) {
       zooCache.clear();
diff --git a/server/base/src/main/java/org/apache/accumulo/server/tables/TableManager.java b/server/base/src/main/java/org/apache/accumulo/server/tables/TableManager.java
index 76a903a..bf5df9f 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/tables/TableManager.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/tables/TableManager.java
@@ -27,9 +27,9 @@ import java.util.Set;
 
 import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.NamespaceNotFoundException;
-import org.apache.accumulo.core.clientImpl.Namespace;
-import org.apache.accumulo.core.clientImpl.Table;
 import org.apache.accumulo.core.clientImpl.Tables;
+import org.apache.accumulo.core.data.NamespaceId;
+import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.master.state.tables.TableState;
 import org.apache.accumulo.core.util.Pair;
 import org.apache.accumulo.fate.zookeeper.IZooReaderWriter;
@@ -55,7 +55,7 @@ public class TableManager {
 
   private static final Logger log = LoggerFactory.getLogger(TableManager.class);
   private static final Set<TableObserver> observers = Collections.synchronizedSet(new HashSet<>());
-  private static final Map<Table.ID,TableState> tableStateCache = Collections
+  private static final Map<TableId,TableState> tableStateCache = Collections
       .synchronizedMap(new HashMap<>());
   private static final byte[] ZERO_BYTE = {'0'};
 
@@ -66,7 +66,7 @@ public class TableManager {
   private ZooCache zooStateCache;
 
   public static void prepareNewNamespaceState(IZooReaderWriter zoo, String instanceId,
-      Namespace.ID namespaceId, String namespace, NodeExistsPolicy existsPolicy)
+      NamespaceId namespaceId, String namespace, NodeExistsPolicy existsPolicy)
       throws KeeperException, InterruptedException {
     log.debug("Creating ZooKeeper entries for new namespace {} (ID: {})", namespace, namespaceId);
     String zPath = Constants.ZROOT + "/" + instanceId + Constants.ZNAMESPACES + "/" + namespaceId;
@@ -77,8 +77,8 @@ public class TableManager {
     zoo.putPersistentData(zPath + Constants.ZNAMESPACE_CONF, new byte[0], existsPolicy);
   }
 
-  public static void prepareNewTableState(IZooReaderWriter zoo, String instanceId, Table.ID tableId,
-      Namespace.ID namespaceId, String tableName, TableState state, NodeExistsPolicy existsPolicy)
+  public static void prepareNewTableState(IZooReaderWriter zoo, String instanceId, TableId tableId,
+      NamespaceId namespaceId, String tableName, TableState state, NodeExistsPolicy existsPolicy)
       throws KeeperException, InterruptedException {
     // state gets created last
     log.debug("Creating ZooKeeper entries for new table {} (ID: {}) in namespace (ID: {})",
@@ -88,8 +88,8 @@ public class TableManager {
     String zTablePath = Constants.ZROOT + "/" + instanceId + Constants.ZTABLES + "/" + tableId;
     zoo.putPersistentData(zTablePath, new byte[0], existsPolicy);
     zoo.putPersistentData(zTablePath + Constants.ZTABLE_CONF, new byte[0], existsPolicy);
-    zoo.putPersistentData(zTablePath + Constants.ZTABLE_NAMESPACE, namespaceId.getUtf8(),
-        existsPolicy);
+    zoo.putPersistentData(zTablePath + Constants.ZTABLE_NAMESPACE,
+        namespaceId.canonical().getBytes(UTF_8), existsPolicy);
     zoo.putPersistentData(zTablePath + Constants.ZTABLE_NAME, tableName.getBytes(UTF_8),
         existsPolicy);
     zoo.putPersistentData(zTablePath + Constants.ZTABLE_FLUSH_ID, ZERO_BYTE, existsPolicy);
@@ -108,7 +108,7 @@ public class TableManager {
     updateTableStateCache();
   }
 
-  public TableState getTableState(Table.ID tableId) {
+  public TableState getTableState(TableId tableId) {
     return tableStateCache.get(tableId);
   }
 
@@ -142,7 +142,7 @@ public class TableManager {
 
   }
 
-  public synchronized void transitionTableState(final Table.ID tableId, final TableState newState) {
+  public synchronized void transitionTableState(final TableId tableId, final TableState newState) {
     Preconditions.checkArgument(newState != TableState.UNKNOWN);
     String statePath = zkRoot + Constants.ZTABLES + "/" + tableId + Constants.ZTABLE_STATE;
 
@@ -194,11 +194,11 @@ public class TableManager {
       for (String tableId : zooStateCache.getChildren(zkRoot + Constants.ZTABLES))
         if (zooStateCache
             .get(zkRoot + Constants.ZTABLES + "/" + tableId + Constants.ZTABLE_STATE) != null)
-          updateTableStateCache(Table.ID.of(tableId));
+          updateTableStateCache(TableId.of(tableId));
     }
   }
 
-  public TableState updateTableStateCache(Table.ID tableId) {
+  public TableState updateTableStateCache(TableId tableId) {
     synchronized (tableStateCache) {
       TableState tState = TableState.UNKNOWN;
       byte[] data = zooStateCache
@@ -216,7 +216,7 @@ public class TableManager {
     }
   }
 
-  public void addTable(Table.ID tableId, Namespace.ID namespaceId, String tableName,
+  public void addTable(TableId tableId, NamespaceId namespaceId, String tableName,
       NodeExistsPolicy existsPolicy)
       throws KeeperException, InterruptedException, NamespaceNotFoundException {
     prepareNewTableState(zoo, instanceID, tableId, namespaceId, tableName, TableState.NEW,
@@ -224,8 +224,8 @@ public class TableManager {
     updateTableStateCache(tableId);
   }
 
-  public void cloneTable(Table.ID srcTableId, Table.ID tableId, String tableName,
-      Namespace.ID namespaceId, Map<String,String> propertiesToSet, Set<String> propertiesToExclude,
+  public void cloneTable(TableId srcTableId, TableId tableId, String tableName,
+      NamespaceId namespaceId, Map<String,String> propertiesToSet, Set<String> propertiesToExclude,
       NodeExistsPolicy existsPolicy) throws KeeperException, InterruptedException {
     prepareNewTableState(zoo, instanceID, tableId, namespaceId, tableName, TableState.NEW,
         existsPolicy);
@@ -246,7 +246,7 @@ public class TableManager {
     updateTableStateCache(tableId);
   }
 
-  public void removeTable(Table.ID tableId) throws KeeperException, InterruptedException {
+  public void removeTable(TableId tableId) throws KeeperException, InterruptedException {
     synchronized (tableStateCache) {
       tableStateCache.remove(tableId);
       zoo.recursiveDelete(zkRoot + Constants.ZTABLES + "/" + tableId + Constants.ZTABLE_STATE,
@@ -274,14 +274,14 @@ public class TableManager {
       final EventType zType = event.getType();
 
       String tablesPrefix = zkRoot + Constants.ZTABLES;
-      Table.ID tableId = null;
+      TableId tableId = null;
 
       if (zPath != null && zPath.startsWith(tablesPrefix + "/")) {
         String suffix = zPath.substring(tablesPrefix.length() + 1);
         if (suffix.contains("/")) {
           String[] sa = suffix.split("/", 2);
           if (Constants.ZTABLE_STATE.equals("/" + sa[1]))
-            tableId = Table.ID.of(sa[0]);
+            tableId = TableId.of(sa[0]);
         }
         if (tableId == null) {
           log.warn("Unknown path in {}", event);
@@ -336,7 +336,7 @@ public class TableManager {
     }
   }
 
-  public void removeNamespace(Namespace.ID namespaceId)
+  public void removeNamespace(NamespaceId namespaceId)
       throws KeeperException, InterruptedException {
     zoo.recursiveDelete(zkRoot + Constants.ZNAMESPACES + "/" + namespaceId, NodeMissingPolicy.SKIP);
   }
diff --git a/server/base/src/main/java/org/apache/accumulo/server/tables/TableObserver.java b/server/base/src/main/java/org/apache/accumulo/server/tables/TableObserver.java
index 2ed2c48..1abfe56 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/tables/TableObserver.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/tables/TableObserver.java
@@ -16,13 +16,13 @@
  */
 package org.apache.accumulo.server.tables;
 
-import org.apache.accumulo.core.clientImpl.Table;
+import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.master.state.tables.TableState;
 
 public interface TableObserver {
   void initialize();
 
-  void stateChanged(Table.ID tableId, TableState tState);
+  void stateChanged(TableId tableId, TableState tState);
 
   void sessionExpired();
 }
diff --git a/server/base/src/main/java/org/apache/accumulo/server/tabletserver/LargestFirstMemoryManager.java b/server/base/src/main/java/org/apache/accumulo/server/tabletserver/LargestFirstMemoryManager.java
index a7d8af8..121cc7a 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/tabletserver/LargestFirstMemoryManager.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/tabletserver/LargestFirstMemoryManager.java
@@ -23,8 +23,8 @@ import java.util.Map.Entry;
 import java.util.TreeMap;
 
 import org.apache.accumulo.core.client.TableNotFoundException;
-import org.apache.accumulo.core.clientImpl.Table;
 import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.dataImpl.KeyExtent;
 import org.apache.accumulo.server.conf.ServerConfiguration;
 import org.slf4j.Logger;
@@ -52,7 +52,7 @@ public class LargestFirstMemoryManager implements MemoryManager {
   // The fraction of memory that needs to be used before we begin flushing.
   private double compactionThreshold;
   private long maxObserved;
-  private final HashMap<Table.ID,Long> mincIdleThresholds = new HashMap<>();
+  private final HashMap<TableId,Long> mincIdleThresholds = new HashMap<>();
   private ServerConfiguration config = null;
 
   private static class TabletInfo {
@@ -141,14 +141,14 @@ public class LargestFirstMemoryManager implements MemoryManager {
   }
 
   protected long getMinCIdleThreshold(KeyExtent extent) {
-    Table.ID tableId = extent.getTableId();
+    TableId tableId = extent.getTableId();
     if (!mincIdleThresholds.containsKey(tableId))
       mincIdleThresholds.put(tableId, config.getTableConfiguration(tableId)
           .getTimeInMillis(Property.TABLE_MINC_COMPACT_IDLETIME));
     return mincIdleThresholds.get(tableId);
   }
 
-  protected boolean tableExists(Table.ID tableId) {
+  protected boolean tableExists(TableId tableId) {
     // make sure that the table still exists by checking if it has a configuration
     return config.getTableConfiguration(tableId) != null;
   }
diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/CheckForMetadataProblems.java b/server/base/src/main/java/org/apache/accumulo/server/util/CheckForMetadataProblems.java
index 248bb08..014c36d 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/util/CheckForMetadataProblems.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/util/CheckForMetadataProblems.java
@@ -116,7 +116,7 @@ public class CheckForMetadataProblems {
         count++;
 
         String tableName = (new KeyExtent(entry.getKey().getRow(), (Text) null)).getTableId()
-            .canonicalID();
+            .canonical();
 
         TreeSet<KeyExtent> tablets = tables.get(tableName);
         if (tablets == null) {
diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/FindOfflineTablets.java b/server/base/src/main/java/org/apache/accumulo/server/util/FindOfflineTablets.java
index 016888a..706cdef 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/util/FindOfflineTablets.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/util/FindOfflineTablets.java
@@ -21,9 +21,9 @@ import java.util.Set;
 import java.util.concurrent.atomic.AtomicBoolean;
 
 import org.apache.accumulo.core.client.TableNotFoundException;
-import org.apache.accumulo.core.clientImpl.Table;
 import org.apache.accumulo.core.clientImpl.Tables;
 import org.apache.accumulo.core.data.Range;
+import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.dataImpl.KeyExtent;
 import org.apache.accumulo.core.master.state.tables.TableState;
 import org.apache.accumulo.core.metadata.MetadataTable;
@@ -92,7 +92,7 @@ public class FindOfflineTablets {
 
     Range range = MetadataSchema.TabletsSection.getRange();
     if (tableName != null) {
-      Table.ID tableId = Tables.getTableId(context, tableName);
+      TableId tableId = Tables.getTableId(context, tableName);
       range = new KeyExtent(tableId, null, null).toMetadataRange();
     }
 
diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/MasterMetadataUtil.java b/server/base/src/main/java/org/apache/accumulo/server/util/MasterMetadataUtil.java
index 6e90c93..2027249 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/util/MasterMetadataUtil.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/util/MasterMetadataUtil.java
@@ -33,11 +33,11 @@ import java.util.concurrent.TimeUnit;
 import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.Scanner;
 import org.apache.accumulo.core.clientImpl.ScannerImpl;
-import org.apache.accumulo.core.clientImpl.Table;
 import org.apache.accumulo.core.data.Key;
 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.TableId;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.dataImpl.KeyExtent;
 import org.apache.accumulo.core.metadata.MetadataTable;
@@ -127,12 +127,12 @@ public class MasterMetadataUtil {
 
     Text metadataPrevEndRow = KeyExtent.decodePrevEndRow(prevEndRowIBW);
 
-    Table.ID tableId = (new KeyExtent(metadataEntry, (Text) null)).getTableId();
+    TableId tableId = (new KeyExtent(metadataEntry, (Text) null)).getTableId();
 
     return fixSplit(context, tableId, metadataEntry, metadataPrevEndRow, oper, splitRatio, lock);
   }
 
-  private static KeyExtent fixSplit(ServerContext context, Table.ID tableId, Text metadataEntry,
+  private static KeyExtent fixSplit(ServerContext context, TableId tableId, Text metadataEntry,
       Text metadataPrevEndRow, Value oper, double splitRatio, ZooLock lock)
       throws AccumuloException {
     if (metadataPrevEndRow == null)
diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/MetadataTableUtil.java b/server/base/src/main/java/org/apache/accumulo/server/util/MetadataTableUtil.java
index 0e8beb2..83c6831 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/util/MetadataTableUtil.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/util/MetadataTableUtil.java
@@ -50,12 +50,12 @@ import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.clientImpl.BatchWriterImpl;
 import org.apache.accumulo.core.clientImpl.Credentials;
 import org.apache.accumulo.core.clientImpl.ScannerImpl;
-import org.apache.accumulo.core.clientImpl.Table;
 import org.apache.accumulo.core.clientImpl.Writer;
 import org.apache.accumulo.core.data.Key;
 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.TableId;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.dataImpl.KeyExtent;
 import org.apache.accumulo.core.metadata.MetadataTable;
@@ -361,7 +361,7 @@ public class MetadataTableUtil {
   public static void addDeleteEntries(KeyExtent extent, Set<FileRef> datafilesToDelete,
       ServerContext context) {
 
-    Table.ID tableId = extent.getTableId();
+    TableId tableId = extent.getTableId();
 
     // TODO could use batch writer,would need to handle failure and retry like update does -
     // ACCUMULO-1294
@@ -371,12 +371,12 @@ public class MetadataTableUtil {
     }
   }
 
-  public static void addDeleteEntry(ServerContext context, Table.ID tableId, String path) {
+  public static void addDeleteEntry(ServerContext context, TableId tableId, String path) {
     update(context, createDeleteMutation(context, tableId, path),
         new KeyExtent(tableId, null, null));
   }
 
-  public static Mutation createDeleteMutation(ServerContext context, Table.ID tableId,
+  public static Mutation createDeleteMutation(ServerContext context, TableId tableId,
       String pathToRemove) {
     Path path = context.getVolumeManager().getFullPath(tableId, pathToRemove);
     Mutation delFlag = new Mutation(new Text(MetadataSchema.DeletesSection.getRowPrefix() + path));
@@ -443,7 +443,7 @@ public class MetadataTableUtil {
     }
   }
 
-  public static void deleteTable(Table.ID tableId, boolean insertDeletes, ServerContext context,
+  public static void deleteTable(TableId tableId, boolean insertDeletes, ServerContext context,
       ZooLock lock) throws AccumuloException {
     try (Scanner ms = new ScannerImpl(context, MetadataTable.ID, Authorizations.EMPTY);
         BatchWriter bw = new BatchWriterImpl(context, MetadataTable.ID,
@@ -618,7 +618,7 @@ public class MetadataTableUtil {
   }
 
   private static Scanner getTabletLogScanner(ServerContext context, KeyExtent extent) {
-    Table.ID tableId = MetadataTable.ID;
+    TableId tableId = MetadataTable.ID;
     if (extent.isMeta())
       tableId = RootTable.ID;
     Scanner scanner = new ScannerImpl(context, tableId, Authorizations.EMPTY);
@@ -704,7 +704,7 @@ public class MetadataTableUtil {
   }
 
   private static void getFiles(Set<String> files, Collection<String> tabletFiles,
-      Table.ID srcTableId) {
+      TableId srcTableId) {
     for (String file : tabletFiles) {
       if (srcTableId != null && !file.startsWith("../") && !file.contains(":")) {
         file = "../" + srcTableId + file;
@@ -713,7 +713,7 @@ public class MetadataTableUtil {
     }
   }
 
-  private static Mutation createCloneMutation(Table.ID srcTableId, Table.ID tableId,
+  private static Mutation createCloneMutation(TableId srcTableId, TableId tableId,
       Map<Key,Value> tablet) {
 
     KeyExtent ke = new KeyExtent(tablet.keySet().iterator().next().getRow(), (Text) null);
@@ -740,7 +740,7 @@ public class MetadataTableUtil {
     return m;
   }
 
-  private static Iterable<TabletMetadata> createCloneScanner(String testTableName, Table.ID tableId,
+  private static Iterable<TabletMetadata> createCloneScanner(String testTableName, TableId tableId,
       AccumuloClient client) throws TableNotFoundException {
 
     String tableName;
@@ -763,7 +763,7 @@ public class MetadataTableUtil {
   }
 
   @VisibleForTesting
-  public static void initializeClone(String testTableName, Table.ID srcTableId, Table.ID tableId,
+  public static void initializeClone(String testTableName, TableId srcTableId, TableId tableId,
       AccumuloClient client, BatchWriter bw)
       throws TableNotFoundException, MutationsRejectedException {
 
@@ -779,12 +779,12 @@ public class MetadataTableUtil {
   }
 
   private static int compareEndRows(Text endRow1, Text endRow2) {
-    return new KeyExtent(Table.ID.of("0"), endRow1, null)
-        .compareTo(new KeyExtent(Table.ID.of("0"), endRow2, null));
+    return new KeyExtent(TableId.of("0"), endRow1, null)
+        .compareTo(new KeyExtent(TableId.of("0"), endRow2, null));
   }
 
   @VisibleForTesting
-  public static int checkClone(String testTableName, Table.ID srcTableId, Table.ID tableId,
+  public static int checkClone(String testTableName, TableId srcTableId, TableId tableId,
       AccumuloClient client, BatchWriter bw)
       throws TableNotFoundException, MutationsRejectedException {
 
@@ -866,7 +866,7 @@ public class MetadataTableUtil {
     return rewrites;
   }
 
-  public static void cloneTable(ServerContext context, Table.ID srcTableId, Table.ID tableId,
+  public static void cloneTable(ServerContext context, TableId srcTableId, TableId tableId,
       VolumeManager volumeManager) throws Exception {
 
     try (BatchWriter bw = context.createBatchWriter(MetadataTable.NAME, new BatchWriterConfig())) {
@@ -932,7 +932,7 @@ public class MetadataTableUtil {
     update(context, zooLock, m, extent);
   }
 
-  public static void removeBulkLoadEntries(AccumuloClient client, Table.ID tableId, long tid)
+  public static void removeBulkLoadEntries(AccumuloClient client, TableId tableId, long tid)
       throws Exception {
     try (
         Scanner mscanner = new IsolatedScanner(
@@ -1005,7 +1005,7 @@ public class MetadataTableUtil {
     // new KeyExtent is only added to force update to write to the metadata table, not the root
     // table
     // because bulk loads aren't supported to the metadata table
-    update(context, m, new KeyExtent(Table.ID.of("anythingNotMetadata"), null, null));
+    update(context, m, new KeyExtent(TableId.of("anythingNotMetadata"), null, null));
   }
 
   public static void removeBulkLoadInProgressFlag(ServerContext context, String path) {
@@ -1016,7 +1016,7 @@ public class MetadataTableUtil {
     // new KeyExtent is only added to force update to write to the metadata table, not the root
     // table
     // because bulk loads aren't supported to the metadata table
-    update(context, m, new KeyExtent(Table.ID.of("anythingNotMetadata"), null, null));
+    update(context, m, new KeyExtent(TableId.of("anythingNotMetadata"), null, null));
   }
 
   /**
@@ -1066,7 +1066,7 @@ public class MetadataTableUtil {
   public static void moveMetaDeleteMarkersFrom14(ServerContext context) {
     // new KeyExtent is only added to force update to write to the metadata table, not the root
     // table
-    KeyExtent notMetadata = new KeyExtent(Table.ID.of("anythingNotMetadata"), null, null);
+    KeyExtent notMetadata = new KeyExtent(TableId.of("anythingNotMetadata"), null, null);
 
     // move delete markers from the normal delete keyspace to the root tablet delete keyspace if the
     // files are for the !METADATA table
diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/NamespacePropUtil.java b/server/base/src/main/java/org/apache/accumulo/server/util/NamespacePropUtil.java
index ecba748..1b51948 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/util/NamespacePropUtil.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/util/NamespacePropUtil.java
@@ -19,8 +19,8 @@ package org.apache.accumulo.server.util;
 import static java.nio.charset.StandardCharsets.UTF_8;
 
 import org.apache.accumulo.core.Constants;
-import org.apache.accumulo.core.clientImpl.Namespace;
 import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.core.data.NamespaceId;
 import org.apache.accumulo.fate.zookeeper.IZooReaderWriter;
 import org.apache.accumulo.fate.zookeeper.ZooUtil.NodeExistsPolicy;
 import org.apache.accumulo.fate.zookeeper.ZooUtil.NodeMissingPolicy;
@@ -28,7 +28,7 @@ import org.apache.accumulo.server.ServerContext;
 import org.apache.zookeeper.KeeperException;
 
 public class NamespacePropUtil {
-  public static boolean setNamespaceProperty(ServerContext context, Namespace.ID namespaceId,
+  public static boolean setNamespaceProperty(ServerContext context, NamespaceId namespaceId,
       String property, String value) throws KeeperException, InterruptedException {
     if (!isPropertyValid(property, value))
       return false;
@@ -53,13 +53,13 @@ public class NamespacePropUtil {
         && Property.isValidTablePropertyKey(property);
   }
 
-  public static void removeNamespaceProperty(ServerContext context, Namespace.ID namespaceId,
+  public static void removeNamespaceProperty(ServerContext context, NamespaceId namespaceId,
       String property) throws InterruptedException, KeeperException {
     String zPath = getPath(context, namespaceId) + "/" + property;
     context.getZooReaderWriter().recursiveDelete(zPath, NodeMissingPolicy.SKIP);
   }
 
-  private static String getPath(ServerContext context, Namespace.ID namespaceId) {
+  private static String getPath(ServerContext context, NamespaceId namespaceId) {
     return context.getZooKeeperRoot() + Constants.ZNAMESPACES + "/" + namespaceId
         + Constants.ZNAMESPACE_CONF;
   }
diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/RandomizeVolumes.java b/server/base/src/main/java/org/apache/accumulo/server/util/RandomizeVolumes.java
index a16b2bf..854cf39 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/util/RandomizeVolumes.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/util/RandomizeVolumes.java
@@ -28,9 +28,9 @@ import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.BatchWriter;
 import org.apache.accumulo.core.client.Scanner;
 import org.apache.accumulo.core.client.TableNotFoundException;
-import org.apache.accumulo.core.clientImpl.Table;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Mutation;
+import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.dataImpl.KeyExtent;
 import org.apache.accumulo.core.master.state.tables.TableState;
@@ -76,7 +76,7 @@ public class RandomizeVolumes {
       log.error("Could not determine the table ID for table {}", tableName);
       return 2;
     }
-    Table.ID tableId = Table.ID.of(tblStr);
+    TableId tableId = TableId.of(tblStr);
     TableState tableState = context.getTableManager().getTableState(tableId);
     if (tableState != TableState.OFFLINE) {
       log.info("Taking {} offline", tableName);
@@ -95,7 +95,7 @@ public class RandomizeVolumes {
       String directory;
       if (oldLocation.contains(":")) {
         String[] parts = oldLocation.split(Path.SEPARATOR);
-        Table.ID tableIdEntry = Table.ID.of(parts[parts.length - 2]);
+        TableId tableIdEntry = TableId.of(parts[parts.length - 2]);
         if (!tableIdEntry.equals(tableId)) {
           log.error("Unexpected table id found: {}, expected {}; skipping", tableIdEntry, tableId);
           continue;
diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/RemoveEntriesForMissingFiles.java b/server/base/src/main/java/org/apache/accumulo/server/util/RemoveEntriesForMissingFiles.java
index 5369600..735f47b 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/util/RemoveEntriesForMissingFiles.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/util/RemoveEntriesForMissingFiles.java
@@ -31,11 +31,11 @@ import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.BatchWriter;
 import org.apache.accumulo.core.client.BatchWriterConfig;
 import org.apache.accumulo.core.client.Scanner;
-import org.apache.accumulo.core.clientImpl.Table;
 import org.apache.accumulo.core.clientImpl.Tables;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.data.Range;
+import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.dataImpl.KeyExtent;
 import org.apache.accumulo.core.metadata.MetadataTable;
@@ -195,7 +195,7 @@ public class RemoveEntriesForMissingFiles {
     } else if (tableName.equals(MetadataTable.NAME)) {
       return checkTable(context, RootTable.NAME, MetadataSchema.TabletsSection.getRange(), fix);
     } else {
-      Table.ID tableId = Tables.getTableId(context, tableName);
+      TableId tableId = Tables.getTableId(context, tableName);
       Range range = new KeyExtent(tableId, null, null).toMetadataRange();
       return checkTable(context, MetadataTable.NAME, range, fix);
     }
diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/ReplicationTableUtil.java b/server/base/src/main/java/org/apache/accumulo/server/util/ReplicationTableUtil.java
index f8defee..fbad0cb 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/util/ReplicationTableUtil.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/util/ReplicationTableUtil.java
@@ -189,7 +189,7 @@ public class ReplicationTableUtil {
 
   private static Mutation createUpdateMutation(Text row, Value v, KeyExtent extent) {
     Mutation m = new Mutation(row);
-    m.put(MetadataSchema.ReplicationSection.COLF, new Text(extent.getTableId().getUtf8()), v);
+    m.put(MetadataSchema.ReplicationSection.COLF, new Text(extent.getTableId().canonical()), v);
     return m;
   }
 }
diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/TableDiskUsage.java b/server/base/src/main/java/org/apache/accumulo/server/util/TableDiskUsage.java
index a405975..c4066af 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/util/TableDiskUsage.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/util/TableDiskUsage.java
@@ -34,9 +34,9 @@ import org.apache.accumulo.core.client.AccumuloClient;
 import org.apache.accumulo.core.client.Scanner;
 import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.clientImpl.ClientContext;
-import org.apache.accumulo.core.clientImpl.Table;
 import org.apache.accumulo.core.clientImpl.Tables;
 import org.apache.accumulo.core.data.Key;
+import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.dataImpl.KeyExtent;
 import org.apache.accumulo.core.metadata.MetadataTable;
@@ -57,12 +57,12 @@ public class TableDiskUsage {
 
   private static final Logger log = LoggerFactory.getLogger(TableDiskUsage.class);
   private int nextInternalId = 0;
-  private Map<Table.ID,Integer> internalIds = new HashMap<>();
-  private Map<Integer,Table.ID> externalIds = new HashMap<>();
+  private Map<TableId,Integer> internalIds = new HashMap<>();
+  private Map<Integer,TableId> externalIds = new HashMap<>();
   private Map<String,Integer[]> tableFiles = new HashMap<>();
   private Map<String,Long> fileSizes = new HashMap<>();
 
-  void addTable(Table.ID tableId) {
+  void addTable(TableId tableId) {
     if (internalIds.containsKey(tableId))
       throw new IllegalArgumentException("Already added table " + tableId);
 
@@ -75,7 +75,7 @@ public class TableDiskUsage {
     externalIds.put(iid, tableId);
   }
 
-  void linkFileAndTable(Table.ID tableId, String file) {
+  void linkFileAndTable(TableId tableId, String file) {
     // get the internal id for this table
     int internalId = internalIds.get(tableId);
 
@@ -96,7 +96,7 @@ public class TableDiskUsage {
     fileSizes.put(file, size);
   }
 
-  Map<List<Table.ID>,Long> calculateUsage() {
+  Map<List<TableId>,Long> calculateUsage() {
 
     // Bitset of tables that contain a file and total usage by all files that share that usage
     Map<List<Integer>,Long> usage = new HashMap<>();
@@ -122,10 +122,10 @@ public class TableDiskUsage {
 
     }
 
-    Map<List<Table.ID>,Long> externalUsage = new HashMap<>();
+    Map<List<TableId>,Long> externalUsage = new HashMap<>();
 
     for (Entry<List<Integer>,Long> entry : usage.entrySet()) {
-      List<Table.ID> externalKey = new ArrayList<>();
+      List<TableId> externalKey = new ArrayList<>();
       List<Integer> key = entry.getKey();
       // table bitset
       for (int i = 0; i < key.size(); i++)
@@ -151,20 +151,20 @@ public class TableDiskUsage {
     printDiskUsage(tableNames, fs, client, line -> System.out.println(line), humanReadable);
   }
 
-  public static Map<TreeSet<String>,Long> getDiskUsage(Set<Table.ID> tableIds, VolumeManager fs,
+  public static Map<TreeSet<String>,Long> getDiskUsage(Set<TableId> tableIds, VolumeManager fs,
       AccumuloClient client) throws IOException {
     TableDiskUsage tdu = new TableDiskUsage();
 
     // Add each tableID
-    for (Table.ID tableId : tableIds)
+    for (TableId tableId : tableIds)
       tdu.addTable(tableId);
 
-    HashSet<Table.ID> tablesReferenced = new HashSet<>(tableIds);
-    HashSet<Table.ID> emptyTableIds = new HashSet<>();
+    HashSet<TableId> tablesReferenced = new HashSet<>(tableIds);
+    HashSet<TableId> emptyTableIds = new HashSet<>();
     HashSet<String> nameSpacesReferenced = new HashSet<>();
 
     // For each table ID
-    for (Table.ID tableId : tableIds) {
+    for (TableId tableId : tableIds) {
       Scanner mdScanner;
       try {
         mdScanner = client.createScanner(MetadataTable.NAME, Authorizations.EMPTY);
@@ -187,8 +187,8 @@ public class TableDiskUsage {
         if (file.contains(":") || file.startsWith("../")) {
           String ref = parts[parts.length - 3];
           // Track any tables which are referenced externally by the current table
-          if (!ref.equals(tableId.canonicalID())) {
-            tablesReferenced.add(Table.ID.of(ref));
+          if (!ref.equals(tableId.canonical())) {
+            tablesReferenced.add(TableId.of(ref));
           }
           if (file.contains(":") && parts.length > 3) {
             List<String> base = Arrays.asList(Arrays.copyOf(parts, parts.length - 3));
@@ -202,7 +202,7 @@ public class TableDiskUsage {
     }
 
     // Each table seen (provided by user, or reference by table the user provided)
-    for (Table.ID tableId : tablesReferenced) {
+    for (TableId tableId : tablesReferenced) {
       for (String tableDir : nameSpacesReferenced) {
         // Find each file and add its size
         FileStatus[] files = fs.globStatus(new Path(tableDir + "/" + tableId + "/*/*"));
@@ -216,7 +216,7 @@ public class TableDiskUsage {
       }
     }
 
-    Map<Table.ID,String> reverseTableIdMap = Tables.getIdToNameMap((ClientContext) client);
+    Map<TableId,String> reverseTableIdMap = Tables.getIdToNameMap((ClientContext) client);
 
     TreeMap<TreeSet<String>,Long> usage = new TreeMap<>((o1, o2) -> {
       int len1 = o1.size();
@@ -244,10 +244,10 @@ public class TableDiskUsage {
       return len1 - len2;
     });
 
-    for (Entry<List<Table.ID>,Long> entry : tdu.calculateUsage().entrySet()) {
+    for (Entry<List<TableId>,Long> entry : tdu.calculateUsage().entrySet()) {
       TreeSet<String> tableNames = new TreeSet<>();
       // Convert size shared by each table id into size shared by each table name
-      for (Table.ID tableId : entry.getKey())
+      for (TableId tableId : entry.getKey())
         tableNames.add(reverseTableIdMap.get(tableId));
 
       // Make table names to shared file size
@@ -256,7 +256,7 @@ public class TableDiskUsage {
 
     if (!emptyTableIds.isEmpty()) {
       TreeSet<String> emptyTables = new TreeSet<>();
-      for (Table.ID tableId : emptyTableIds) {
+      for (TableId tableId : emptyTableIds) {
         emptyTables.add(reverseTableIdMap.get(tableId));
       }
       usage.put(emptyTables, 0L);
@@ -269,11 +269,11 @@ public class TableDiskUsage {
       AccumuloClient client, Printer printer, boolean humanReadable)
       throws TableNotFoundException, IOException {
 
-    HashSet<Table.ID> tableIds = new HashSet<>();
+    HashSet<TableId> tableIds = new HashSet<>();
 
     // Get table IDs for all tables requested to be 'du'
     for (String tableName : tableNames) {
-      Table.ID tableId = Tables.getTableId((ClientContext) client, tableName);
+      TableId tableId = Tables.getTableId((ClientContext) client, tableName);
       if (tableId == null)
         throw new TableNotFoundException(null, tableName, "Table " + tableName + " not found");
 
diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/TablePropUtil.java b/server/base/src/main/java/org/apache/accumulo/server/util/TablePropUtil.java
index b5fcee1..5e5ba22 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/util/TablePropUtil.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/util/TablePropUtil.java
@@ -19,8 +19,8 @@ package org.apache.accumulo.server.util;
 import static java.nio.charset.StandardCharsets.UTF_8;
 
 import org.apache.accumulo.core.Constants;
-import org.apache.accumulo.core.clientImpl.Table;
 import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.fate.zookeeper.ZooReaderWriter;
 import org.apache.accumulo.fate.zookeeper.ZooUtil.NodeExistsPolicy;
 import org.apache.accumulo.fate.zookeeper.ZooUtil.NodeMissingPolicy;
@@ -29,13 +29,13 @@ import org.apache.zookeeper.KeeperException;
 
 public class TablePropUtil {
 
-  public static boolean setTableProperty(ServerContext context, Table.ID tableId, String property,
+  public static boolean setTableProperty(ServerContext context, TableId tableId, String property,
       String value) throws KeeperException, InterruptedException {
     return setTableProperty(context.getZooReaderWriter(), context.getZooKeeperRoot(), tableId,
         property, value);
   }
 
-  public static boolean setTableProperty(ZooReaderWriter zoo, String zkRoot, Table.ID tableId,
+  public static boolean setTableProperty(ZooReaderWriter zoo, String zkRoot, TableId tableId,
       String property, String value) throws KeeperException, InterruptedException {
     if (!isPropertyValid(property, value))
       return false;
@@ -57,13 +57,13 @@ public class TablePropUtil {
         && Property.isValidTablePropertyKey(property);
   }
 
-  public static void removeTableProperty(ServerContext context, Table.ID tableId, String property)
+  public static void removeTableProperty(ServerContext context, TableId tableId, String property)
       throws InterruptedException, KeeperException {
     String zPath = getTablePath(context.getZooKeeperRoot(), tableId) + "/" + property;
     context.getZooReaderWriter().recursiveDelete(zPath, NodeMissingPolicy.SKIP);
   }
 
-  private static String getTablePath(String zkRoot, Table.ID tableId) {
-    return zkRoot + Constants.ZTABLES + "/" + tableId.canonicalID() + Constants.ZTABLE_CONF;
+  private static String getTablePath(String zkRoot, TableId tableId) {
+    return zkRoot + Constants.ZTABLES + "/" + tableId.canonical() + Constants.ZTABLE_CONF;
   }
 }
diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/VerifyTabletAssignments.java b/server/base/src/main/java/org/apache/accumulo/server/util/VerifyTabletAssignments.java
index 24e731c..4e0db56 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/util/VerifyTabletAssignments.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/util/VerifyTabletAssignments.java
@@ -32,10 +32,10 @@ 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.clientImpl.ClientContext;
-import org.apache.accumulo.core.clientImpl.Table;
 import org.apache.accumulo.core.clientImpl.Tables;
 import org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException;
 import org.apache.accumulo.core.data.Range;
+import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.dataImpl.KeyExtent;
 import org.apache.accumulo.core.dataImpl.thrift.InitialMultiScan;
 import org.apache.accumulo.core.dataImpl.thrift.IterInfo;
@@ -90,7 +90,7 @@ public class VerifyTabletAssignments {
 
     TreeMap<KeyExtent,String> tabletLocations = new TreeMap<>();
 
-    Table.ID tableId = Tables.getNameToIdMap(context).get(tableName);
+    TableId tableId = Tables.getNameToIdMap(context).get(tableName);
     MetadataServicer.forTableId(context, tableId).getTabletLocations(tabletLocations);
 
     final HashSet<KeyExtent> failures = new HashSet<>();
diff --git a/server/base/src/test/java/org/apache/accumulo/server/client/BulkImporterTest.java b/server/base/src/test/java/org/apache/accumulo/server/client/BulkImporterTest.java
index ddc4920..56c13a9 100644
--- a/server/base/src/test/java/org/apache/accumulo/server/client/BulkImporterTest.java
+++ b/server/base/src/test/java/org/apache/accumulo/server/client/BulkImporterTest.java
@@ -27,7 +27,6 @@ import java.util.SortedSet;
 import java.util.TreeSet;
 
 import org.apache.accumulo.core.clientImpl.ClientContext;
-import org.apache.accumulo.core.clientImpl.Table;
 import org.apache.accumulo.core.clientImpl.TabletLocator;
 import org.apache.accumulo.core.clientImpl.TabletLocator.TabletLocation;
 import org.apache.accumulo.core.conf.DefaultConfiguration;
@@ -35,6 +34,7 @@ import org.apache.accumulo.core.crypto.CryptoServiceFactory;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.data.Range;
+import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.dataImpl.KeyExtent;
 import org.apache.accumulo.core.file.FileOperations;
@@ -53,7 +53,7 @@ import org.junit.Test;
 public class BulkImporterTest {
 
   static final SortedSet<KeyExtent> fakeMetaData = new TreeSet<>();
-  static final Table.ID tableId = Table.ID.of("1");
+  static final TableId tableId = TableId.of("1");
 
   static {
     fakeMetaData.add(new KeyExtent(tableId, new Text("a"), null));
@@ -177,19 +177,19 @@ public class BulkImporterTest {
     // a correct startRow so that findOverlappingTablets works as intended.
 
     // 1;2;1
-    KeyExtent extent = new KeyExtent(Table.ID.of("1"), new Text("2"), new Text("1"));
+    KeyExtent extent = new KeyExtent(TableId.of("1"), new Text("2"), new Text("1"));
     assertEquals(new Text("1\0"), BulkImporter.getStartRowForExtent(extent));
 
     // 1;2<
-    extent = new KeyExtent(Table.ID.of("1"), new Text("2"), null);
+    extent = new KeyExtent(TableId.of("1"), new Text("2"), null);
     assertNull(BulkImporter.getStartRowForExtent(extent));
 
     // 1<<
-    extent = new KeyExtent(Table.ID.of("1"), null, null);
+    extent = new KeyExtent(TableId.of("1"), null, null);
     assertNull(BulkImporter.getStartRowForExtent(extent));
 
     // 1;8;7777777
-    extent = new KeyExtent(Table.ID.of("1"), new Text("8"), new Text("7777777"));
+    extent = new KeyExtent(TableId.of("1"), new Text("8"), new Text("7777777"));
     assertEquals(new Text("7777777\0"), BulkImporter.getStartRowForExtent(extent));
   }
 }
diff --git a/server/base/src/test/java/org/apache/accumulo/server/conf/NamespaceConfigurationTest.java b/server/base/src/test/java/org/apache/accumulo/server/conf/NamespaceConfigurationTest.java
index 1ce92c5..7f023e7 100644
--- a/server/base/src/test/java/org/apache/accumulo/server/conf/NamespaceConfigurationTest.java
+++ b/server/base/src/test/java/org/apache/accumulo/server/conf/NamespaceConfigurationTest.java
@@ -39,6 +39,7 @@ import org.apache.accumulo.core.clientImpl.Namespace;
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
 import org.apache.accumulo.core.conf.ConfigurationObserver;
 import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.core.data.NamespaceId;
 import org.apache.accumulo.fate.zookeeper.ZooCache;
 import org.apache.accumulo.fate.zookeeper.ZooCacheFactory;
 import org.apache.accumulo.fate.zookeeper.ZooUtil;
@@ -47,7 +48,7 @@ import org.junit.Before;
 import org.junit.Test;
 
 public class NamespaceConfigurationTest {
-  private static final Namespace.ID NSID = Namespace.ID.of("namespace");
+  private static final NamespaceId NSID = NamespaceId.of("namespace");
   private static final String ZOOKEEPERS = "localhost";
   private static final int ZK_SESSION_TIMEOUT = 120000;
 
@@ -110,10 +111,10 @@ public class NamespaceConfigurationTest {
 
   @Test
   public void testGet_SkipParentIfAccumuloNS() {
-    c = new NamespaceConfiguration(Namespace.ID.ACCUMULO, context, parent);
+    c = new NamespaceConfiguration(Namespace.ACCUMULO.id(), context, parent);
     c.setZooCacheFactory(zcf);
     Property p = Property.INSTANCE_SECRET;
-    expect(zc.get(ZooUtil.getRoot(iid) + Constants.ZNAMESPACES + "/" + Namespace.ID.ACCUMULO
+    expect(zc.get(ZooUtil.getRoot(iid) + Constants.ZNAMESPACES + "/" + Namespace.ACCUMULO.id()
         + Constants.ZNAMESPACE_CONF + "/" + p.getKey())).andReturn(null);
     replay(zc);
     assertNull(c.get(Property.INSTANCE_SECRET));
diff --git a/server/base/src/test/java/org/apache/accumulo/server/conf/ServerConfigurationFactoryTest.java b/server/base/src/test/java/org/apache/accumulo/server/conf/ServerConfigurationFactoryTest.java
index 83b708f..8f29544 100644
--- a/server/base/src/test/java/org/apache/accumulo/server/conf/ServerConfigurationFactoryTest.java
+++ b/server/base/src/test/java/org/apache/accumulo/server/conf/ServerConfigurationFactoryTest.java
@@ -30,10 +30,10 @@ import static org.junit.Assert.assertSame;
 
 import java.util.Properties;
 
-import org.apache.accumulo.core.clientImpl.Namespace;
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
 import org.apache.accumulo.core.conf.DefaultConfiguration;
 import org.apache.accumulo.core.conf.SiteConfiguration;
+import org.apache.accumulo.core.data.NamespaceId;
 import org.apache.accumulo.fate.zookeeper.ZooCache;
 import org.apache.accumulo.fate.zookeeper.ZooCacheFactory;
 import org.apache.accumulo.server.ServerContext;
@@ -119,7 +119,7 @@ public class ServerConfigurationFactoryTest {
     assertNotNull(c);
   }
 
-  private static final Namespace.ID NSID = Namespace.ID.of("NAMESPACE");
+  private static final NamespaceId NSID = NamespaceId.of("NAMESPACE");
 
   @Test
   public void testGetNamespaceConfiguration() {
diff --git a/server/base/src/test/java/org/apache/accumulo/server/conf/TableConfigurationTest.java b/server/base/src/test/java/org/apache/accumulo/server/conf/TableConfigurationTest.java
index c1eb338..af932aa 100644
--- a/server/base/src/test/java/org/apache/accumulo/server/conf/TableConfigurationTest.java
+++ b/server/base/src/test/java/org/apache/accumulo/server/conf/TableConfigurationTest.java
@@ -34,9 +34,9 @@ import java.util.UUID;
 import java.util.function.Predicate;
 
 import org.apache.accumulo.core.Constants;
-import org.apache.accumulo.core.clientImpl.Table;
 import org.apache.accumulo.core.conf.ConfigurationObserver;
 import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.fate.zookeeper.ZooCache;
 import org.apache.accumulo.fate.zookeeper.ZooCacheFactory;
 import org.apache.accumulo.fate.zookeeper.ZooUtil;
@@ -45,7 +45,7 @@ import org.junit.Before;
 import org.junit.Test;
 
 public class TableConfigurationTest {
-  private static final Table.ID TID = Table.ID.of("table");
+  private static final TableId TID = TableId.of("table");
   private static final String ZOOKEEPERS = "localhost";
   private static final int ZK_SESSION_TIMEOUT = 120000;
 
diff --git a/server/base/src/test/java/org/apache/accumulo/server/fs/PerTableVolumeChooserTest.java b/server/base/src/test/java/org/apache/accumulo/server/fs/PerTableVolumeChooserTest.java
index eaec3cb..cdf8926 100644
--- a/server/base/src/test/java/org/apache/accumulo/server/fs/PerTableVolumeChooserTest.java
+++ b/server/base/src/test/java/org/apache/accumulo/server/fs/PerTableVolumeChooserTest.java
@@ -24,8 +24,8 @@ import static org.easymock.EasyMock.verify;
 import static org.junit.Assert.assertSame;
 import static org.junit.Assert.fail;
 
-import org.apache.accumulo.core.clientImpl.Table;
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
+import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.server.conf.ServerConfigurationFactory;
 import org.apache.accumulo.server.conf.TableConfiguration;
 import org.apache.accumulo.server.fs.VolumeChooser.VolumeChooserException;
@@ -78,7 +78,7 @@ public class PerTableVolumeChooserTest {
   }
 
   private VolumeChooser getTableDelegate() {
-    VolumeChooserEnvironment env = new VolumeChooserEnvironment(Table.ID.of("testTable"), null,
+    VolumeChooserEnvironment env = new VolumeChooserEnvironment(TableId.of("testTable"), null,
         null);
     return chooser.getDelegateChooser(env);
   }
diff --git a/server/base/src/test/java/org/apache/accumulo/server/fs/PreferredVolumeChooserTest.java b/server/base/src/test/java/org/apache/accumulo/server/fs/PreferredVolumeChooserTest.java
index f1edb9c..18ad583 100644
--- a/server/base/src/test/java/org/apache/accumulo/server/fs/PreferredVolumeChooserTest.java
+++ b/server/base/src/test/java/org/apache/accumulo/server/fs/PreferredVolumeChooserTest.java
@@ -27,8 +27,8 @@ import static org.junit.Assert.fail;
 
 import java.util.Arrays;
 
-import org.apache.accumulo.core.clientImpl.Table;
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
+import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.server.conf.ServerConfigurationFactory;
 import org.apache.accumulo.server.conf.TableConfiguration;
 import org.apache.accumulo.server.fs.VolumeChooser.VolumeChooserException;
@@ -74,7 +74,7 @@ public class PreferredVolumeChooserTest {
   }
 
   private String[] chooseForTable() {
-    VolumeChooserEnvironment env = new VolumeChooserEnvironment(Table.ID.of("testTable"), null,
+    VolumeChooserEnvironment env = new VolumeChooserEnvironment(TableId.of("testTable"), null,
         null);
     return chooser.getPreferredVolumes(env, ALL_OPTIONS);
   }
diff --git a/server/base/src/test/java/org/apache/accumulo/server/fs/VolumeManagerImplTest.java b/server/base/src/test/java/org/apache/accumulo/server/fs/VolumeManagerImplTest.java
index a3d704d..429adad 100644
--- a/server/base/src/test/java/org/apache/accumulo/server/fs/VolumeManagerImplTest.java
+++ b/server/base/src/test/java/org/apache/accumulo/server/fs/VolumeManagerImplTest.java
@@ -22,9 +22,9 @@ import static org.junit.Assert.assertTrue;
 import java.util.Arrays;
 import java.util.List;
 
-import org.apache.accumulo.core.clientImpl.Table;
 import org.apache.accumulo.core.conf.ConfigurationCopy;
 import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.server.fs.VolumeManager.FileType;
 import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.conf.Configuration;
@@ -137,7 +137,7 @@ public class VolumeManagerImplTest {
     conf.set(Property.GENERAL_VOLUME_CHOOSER, WrongVolumeChooser.class.getName());
     thrown.expect(RuntimeException.class);
     VolumeManager vm = VolumeManagerImpl.get(conf, hadoopConf);
-    VolumeChooserEnvironment chooserEnv = new VolumeChooserEnvironment(Table.ID.of("sometable"),
+    VolumeChooserEnvironment chooserEnv = new VolumeChooserEnvironment(TableId.of("sometable"),
         null, null);
     String choice = vm.choose(chooserEnv, volumes.toArray(new String[0]));
     assertTrue("shouldn't see invalid options from misbehaving chooser.", volumes.contains(choice));
diff --git a/server/base/src/test/java/org/apache/accumulo/server/master/balancer/BaseHostRegexTableLoadBalancerTest.java b/server/base/src/test/java/org/apache/accumulo/server/master/balancer/BaseHostRegexTableLoadBalancerTest.java
index 3fadbe4..f53cd62 100644
--- a/server/base/src/test/java/org/apache/accumulo/server/master/balancer/BaseHostRegexTableLoadBalancerTest.java
+++ b/server/base/src/test/java/org/apache/accumulo/server/master/balancer/BaseHostRegexTableLoadBalancerTest.java
@@ -35,13 +35,13 @@ import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.admin.TableOperations;
 import org.apache.accumulo.core.clientImpl.ClientContext;
 import org.apache.accumulo.core.clientImpl.Namespace;
-import org.apache.accumulo.core.clientImpl.Table;
 import org.apache.accumulo.core.clientImpl.TableOperationsImpl;
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
 import org.apache.accumulo.core.conf.ConfigurationCopy;
 import org.apache.accumulo.core.conf.DefaultConfiguration;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.conf.SiteConfiguration;
+import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.dataImpl.KeyExtent;
 import org.apache.accumulo.core.master.thrift.TableInfo;
 import org.apache.accumulo.core.master.thrift.TabletServerStatus;
@@ -58,9 +58,9 @@ public abstract class BaseHostRegexTableLoadBalancerTest extends HostRegexTableL
 
   protected static class TestTable {
     private String tableName;
-    private Table.ID id;
+    private TableId id;
 
-    TestTable(String tableName, Table.ID id) {
+    TestTable(String tableName, TableId id) {
       this.tableName = tableName;
       this.id = id;
     }
@@ -69,7 +69,7 @@ public abstract class BaseHostRegexTableLoadBalancerTest extends HostRegexTableL
       return tableName;
     }
 
-    public Table.ID getId() {
+    public TableId getId() {
       return id;
     }
   }
@@ -106,10 +106,10 @@ public abstract class BaseHostRegexTableLoadBalancerTest extends HostRegexTableL
     }
 
     @Override
-    public TableConfiguration getTableConfiguration(final Table.ID tableId) {
+    public TableConfiguration getTableConfiguration(final TableId tableId) {
       // create a dummy namespaceConfiguration to satisfy requireNonNull in TableConfiguration
       // constructor
-      NamespaceConfiguration dummyConf = new NamespaceConfiguration(Namespace.ID.DEFAULT, context,
+      NamespaceConfiguration dummyConf = new NamespaceConfiguration(Namespace.DEFAULT.id(), context,
           DefaultConfiguration.getInstance());
       return new TableConfiguration(context, tableId, dummyConf) {
         @Override
@@ -134,13 +134,13 @@ public abstract class BaseHostRegexTableLoadBalancerTest extends HostRegexTableL
     }
   }
 
-  protected static final TestTable FOO = new TestTable("foo", Table.ID.of("1"));
-  protected static final TestTable BAR = new TestTable("bar", Table.ID.of("2"));
-  protected static final TestTable BAZ = new TestTable("baz", Table.ID.of("3"));
+  protected static final TestTable FOO = new TestTable("foo", TableId.of("1"));
+  protected static final TestTable BAR = new TestTable("bar", TableId.of("2"));
+  protected static final TestTable BAZ = new TestTable("baz", TableId.of("3"));
 
   protected class TestDefaultBalancer extends DefaultLoadBalancer {
     @Override
-    public List<TabletStats> getOnlineTabletsForTable(TServerInstance tserver, Table.ID tableId) {
+    public List<TabletStats> getOnlineTabletsForTable(TServerInstance tserver, TableId tableId) {
       String tableName = idToTableName(tableId);
       TServerInstance initialLocation = initialTableLocation.get(tableName);
       if (tserver.equals(initialLocation)) {
@@ -246,7 +246,7 @@ public abstract class BaseHostRegexTableLoadBalancerTest extends HostRegexTableL
   }
 
   protected boolean tabletInBounds(KeyExtent ke, TServerInstance tsi) {
-    String tid = ke.getTableId().canonicalID();
+    String tid = ke.getTableId().canonical();
     String host = tsi.host();
     if (tid.equals("1")
         && (host.equals("192.168.0.1") || host.equals("192.168.0.2") || host.equals("192.168.0.3")
@@ -262,7 +262,7 @@ public abstract class BaseHostRegexTableLoadBalancerTest extends HostRegexTableL
           || host.equals("192.168.0.15"));
   }
 
-  protected String idToTableName(Table.ID id) {
+  protected String idToTableName(TableId id) {
     if (id.equals(FOO.getId())) {
       return FOO.getTableName();
     } else if (id.equals(BAR.getId())) {
@@ -280,9 +280,9 @@ public abstract class BaseHostRegexTableLoadBalancerTest extends HostRegexTableL
       @Override
       public Map<String,String> tableIdMap() {
         HashMap<String,String> tables = new HashMap<>();
-        tables.put(FOO.getTableName(), FOO.getId().canonicalID());
-        tables.put(BAR.getTableName(), BAR.getId().canonicalID());
-        tables.put(BAZ.getTableName(), BAZ.getId().canonicalID());
+        tables.put(FOO.getTableName(), FOO.getId().canonical());
+        tables.put(BAR.getTableName(), BAR.getId().canonical());
+        tables.put(BAZ.getTableName(), BAZ.getId().canonical());
         return tables;
       }
 
@@ -298,7 +298,7 @@ public abstract class BaseHostRegexTableLoadBalancerTest extends HostRegexTableL
   }
 
   @Override
-  protected TabletBalancer getBalancerForTable(Table.ID table) {
+  protected TabletBalancer getBalancerForTable(TableId table) {
     return new TestDefaultBalancer();
   }
 
@@ -317,9 +317,9 @@ public abstract class BaseHostRegexTableLoadBalancerTest extends HostRegexTableL
     for (int i = 1; i <= numTservers; i++) {
       TabletServerStatus status = new TabletServerStatus();
       Map<String,TableInfo> tableMap = new HashMap<>();
-      tableMap.put(FOO.getId().canonicalID(), new TableInfo());
-      tableMap.put(BAR.getId().canonicalID(), new TableInfo());
-      tableMap.put(BAZ.getId().canonicalID(), new TableInfo());
+      tableMap.put(FOO.getId().canonical(), new TableInfo());
+      tableMap.put(BAR.getId().canonical(), new TableInfo());
+      tableMap.put(BAZ.getId().canonical(), new TableInfo());
       status.setTableMap(tableMap);
       current.put(new TServerInstance(base + i + ":9997", 1), status);
     }
diff --git a/server/base/src/test/java/org/apache/accumulo/server/master/balancer/ChaoticLoadBalancerTest.java b/server/base/src/test/java/org/apache/accumulo/server/master/balancer/ChaoticLoadBalancerTest.java
index ec1c6f8..70fcf40 100644
--- a/server/base/src/test/java/org/apache/accumulo/server/master/balancer/ChaoticLoadBalancerTest.java
+++ b/server/base/src/test/java/org/apache/accumulo/server/master/balancer/ChaoticLoadBalancerTest.java
@@ -28,7 +28,7 @@ import java.util.Set;
 import java.util.SortedMap;
 import java.util.TreeMap;
 
-import org.apache.accumulo.core.clientImpl.Table;
+import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.dataImpl.KeyExtent;
 import org.apache.accumulo.core.master.thrift.TableInfo;
 import org.apache.accumulo.core.master.thrift.TabletServerStatus;
@@ -48,10 +48,10 @@ public class ChaoticLoadBalancerTest {
       TabletServerStatus result = new TabletServerStatus();
       result.tableMap = new HashMap<>();
       for (KeyExtent extent : extents) {
-        Table.ID table = extent.getTableId();
-        TableInfo info = result.tableMap.get(table.canonicalID());
+        TableId table = extent.getTableId();
+        TableInfo info = result.tableMap.get(table.canonical());
         if (info == null)
-          result.tableMap.put(table.canonicalID(), info = new TableInfo());
+          result.tableMap.put(table.canonical(), info = new TableInfo());
         info.onlineTablets++;
         info.recs = info.onlineTablets;
         info.ingestRate = 123.;
@@ -66,7 +66,7 @@ public class ChaoticLoadBalancerTest {
   class TestChaoticLoadBalancer extends ChaoticLoadBalancer {
 
     @Override
-    public List<TabletStats> getOnlineTabletsForTable(TServerInstance tserver, Table.ID table) {
+    public List<TabletStats> getOnlineTabletsForTable(TServerInstance tserver, TableId table) {
       List<TabletStats> result = new ArrayList<>();
       for (KeyExtent extent : servers.get(tserver).extents) {
         if (extent.getTableId().equals(table)) {
@@ -156,7 +156,7 @@ public class ChaoticLoadBalancerTest {
   }
 
   private static KeyExtent makeExtent(String table, String end, String prev) {
-    return new KeyExtent(Table.ID.of(table), toText(end), toText(prev));
+    return new KeyExtent(TableId.of(table), toText(end), toText(prev));
   }
 
   private static Text toText(String value) {
diff --git a/server/base/src/test/java/org/apache/accumulo/server/master/balancer/DefaultLoadBalancerTest.java b/server/base/src/test/java/org/apache/accumulo/server/master/balancer/DefaultLoadBalancerTest.java
index 6f2119c..6ef9ead 100644
--- a/server/base/src/test/java/org/apache/accumulo/server/master/balancer/DefaultLoadBalancerTest.java
+++ b/server/base/src/test/java/org/apache/accumulo/server/master/balancer/DefaultLoadBalancerTest.java
@@ -31,7 +31,7 @@ import java.util.Set;
 import java.util.SortedMap;
 import java.util.TreeMap;
 
-import org.apache.accumulo.core.clientImpl.Table;
+import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.dataImpl.KeyExtent;
 import org.apache.accumulo.core.master.thrift.TableInfo;
 import org.apache.accumulo.core.master.thrift.TabletServerStatus;
@@ -52,10 +52,10 @@ public class DefaultLoadBalancerTest {
       TabletServerStatus result = new TabletServerStatus();
       result.tableMap = new HashMap<>();
       for (KeyExtent extent : extents) {
-        Table.ID tableId = extent.getTableId();
-        TableInfo info = result.tableMap.get(tableId.canonicalID());
+        TableId tableId = extent.getTableId();
+        TableInfo info = result.tableMap.get(tableId.canonical());
         if (info == null)
-          result.tableMap.put(tableId.canonicalID(), info = new TableInfo());
+          result.tableMap.put(tableId.canonical(), info = new TableInfo());
         info.onlineTablets++;
         info.recs = info.onlineTablets;
         info.ingestRate = 123.;
@@ -71,7 +71,7 @@ public class DefaultLoadBalancerTest {
   class TestDefaultLoadBalancer extends DefaultLoadBalancer {
 
     @Override
-    public List<TabletStats> getOnlineTabletsForTable(TServerInstance tserver, Table.ID table) {
+    public List<TabletStats> getOnlineTabletsForTable(TServerInstance tserver, TableId table) {
       List<TabletStats> result = new ArrayList<>();
       for (KeyExtent extent : servers.get(tserver).extents) {
         if (extent.getTableId().equals(table)) {
@@ -270,7 +270,7 @@ public class DefaultLoadBalancerTest {
       for (FakeTServer server : servers.values()) {
         Map<String,Integer> counts = new HashMap<>();
         for (KeyExtent extent : server.extents) {
-          String t = extent.getTableId().canonicalID();
+          String t = extent.getTableId().canonical();
           if (counts.get(t) == null)
             counts.put(t, 0);
           counts.put(t, counts.get(t) + 1);
@@ -283,7 +283,7 @@ public class DefaultLoadBalancerTest {
   }
 
   private static KeyExtent makeExtent(String table, String end, String prev) {
-    return new KeyExtent(Table.ID.of(table), toText(end), toText(prev));
+    return new KeyExtent(TableId.of(table), toText(end), toText(prev));
   }
 
   private static Text toText(String value) {
diff --git a/server/base/src/test/java/org/apache/accumulo/server/master/balancer/GroupBalancerTest.java b/server/base/src/test/java/org/apache/accumulo/server/master/balancer/GroupBalancerTest.java
index 3932404..8169d67 100644
--- a/server/base/src/test/java/org/apache/accumulo/server/master/balancer/GroupBalancerTest.java
+++ b/server/base/src/test/java/org/apache/accumulo/server/master/balancer/GroupBalancerTest.java
@@ -33,7 +33,7 @@ import java.util.SortedMap;
 import java.util.TreeMap;
 import java.util.function.Function;
 
-import org.apache.accumulo.core.clientImpl.Table;
+import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.dataImpl.KeyExtent;
 import org.apache.accumulo.core.master.thrift.TabletServerStatus;
 import org.apache.accumulo.core.util.MapCounter;
@@ -72,7 +72,7 @@ public class GroupBalancerTest {
 
     public void addTablet(String er, String location) {
       TServerInstance tsi = new TServerInstance(location, 6);
-      tabletLocs.put(new KeyExtent(Table.ID.of("b"), er == null ? null : new Text(er), null),
+      tabletLocs.put(new KeyExtent(TableId.of("b"), er == null ? null : new Text(er), null),
           new TServerInstance(location, 6));
       tservers.add(tsi);
     }
@@ -82,7 +82,7 @@ public class GroupBalancerTest {
     }
 
     public void balance(final int maxMigrations) {
-      GroupBalancer balancer = new GroupBalancer(Table.ID.of("1")) {
+      GroupBalancer balancer = new GroupBalancer(TableId.of("1")) {
 
         @Override
         protected Iterable<Pair<KeyExtent,Location>> getLocationProvider() {
diff --git a/server/base/src/test/java/org/apache/accumulo/server/master/balancer/HostRegexTableLoadBalancerReconfigurationTest.java b/server/base/src/test/java/org/apache/accumulo/server/master/balancer/HostRegexTableLoadBalancerReconfigurationTest.java
index 06c21fa..40a28c1 100644
--- a/server/base/src/test/java/org/apache/accumulo/server/master/balancer/HostRegexTableLoadBalancerReconfigurationTest.java
+++ b/server/base/src/test/java/org/apache/accumulo/server/master/balancer/HostRegexTableLoadBalancerReconfigurationTest.java
@@ -31,7 +31,7 @@ import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Set;
 
-import org.apache.accumulo.core.clientImpl.Table;
+import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.dataImpl.KeyExtent;
 import org.apache.accumulo.core.tabletserver.thrift.TabletStats;
 import org.apache.accumulo.fate.util.UtilWaitThread;
@@ -106,7 +106,7 @@ public class HostRegexTableLoadBalancerReconfigurationTest
   }
 
   @Override
-  public List<TabletStats> getOnlineTabletsForTable(TServerInstance tserver, Table.ID tableId) {
+  public List<TabletStats> getOnlineTabletsForTable(TServerInstance tserver, TableId tableId) {
     List<TabletStats> tablets = new ArrayList<>();
     // Report assignment information
     for (Entry<KeyExtent,TServerInstance> e : this.assignments.entrySet()) {
diff --git a/server/base/src/test/java/org/apache/accumulo/server/master/balancer/HostRegexTableLoadBalancerTest.java b/server/base/src/test/java/org/apache/accumulo/server/master/balancer/HostRegexTableLoadBalancerTest.java
index 776b39f..f91e0f1 100644
--- a/server/base/src/test/java/org/apache/accumulo/server/master/balancer/HostRegexTableLoadBalancerTest.java
+++ b/server/base/src/test/java/org/apache/accumulo/server/master/balancer/HostRegexTableLoadBalancerTest.java
@@ -16,6 +16,7 @@
  */
 package org.apache.accumulo.server.master.balancer;
 
+import static java.nio.charset.StandardCharsets.UTF_8;
 import static org.easymock.EasyMock.expect;
 import static org.easymock.EasyMock.replay;
 import static org.junit.Assert.assertEquals;
@@ -36,11 +37,11 @@ import java.util.function.Predicate;
 import java.util.regex.Pattern;
 
 import org.apache.accumulo.core.clientImpl.Namespace;
-import org.apache.accumulo.core.clientImpl.Table;
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
 import org.apache.accumulo.core.conf.ConfigurationCopy;
 import org.apache.accumulo.core.conf.DefaultConfiguration;
 import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.dataImpl.KeyExtent;
 import org.apache.accumulo.core.dataImpl.thrift.TKeyExtent;
 import org.apache.accumulo.core.master.thrift.TabletServerStatus;
@@ -83,7 +84,7 @@ public class HostRegexTableLoadBalancerTest extends BaseHostRegexTableLoadBalanc
     assertEquals(Pattern.compile("r01.*").pattern(), patterns.get(FOO.getTableName()).pattern());
     assertTrue(patterns.containsKey(BAR.getTableName()));
     assertEquals(Pattern.compile("r02.*").pattern(), patterns.get(BAR.getTableName()).pattern());
-    Map<Table.ID,String> tids = this.getTableIdToTableName();
+    Map<TableId,String> tids = this.getTableIdToTableName();
     assertEquals(3, tids.size());
     assertTrue(tids.containsKey(FOO.getId()));
     assertEquals(FOO.getTableName(), tids.get(FOO.getId()));
@@ -194,8 +195,8 @@ public class HostRegexTableLoadBalancerTest extends BaseHostRegexTableLoadBalanc
     initFactory(new TestServerConfigurationFactory(context) {
 
       @Override
-      public TableConfiguration getTableConfiguration(Table.ID tableId) {
-        NamespaceConfiguration defaultConf = new NamespaceConfiguration(Namespace.ID.DEFAULT,
+      public TableConfiguration getTableConfiguration(TableId tableId) {
+        NamespaceConfiguration defaultConf = new NamespaceConfiguration(Namespace.DEFAULT.id(),
             this.context, DefaultConfiguration.getInstance());
         return new TableConfiguration(this.context, tableId, defaultConf) {
           HashMap<String,String> tableProperties = new HashMap<>();
@@ -284,8 +285,8 @@ public class HostRegexTableLoadBalancerTest extends BaseHostRegexTableLoadBalanc
       }
 
       @Override
-      public TableConfiguration getTableConfiguration(Table.ID tableId) {
-        NamespaceConfiguration defaultConf = new NamespaceConfiguration(Namespace.ID.DEFAULT,
+      public TableConfiguration getTableConfiguration(TableId tableId) {
+        NamespaceConfiguration defaultConf = new NamespaceConfiguration(Namespace.DEFAULT.id(),
             this.context, DefaultConfiguration.getInstance());
         return new TableConfiguration(context, tableId, defaultConf) {
           HashMap<String,String> tableProperties = new HashMap<>();
@@ -511,13 +512,13 @@ public class HostRegexTableLoadBalancerTest extends BaseHostRegexTableLoadBalanc
   }
 
   @Override
-  public List<TabletStats> getOnlineTabletsForTable(TServerInstance tserver, Table.ID tableId) {
+  public List<TabletStats> getOnlineTabletsForTable(TServerInstance tserver, TableId tableId) {
     // Report incorrect information so that balance will create an assignment
     List<TabletStats> tablets = new ArrayList<>();
     if (tableId.equals(BAR.getId()) && tserver.host().equals("192.168.0.1")) {
       // Report that we have a bar tablet on this server
       TKeyExtent tke = new TKeyExtent();
-      tke.setTable(BAR.getId().getUtf8());
+      tke.setTable(BAR.getId().canonical().getBytes(UTF_8));
       tke.setEndRow("11".getBytes());
       tke.setPrevEndRow("10".getBytes());
       TabletStats ts = new TabletStats();
@@ -526,7 +527,7 @@ public class HostRegexTableLoadBalancerTest extends BaseHostRegexTableLoadBalanc
     } else if (tableId.equals(FOO.getId()) && tserver.host().equals("192.168.0.6")) {
       // Report that we have a foo tablet on this server
       TKeyExtent tke = new TKeyExtent();
-      tke.setTable(FOO.getId().getUtf8());
+      tke.setTable(FOO.getId().canonical().getBytes(UTF_8));
       tke.setEndRow("1".getBytes());
       tke.setPrevEndRow("0".getBytes());
       TabletStats ts = new TabletStats();
diff --git a/server/base/src/test/java/org/apache/accumulo/server/master/balancer/TableLoadBalancerTest.java b/server/base/src/test/java/org/apache/accumulo/server/master/balancer/TableLoadBalancerTest.java
index f0749ec..44829e3 100644
--- a/server/base/src/test/java/org/apache/accumulo/server/master/balancer/TableLoadBalancerTest.java
+++ b/server/base/src/test/java/org/apache/accumulo/server/master/balancer/TableLoadBalancerTest.java
@@ -31,10 +31,10 @@ import java.util.TreeMap;
 import java.util.UUID;
 
 import org.apache.accumulo.core.client.admin.TableOperations;
-import org.apache.accumulo.core.clientImpl.Table;
 import org.apache.accumulo.core.conf.DefaultConfiguration;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.conf.SiteConfiguration;
+import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.dataImpl.KeyExtent;
 import org.apache.accumulo.core.master.thrift.TableInfo;
 import org.apache.accumulo.core.master.thrift.TabletServerStatus;
@@ -81,11 +81,11 @@ public class TableLoadBalancerTest {
 
   static SortedMap<TServerInstance,TabletServerStatus> state;
 
-  static List<TabletStats> generateFakeTablets(TServerInstance tserver, Table.ID tableId) {
+  static List<TabletStats> generateFakeTablets(TServerInstance tserver, TableId tableId) {
     List<TabletStats> result = new ArrayList<>();
     TabletServerStatus tableInfo = state.get(tserver);
     // generate some fake tablets
-    for (int i = 0; i < tableInfo.tableMap.get(tableId.canonicalID()).onlineTablets; i++) {
+    for (int i = 0; i < tableInfo.tableMap.get(tableId.canonical()).onlineTablets; i++) {
       TabletStats stats = new TabletStats();
       stats.extent = new KeyExtent(tableId, new Text(tserver.host() + String.format("%03d", i + 1)),
           new Text(tserver.host() + String.format("%03d", i))).toThrift();
@@ -97,7 +97,7 @@ public class TableLoadBalancerTest {
   static class DefaultLoadBalancer
       extends org.apache.accumulo.server.master.balancer.DefaultLoadBalancer {
 
-    public DefaultLoadBalancer(Table.ID table) {
+    public DefaultLoadBalancer(TableId table) {
       super(table);
     }
 
@@ -105,7 +105,7 @@ public class TableLoadBalancerTest {
     public void init(ServerContext context) {}
 
     @Override
-    public List<TabletStats> getOnlineTabletsForTable(TServerInstance tserver, Table.ID tableId) {
+    public List<TabletStats> getOnlineTabletsForTable(TServerInstance tserver, TableId tableId) {
       return generateFakeTablets(tserver, tableId);
     }
   }
@@ -115,13 +115,13 @@ public class TableLoadBalancerTest {
 
     // use our new classname to test class loading
     @Override
-    protected String getLoadBalancerClassNameForTable(Table.ID table) {
+    protected String getLoadBalancerClassNameForTable(TableId table) {
       return DefaultLoadBalancer.class.getName();
     }
 
     // we don't have real tablet servers to ask: invent some online tablets
     @Override
-    public List<TabletStats> getOnlineTabletsForTable(TServerInstance tserver, Table.ID tableId) {
+    public List<TabletStats> getOnlineTabletsForTable(TServerInstance tserver, TableId tableId) {
       return generateFakeTablets(tserver, tableId);
     }
 
@@ -153,7 +153,7 @@ public class TableLoadBalancerTest {
     ServerConfigurationFactory confFactory = new ServerConfigurationFactory(context,
         new SiteConfiguration()) {
       @Override
-      public TableConfiguration getTableConfiguration(Table.ID tableId) {
+      public TableConfiguration getTableConfiguration(TableId tableId) {
         // create a dummy namespaceConfiguration to satisfy requireNonNull in TableConfiguration
         // constructor
         NamespaceConfiguration dummyConf = new NamespaceConfiguration(null, context, null);
@@ -189,14 +189,14 @@ public class TableLoadBalancerTest {
     tls.init(context2);
     tls.balance(state, migrations, migrationsOut);
     int count = 0;
-    Map<Table.ID,Integer> movedByTable = new HashMap<>();
-    movedByTable.put(Table.ID.of(t1Id), 0);
-    movedByTable.put(Table.ID.of(t2Id), 0);
-    movedByTable.put(Table.ID.of(t3Id), 0);
+    Map<TableId,Integer> movedByTable = new HashMap<>();
+    movedByTable.put(TableId.of(t1Id), 0);
+    movedByTable.put(TableId.of(t2Id), 0);
+    movedByTable.put(TableId.of(t3Id), 0);
     for (TabletMigration migration : migrationsOut) {
       if (migration.oldServer.equals(svr))
         count++;
-      Table.ID key = migration.tablet.getTableId();
+      TableId key = migration.tablet.getTableId();
       movedByTable.put(key, movedByTable.get(key) + 1);
     }
     assertEquals(15, count);
diff --git a/server/base/src/test/java/org/apache/accumulo/server/master/state/MergeInfoTest.java b/server/base/src/test/java/org/apache/accumulo/server/master/state/MergeInfoTest.java
index b702c45..1f26537 100644
--- a/server/base/src/test/java/org/apache/accumulo/server/master/state/MergeInfoTest.java
+++ b/server/base/src/test/java/org/apache/accumulo/server/master/state/MergeInfoTest.java
@@ -31,7 +31,7 @@ import java.io.ByteArrayOutputStream;
 import java.io.DataInputStream;
 import java.io.DataOutputStream;
 
-import org.apache.accumulo.core.clientImpl.Table;
+import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.dataImpl.KeyExtent;
 import org.apache.hadoop.io.DataInputBuffer;
 import org.apache.hadoop.io.DataOutputBuffer;
@@ -72,7 +72,7 @@ public class MergeInfoTest {
     String table = "table";
     Text endRow = new Text("end");
     Text prevEndRow = new Text("begin");
-    keyExtent = new KeyExtent(Table.ID.of(table), endRow, prevEndRow);
+    keyExtent = new KeyExtent(TableId.of(table), endRow, prevEndRow);
     mi = new MergeInfo(keyExtent, MergeInfo.Operation.DELETE);
     mi.setState(MergeState.STARTED);
     ByteArrayOutputStream baos = new ByteArrayOutputStream();
@@ -89,10 +89,10 @@ public class MergeInfoTest {
 
   @Test
   public void testNeedsToBeChopped_DifferentTables() {
-    expect(keyExtent.getTableId()).andReturn(Table.ID.of("table1"));
+    expect(keyExtent.getTableId()).andReturn(TableId.of("table1"));
     replay(keyExtent);
     KeyExtent keyExtent2 = createMock(KeyExtent.class);
-    expect(keyExtent2.getTableId()).andReturn(Table.ID.of("table2"));
+    expect(keyExtent2.getTableId()).andReturn(TableId.of("table2"));
     replay(keyExtent2);
     mi = new MergeInfo(keyExtent, MergeInfo.Operation.MERGE);
     assertFalse(mi.needsToBeChopped(keyExtent2));
@@ -100,9 +100,9 @@ public class MergeInfoTest {
 
   @Test
   public void testNeedsToBeChopped_NotDelete() {
-    expect(keyExtent.getTableId()).andReturn(Table.ID.of("table1"));
+    expect(keyExtent.getTableId()).andReturn(TableId.of("table1"));
     KeyExtent keyExtent2 = createMock(KeyExtent.class);
-    expect(keyExtent2.getTableId()).andReturn(Table.ID.of("table1"));
+    expect(keyExtent2.getTableId()).andReturn(TableId.of("table1"));
     replay(keyExtent2);
     expect(keyExtent.overlaps(keyExtent2)).andReturn(true);
     replay(keyExtent);
@@ -126,11 +126,11 @@ public class MergeInfoTest {
   }
 
   private void testNeedsToBeChopped_Delete(String prevEndRow, boolean expected) {
-    expect(keyExtent.getTableId()).andReturn(Table.ID.of("table1"));
+    expect(keyExtent.getTableId()).andReturn(TableId.of("table1"));
     expect(keyExtent.getEndRow()).andReturn(new Text("prev"));
     replay(keyExtent);
     KeyExtent keyExtent2 = createMock(KeyExtent.class);
-    expect(keyExtent2.getTableId()).andReturn(Table.ID.of("table1"));
+    expect(keyExtent2.getTableId()).andReturn(TableId.of("table1"));
     expect(keyExtent2.getPrevEndRow()).andReturn(prevEndRow != null ? new Text(prevEndRow) : null);
     expectLastCall().anyTimes();
     replay(keyExtent2);
@@ -151,9 +151,9 @@ public class MergeInfoTest {
   public void testOverlaps_DoesNotNeedChopping() {
     KeyExtent keyExtent2 = createMock(KeyExtent.class);
     expect(keyExtent.overlaps(keyExtent2)).andReturn(false);
-    expect(keyExtent.getTableId()).andReturn(Table.ID.of("table1"));
+    expect(keyExtent.getTableId()).andReturn(TableId.of("table1"));
     replay(keyExtent);
-    expect(keyExtent2.getTableId()).andReturn(Table.ID.of("table2"));
+    expect(keyExtent2.getTableId()).andReturn(TableId.of("table2"));
     replay(keyExtent2);
     mi = new MergeInfo(keyExtent, MergeInfo.Operation.MERGE);
     assertFalse(mi.overlaps(keyExtent2));
@@ -163,10 +163,10 @@ public class MergeInfoTest {
   public void testOverlaps_NeedsChopping() {
     KeyExtent keyExtent2 = createMock(KeyExtent.class);
     expect(keyExtent.overlaps(keyExtent2)).andReturn(false);
-    expect(keyExtent.getTableId()).andReturn(Table.ID.of("table1"));
+    expect(keyExtent.getTableId()).andReturn(TableId.of("table1"));
     expect(keyExtent.getEndRow()).andReturn(new Text("prev"));
     replay(keyExtent);
-    expect(keyExtent2.getTableId()).andReturn(Table.ID.of("table1"));
+    expect(keyExtent2.getTableId()).andReturn(TableId.of("table1"));
     expect(keyExtent2.getPrevEndRow()).andReturn(new Text("prev"));
     expectLastCall().anyTimes();
     replay(keyExtent2);
@@ -188,7 +188,7 @@ public class MergeInfoTest {
   }
 
   private static KeyExtent ke(String tableId, String endRow, String prevEndRow) {
-    return new KeyExtent(Table.ID.of(tableId), endRow == null ? null : new Text(endRow),
+    return new KeyExtent(TableId.of(tableId), endRow == null ? null : new Text(endRow),
         prevEndRow == null ? null : new Text(prevEndRow));
   }
 
diff --git a/server/base/src/test/java/org/apache/accumulo/server/problems/ProblemReportTest.java b/server/base/src/test/java/org/apache/accumulo/server/problems/ProblemReportTest.java
index 8808567..6bef497 100644
--- a/server/base/src/test/java/org/apache/accumulo/server/problems/ProblemReportTest.java
+++ b/server/base/src/test/java/org/apache/accumulo/server/problems/ProblemReportTest.java
@@ -32,7 +32,8 @@ import java.io.ByteArrayOutputStream;
 import java.io.DataOutputStream;
 
 import org.apache.accumulo.core.Constants;
-import org.apache.accumulo.core.clientImpl.Table;
+import org.apache.accumulo.core.data.TableId;
+import org.apache.accumulo.core.metadata.MetadataTable;
 import org.apache.accumulo.core.util.Encoding;
 import org.apache.accumulo.fate.zookeeper.ZooReaderWriter;
 import org.apache.accumulo.fate.zookeeper.ZooUtil;
@@ -44,7 +45,7 @@ import org.junit.Before;
 import org.junit.Test;
 
 public class ProblemReportTest {
-  private static final Table.ID TABLE_ID = Table.ID.of("table");
+  private static final TableId TABLE_ID = TableId.of("table");
   private static final String RESOURCE = "resource";
   private static final String SERVER = "server";
 
@@ -87,8 +88,8 @@ public class ProblemReportTest {
     ProblemReport r2 = new ProblemReport(TABLE_ID, ProblemType.FILE_READ, RESOURCE, SERVER, null);
     assertEquals(r, r2);
     assertEquals(r2, r);
-    ProblemReport rx1 = new ProblemReport(Table.ID.METADATA, ProblemType.FILE_READ, RESOURCE,
-        SERVER, null);
+    ProblemReport rx1 = new ProblemReport(MetadataTable.ID, ProblemType.FILE_READ, RESOURCE, SERVER,
+        null);
     assertNotEquals(r, rx1);
     ProblemReport rx2 = new ProblemReport(TABLE_ID, ProblemType.FILE_WRITE, RESOURCE, SERVER, null);
     assertNotEquals(r, rx2);
@@ -122,11 +123,11 @@ public class ProblemReportTest {
     assertEquals(r.hashCode(), re2.hashCode());
   }
 
-  private byte[] makeZPathFileName(Table.ID table, ProblemType problemType, String resource)
+  private byte[] makeZPathFileName(TableId table, ProblemType problemType, String resource)
       throws Exception {
     ByteArrayOutputStream baos = new ByteArrayOutputStream();
     DataOutputStream dos = new DataOutputStream(baos);
-    dos.writeUTF(table.canonicalID());
+    dos.writeUTF(table.canonical());
     dos.writeUTF(problemType.name());
     dos.writeUTF(resource);
     dos.close();
diff --git a/server/base/src/test/java/org/apache/accumulo/server/problems/ProblemReportingIteratorTest.java b/server/base/src/test/java/org/apache/accumulo/server/problems/ProblemReportingIteratorTest.java
index ff84d5a..e0adfc7 100644
--- a/server/base/src/test/java/org/apache/accumulo/server/problems/ProblemReportingIteratorTest.java
+++ b/server/base/src/test/java/org/apache/accumulo/server/problems/ProblemReportingIteratorTest.java
@@ -27,10 +27,10 @@ import static org.junit.Assert.assertTrue;
 import java.util.Collection;
 import java.util.concurrent.atomic.AtomicBoolean;
 
-import org.apache.accumulo.core.clientImpl.Table;
 import org.apache.accumulo.core.data.ByteSequence;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Range;
+import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.iterators.system.InterruptibleIterator;
 import org.easymock.EasyMock;
@@ -38,7 +38,7 @@ import org.junit.Before;
 import org.junit.Test;
 
 public class ProblemReportingIteratorTest {
-  private static final Table.ID TABLE_ID = Table.ID.of("table");
+  private static final TableId TABLE_ID = TableId.of("table");
   private static final String RESOURCE = "resource";
 
   private InterruptibleIterator ii;
diff --git a/server/base/src/test/java/org/apache/accumulo/server/util/ReplicationTableUtilTest.java b/server/base/src/test/java/org/apache/accumulo/server/util/ReplicationTableUtilTest.java
index d92251f..3d2ab17 100644
--- a/server/base/src/test/java/org/apache/accumulo/server/util/ReplicationTableUtilTest.java
+++ b/server/base/src/test/java/org/apache/accumulo/server/util/ReplicationTableUtilTest.java
@@ -39,11 +39,11 @@ import org.apache.accumulo.core.client.admin.TableOperations;
 import org.apache.accumulo.core.client.security.tokens.PasswordToken;
 import org.apache.accumulo.core.clientImpl.ClientContext;
 import org.apache.accumulo.core.clientImpl.Credentials;
-import org.apache.accumulo.core.clientImpl.Table;
 import org.apache.accumulo.core.clientImpl.Writer;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.data.ColumnUpdate;
 import org.apache.accumulo.core.data.Mutation;
+import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.dataImpl.KeyExtent;
 import org.apache.accumulo.core.iterators.Combiner;
@@ -88,7 +88,7 @@ public class ReplicationTableUtilTest {
     String myFile = "file:////home/user/accumulo/wal/server+port/" + uuid;
 
     long createdTime = System.currentTimeMillis();
-    ReplicationTableUtil.updateFiles(context, new KeyExtent(Table.ID.of("1"), null, null), myFile,
+    ReplicationTableUtil.updateFiles(context, new KeyExtent(TableId.of("1"), null, null), myFile,
         StatusUtil.fileCreated(createdTime));
 
     verify(writer);
@@ -116,7 +116,7 @@ public class ReplicationTableUtilTest {
     String file = "file:///accumulo/wal/127.0.0.1+9997" + UUID.randomUUID();
     Path filePath = new Path(file);
     Text row = new Text(filePath.toString());
-    KeyExtent extent = new KeyExtent(Table.ID.of("1"), new Text("b"), new Text("a"));
+    KeyExtent extent = new KeyExtent(TableId.of("1"), new Text("b"), new Text("a"));
 
     Mutation m = ReplicationTableUtil.createUpdateMutation(filePath, ProtobufUtil.toValue(stat),
         extent);
@@ -127,7 +127,7 @@ public class ReplicationTableUtilTest {
     ColumnUpdate col = m.getUpdates().get(0);
 
     assertEquals(MetadataSchema.ReplicationSection.COLF, new Text(col.getColumnFamily()));
-    assertEquals(extent.getTableId().canonicalID(), new Text(col.getColumnQualifier()).toString());
+    assertEquals(extent.getTableId().canonical(), new Text(col.getColumnQualifier()).toString());
     assertEquals(0, col.getColumnVisibility().length);
     assertArrayEquals(stat.toByteArray(), col.getValue());
   }
diff --git a/server/gc/src/main/java/org/apache/accumulo/gc/GarbageCollectionAlgorithm.java b/server/gc/src/main/java/org/apache/accumulo/gc/GarbageCollectionAlgorithm.java
index d193840..fdc1bbc 100644
--- a/server/gc/src/main/java/org/apache/accumulo/gc/GarbageCollectionAlgorithm.java
+++ b/server/gc/src/main/java/org/apache/accumulo/gc/GarbageCollectionAlgorithm.java
@@ -30,7 +30,7 @@ import java.util.TreeMap;
 
 import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.TableNotFoundException;
-import org.apache.accumulo.core.clientImpl.Table;
+import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.trace.Span;
 import org.apache.accumulo.core.trace.Trace;
 import org.apache.accumulo.gc.GarbageCollectionEnvironment.Reference;
@@ -240,25 +240,25 @@ public class GarbageCollectionAlgorithm {
 
   private void cleanUpDeletedTableDirs(GarbageCollectionEnvironment gce,
       SortedMap<String,String> candidateMap) throws IOException {
-    HashSet<Table.ID> tableIdsWithDeletes = new HashSet<>();
+    HashSet<TableId> tableIdsWithDeletes = new HashSet<>();
 
     // find the table ids that had dirs deleted
     for (String delete : candidateMap.keySet()) {
       String[] tokens = delete.split("/");
       if (tokens.length == 2) {
         // its a directory
-        Table.ID tableId = Table.ID.of(delete.split("/")[0]);
+        TableId tableId = TableId.of(delete.split("/")[0]);
         tableIdsWithDeletes.add(tableId);
       }
     }
 
-    Set<Table.ID> tableIdsInZookeeper = gce.getTableIDs();
+    Set<TableId> tableIdsInZookeeper = gce.getTableIDs();
 
     tableIdsWithDeletes.removeAll(tableIdsInZookeeper);
 
     // tableIdsWithDeletes should now contain the set of deleted tables that had dirs deleted
 
-    for (Table.ID delTableId : tableIdsWithDeletes) {
+    for (TableId delTableId : tableIdsWithDeletes) {
       gce.deleteTableDirIfEmpty(delTableId);
     }
 
diff --git a/server/gc/src/main/java/org/apache/accumulo/gc/GarbageCollectionEnvironment.java b/server/gc/src/main/java/org/apache/accumulo/gc/GarbageCollectionEnvironment.java
index 4f59a60..0903b28 100644
--- a/server/gc/src/main/java/org/apache/accumulo/gc/GarbageCollectionEnvironment.java
+++ b/server/gc/src/main/java/org/apache/accumulo/gc/GarbageCollectionEnvironment.java
@@ -26,9 +26,8 @@ import java.util.SortedMap;
 import java.util.stream.Stream;
 
 import org.apache.accumulo.core.client.TableNotFoundException;
-import org.apache.accumulo.core.clientImpl.Table;
-import org.apache.accumulo.core.clientImpl.Table.ID;
 import org.apache.accumulo.core.data.Key;
+import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.metadata.MetadataTable;
 import org.apache.accumulo.core.metadata.RootTable;
@@ -62,11 +61,11 @@ public interface GarbageCollectionEnvironment {
   Iterator<String> getBlipIterator() throws TableNotFoundException;
 
   static class Reference {
-    public final ID id;
+    public final TableId id;
     public final String ref;
     public final boolean isDir;
 
-    Reference(ID id, String ref, boolean isDir) {
+    Reference(TableId id, String ref, boolean isDir) {
       this.id = id;
       this.ref = ref;
       this.isDir = isDir;
@@ -87,7 +86,7 @@ public interface GarbageCollectionEnvironment {
    *
    * @return The valueSet for the table name to table id map.
    */
-  Set<Table.ID> getTableIDs();
+  Set<TableId> getTableIDs();
 
   /**
    * Delete the given files from the provided {@link Map} of relative path to absolute path for each
@@ -104,7 +103,7 @@ public interface GarbageCollectionEnvironment {
    * @param tableID
    *          The id of the table whose directory we are to operate on
    */
-  void deleteTableDirIfEmpty(Table.ID tableID) throws IOException;
+  void deleteTableDirIfEmpty(TableId tableID) throws IOException;
 
   /**
    * Increment the number of candidates for deletion for the current garbage collection run
diff --git a/server/gc/src/main/java/org/apache/accumulo/gc/SimpleGarbageCollector.java b/server/gc/src/main/java/org/apache/accumulo/gc/SimpleGarbageCollector.java
index 6bff46b..41f2540 100644
--- a/server/gc/src/main/java/org/apache/accumulo/gc/SimpleGarbageCollector.java
+++ b/server/gc/src/main/java/org/apache/accumulo/gc/SimpleGarbageCollector.java
@@ -39,7 +39,6 @@ import org.apache.accumulo.core.client.IsolatedScanner;
 import org.apache.accumulo.core.client.MutationsRejectedException;
 import org.apache.accumulo.core.client.Scanner;
 import org.apache.accumulo.core.client.TableNotFoundException;
-import org.apache.accumulo.core.clientImpl.Table;
 import org.apache.accumulo.core.clientImpl.Tables;
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
 import org.apache.accumulo.core.conf.Property;
@@ -47,6 +46,7 @@ import org.apache.accumulo.core.data.Key;
 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.TableId;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.gc.thrift.GCMonitorService.Iface;
 import org.apache.accumulo.core.gc.thrift.GCMonitorService.Processor;
@@ -291,7 +291,7 @@ public class SimpleGarbageCollector implements Iface {
     }
 
     @Override
-    public Set<Table.ID> getTableIDs() {
+    public Set<TableId> getTableIDs() {
       return Tables.getIdToNameMap(context).keySet();
     }
 
@@ -397,7 +397,7 @@ public class SimpleGarbageCollector implements Iface {
                 }
                 String parts[] = fullPath.toString().split(Constants.ZTABLES)[1].split("/");
                 if (parts.length > 2) {
-                  Table.ID tableId = Table.ID.of(parts[1]);
+                  TableId tableId = TableId.of(parts[1]);
                   String tabletDir = parts[2];
                   context.getTableManager().updateTableStateCache(tableId);
                   TableState tableState = context.getTableManager().getTableState(tableId);
@@ -445,7 +445,7 @@ public class SimpleGarbageCollector implements Iface {
     }
 
     @Override
-    public void deleteTableDirIfEmpty(Table.ID tableID) throws IOException {
+    public void deleteTableDirIfEmpty(TableId tableID) throws IOException {
       // if dir exist and is empty, then empty list is returned...
       // hadoop 2.0 will throw an exception if the file does not exist
       for (String dir : ServerConstants.getTablesDirs(context)) {
diff --git a/server/gc/src/test/java/org/apache/accumulo/gc/GarbageCollectionTest.java b/server/gc/src/test/java/org/apache/accumulo/gc/GarbageCollectionTest.java
index 5e1c17c..bb39719 100644
--- a/server/gc/src/test/java/org/apache/accumulo/gc/GarbageCollectionTest.java
+++ b/server/gc/src/test/java/org/apache/accumulo/gc/GarbageCollectionTest.java
@@ -32,7 +32,7 @@ import java.util.TreeMap;
 import java.util.TreeSet;
 import java.util.stream.Stream;
 
-import org.apache.accumulo.core.clientImpl.Table;
+import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.server.replication.StatusUtil;
 import org.apache.accumulo.server.replication.proto.Replication.Status;
 import org.junit.Test;
@@ -42,10 +42,10 @@ public class GarbageCollectionTest {
     TreeSet<String> candidates = new TreeSet<>();
     ArrayList<String> blips = new ArrayList<>();
     Map<String,Reference> references = new TreeMap<>();
-    HashSet<Table.ID> tableIds = new HashSet<>();
+    HashSet<TableId> tableIds = new HashSet<>();
 
     ArrayList<String> deletes = new ArrayList<>();
-    ArrayList<Table.ID> tablesDirsToDelete = new ArrayList<>();
+    ArrayList<TableId> tablesDirsToDelete = new ArrayList<>();
     TreeMap<String,Status> filesToReplicate = new TreeMap<>();
 
     @Override
@@ -69,7 +69,7 @@ public class GarbageCollectionTest {
     }
 
     @Override
-    public Set<Table.ID> getTableIDs() {
+    public Set<TableId> getTableIDs() {
       return tableIds;
     }
 
@@ -80,13 +80,13 @@ public class GarbageCollectionTest {
     }
 
     @Override
-    public void deleteTableDirIfEmpty(Table.ID tableID) {
+    public void deleteTableDirIfEmpty(TableId tableID) {
       tablesDirsToDelete.add(tableID);
     }
 
     public void addFileReference(String tableId, String endRow, String file) {
       references.put(tableId + ":" + endRow + ":" + file,
-          new Reference(Table.ID.of(tableId), file, false));
+          new Reference(TableId.of(tableId), file, false));
     }
 
     public void removeFileReference(String tableId, String endRow, String file) {
@@ -94,7 +94,7 @@ public class GarbageCollectionTest {
     }
 
     public void addDirReference(String tableId, String endRow, String dir) {
-      references.put(tableId + ":" + endRow, new Reference(Table.ID.of(tableId), dir, true));
+      references.put(tableId + ":" + endRow, new Reference(TableId.of(tableId), dir, true));
     }
 
     public void removeDirReference(String tableId, String endRow) {
@@ -512,7 +512,7 @@ public class GarbageCollectionTest {
 
     TestGCE gce = new TestGCE();
 
-    gce.tableIds.add(Table.ID.of("4"));
+    gce.tableIds.add(TableId.of("4"));
 
     gce.candidates.add("/4/t-0");
     gce.candidates.add("/4/t-0/F002.rf");
@@ -524,9 +524,9 @@ public class GarbageCollectionTest {
 
     gca.collect(gce);
 
-    HashSet<Table.ID> tids = new HashSet<>();
-    tids.add(Table.ID.of("5"));
-    tids.add(Table.ID.of("6"));
+    HashSet<TableId> tids = new HashSet<>();
+    tids.add(TableId.of("5"));
+    tids.add(TableId.of("6"));
 
     assertEquals(tids.size(), gce.tablesDirsToDelete.size());
     assertTrue(tids.containsAll(gce.tablesDirsToDelete));
diff --git a/server/master/src/main/java/org/apache/accumulo/master/FateServiceHandler.java b/server/master/src/main/java/org/apache/accumulo/master/FateServiceHandler.java
index 1857fa3..b535b5b 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/FateServiceHandler.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/FateServiceHandler.java
@@ -40,9 +40,7 @@ import org.apache.accumulo.core.client.admin.CompactionStrategyConfig;
 import org.apache.accumulo.core.client.admin.InitialTableState;
 import org.apache.accumulo.core.client.admin.TimeType;
 import org.apache.accumulo.core.clientImpl.CompactionStrategyConfigUtil;
-import org.apache.accumulo.core.clientImpl.Namespace;
 import org.apache.accumulo.core.clientImpl.Namespaces;
-import org.apache.accumulo.core.clientImpl.Table;
 import org.apache.accumulo.core.clientImpl.TableOperationsImpl;
 import org.apache.accumulo.core.clientImpl.Tables;
 import org.apache.accumulo.core.clientImpl.thrift.SecurityErrorCode;
@@ -50,6 +48,8 @@ import org.apache.accumulo.core.clientImpl.thrift.TableOperation;
 import org.apache.accumulo.core.clientImpl.thrift.TableOperationExceptionType;
 import org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException;
 import org.apache.accumulo.core.clientImpl.thrift.ThriftTableOperationException;
+import org.apache.accumulo.core.data.NamespaceId;
+import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.iterators.IteratorUtil;
 import org.apache.accumulo.core.master.thrift.BulkImportState;
 import org.apache.accumulo.core.master.thrift.FateOperation;
@@ -125,7 +125,7 @@ class FateServiceHandler implements FateService.Iface {
             Namespaces.NOT_DEFAULT.and(Namespaces.NOT_ACCUMULO));
         String newName = validateNamespaceArgument(arguments.get(1), tableOp, null);
 
-        Namespace.ID namespaceId = ClientServiceHandler.checkNamespaceId(master.getContext(),
+        NamespaceId namespaceId = ClientServiceHandler.checkNamespaceId(master.getContext(),
             oldName, tableOp);
         if (!master.security.canRenameNamespace(c, namespaceId))
           throw new ThriftSecurityException(c.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
@@ -139,7 +139,7 @@ class FateServiceHandler implements FateService.Iface {
         String namespace = validateNamespaceArgument(arguments.get(0), tableOp,
             Namespaces.NOT_DEFAULT.and(Namespaces.NOT_ACCUMULO));
 
-        Namespace.ID namespaceId = ClientServiceHandler.checkNamespaceId(master.getContext(),
+        NamespaceId namespaceId = ClientServiceHandler.checkNamespaceId(master.getContext(),
             namespace, tableOp);
         if (!master.security.canDeleteNamespace(c, namespaceId))
           throw new ThriftSecurityException(c.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
@@ -168,7 +168,7 @@ class FateServiceHandler implements FateService.Iface {
                 "Exception thrown while writing splits to file system");
           }
         }
-        Namespace.ID namespaceId;
+        NamespaceId namespaceId;
 
         try {
           namespaceId = Namespaces.getNamespaceId(master.getContext(),
@@ -210,9 +210,9 @@ class FateServiceHandler implements FateService.Iface {
 
             });
 
-        Table.ID tableId = ClientServiceHandler.checkTableId(master.getContext(), oldTableName,
+        TableId tableId = ClientServiceHandler.checkTableId(master.getContext(), oldTableName,
             tableOp);
-        Namespace.ID namespaceId = getNamespaceIdFromTableId(tableOp, tableId);
+        NamespaceId namespaceId = getNamespaceIdFromTableId(tableOp, tableId);
 
         final boolean canRename;
         try {
@@ -239,9 +239,9 @@ class FateServiceHandler implements FateService.Iface {
       }
       case TABLE_CLONE: {
         TableOperation tableOp = TableOperation.CLONE;
-        Table.ID srcTableId = validateTableIdArgument(arguments.get(0), tableOp, NOT_ROOT_ID);
+        TableId srcTableId = validateTableIdArgument(arguments.get(0), tableOp, NOT_ROOT_ID);
         String tableName = validateTableNameArgument(arguments.get(1), tableOp, NOT_SYSTEM);
-        Namespace.ID namespaceId;
+        NamespaceId namespaceId;
         try {
           namespaceId = Namespaces.getNamespaceId(master.getContext(),
               Tables.qualify(tableName).getFirst());
@@ -292,9 +292,9 @@ class FateServiceHandler implements FateService.Iface {
         TableOperation tableOp = TableOperation.DELETE;
         String tableName = validateTableNameArgument(arguments.get(0), tableOp, NOT_SYSTEM);
 
-        final Table.ID tableId = ClientServiceHandler.checkTableId(master.getContext(), tableName,
+        final TableId tableId = ClientServiceHandler.checkTableId(master.getContext(), tableName,
             tableOp);
-        Namespace.ID namespaceId = getNamespaceIdFromTableId(tableOp, tableId);
+        NamespaceId namespaceId = getNamespaceIdFromTableId(tableOp, tableId);
 
         final boolean canDeleteTable;
         try {
@@ -312,8 +312,8 @@ class FateServiceHandler implements FateService.Iface {
       }
       case TABLE_ONLINE: {
         TableOperation tableOp = TableOperation.ONLINE;
-        final Table.ID tableId = validateTableIdArgument(arguments.get(0), tableOp, NOT_ROOT_ID);
-        Namespace.ID namespaceId = getNamespaceIdFromTableId(tableOp, tableId);
+        final TableId tableId = validateTableIdArgument(arguments.get(0), tableOp, NOT_ROOT_ID);
+        NamespaceId namespaceId = getNamespaceIdFromTableId(tableOp, tableId);
 
         final boolean canOnlineOfflineTable;
         try {
@@ -333,8 +333,8 @@ class FateServiceHandler implements FateService.Iface {
       }
       case TABLE_OFFLINE: {
         TableOperation tableOp = TableOperation.OFFLINE;
-        final Table.ID tableId = validateTableIdArgument(arguments.get(0), tableOp, NOT_ROOT_ID);
-        Namespace.ID namespaceId = getNamespaceIdFromTableId(tableOp, tableId);
+        final TableId tableId = validateTableIdArgument(arguments.get(0), tableOp, NOT_ROOT_ID);
+        NamespaceId namespaceId = getNamespaceIdFromTableId(tableOp, tableId);
 
         final boolean canOnlineOfflineTable;
         try {
@@ -358,9 +358,9 @@ class FateServiceHandler implements FateService.Iface {
         Text startRow = ByteBufferUtil.toText(arguments.get(1));
         Text endRow = ByteBufferUtil.toText(arguments.get(2));
 
-        final Table.ID tableId = ClientServiceHandler.checkTableId(master.getContext(), tableName,
+        final TableId tableId = ClientServiceHandler.checkTableId(master.getContext(), tableName,
             tableOp);
-        Namespace.ID namespaceId = getNamespaceIdFromTableId(tableOp, tableId);
+        NamespaceId namespaceId = getNamespaceIdFromTableId(tableOp, tableId);
 
         final boolean canMerge;
         try {
@@ -385,9 +385,9 @@ class FateServiceHandler implements FateService.Iface {
         Text startRow = ByteBufferUtil.toText(arguments.get(1));
         Text endRow = ByteBufferUtil.toText(arguments.get(2));
 
-        final Table.ID tableId = ClientServiceHandler.checkTableId(master.getContext(), tableName,
+        final TableId tableId = ClientServiceHandler.checkTableId(master.getContext(), tableName,
             tableOp);
-        Namespace.ID namespaceId = getNamespaceIdFromTableId(tableOp, tableId);
+        NamespaceId namespaceId = getNamespaceIdFromTableId(tableOp, tableId);
 
         final boolean canDeleteRange;
         try {
@@ -413,9 +413,9 @@ class FateServiceHandler implements FateService.Iface {
         String failDir = ByteBufferUtil.toString(arguments.get(2));
         boolean setTime = Boolean.parseBoolean(ByteBufferUtil.toString(arguments.get(3)));
 
-        final Table.ID tableId = ClientServiceHandler.checkTableId(master.getContext(), tableName,
+        final TableId tableId = ClientServiceHandler.checkTableId(master.getContext(), tableName,
             tableOp);
-        Namespace.ID namespaceId = getNamespaceIdFromTableId(tableOp, tableId);
+        NamespaceId namespaceId = getNamespaceIdFromTableId(tableOp, tableId);
 
         final boolean canBulkImport;
         try {
@@ -438,14 +438,14 @@ class FateServiceHandler implements FateService.Iface {
       }
       case TABLE_COMPACT: {
         TableOperation tableOp = TableOperation.COMPACT;
-        Table.ID tableId = validateTableIdArgument(arguments.get(0), tableOp, null);
+        TableId tableId = validateTableIdArgument(arguments.get(0), tableOp, null);
         byte[] startRow = ByteBufferUtil.toBytes(arguments.get(1));
         byte[] endRow = ByteBufferUtil.toBytes(arguments.get(2));
         List<IteratorSetting> iterators = IteratorUtil
             .decodeIteratorSettings(ByteBufferUtil.toBytes(arguments.get(3)));
         CompactionStrategyConfig compactionStrategy = CompactionStrategyConfigUtil
             .decode(ByteBufferUtil.toBytes(arguments.get(4)));
-        Namespace.ID namespaceId = getNamespaceIdFromTableId(tableOp, tableId);
+        NamespaceId namespaceId = getNamespaceIdFromTableId(tableOp, tableId);
 
         final boolean canCompact;
         try {
@@ -464,8 +464,8 @@ class FateServiceHandler implements FateService.Iface {
       }
       case TABLE_CANCEL_COMPACT: {
         TableOperation tableOp = TableOperation.COMPACT_CANCEL;
-        Table.ID tableId = validateTableIdArgument(arguments.get(0), tableOp, null);
-        Namespace.ID namespaceId = getNamespaceIdFromTableId(tableOp, tableId);
+        TableId tableId = validateTableIdArgument(arguments.get(0), tableOp, null);
+        NamespaceId namespaceId = getNamespaceIdFromTableId(tableOp, tableId);
 
         final boolean canCancelCompact;
         try {
@@ -486,7 +486,7 @@ class FateServiceHandler implements FateService.Iface {
         TableOperation tableOp = TableOperation.IMPORT;
         String tableName = validateTableNameArgument(arguments.get(0), tableOp, NOT_SYSTEM);
         String exportDir = ByteBufferUtil.toString(arguments.get(1));
-        Namespace.ID namespaceId;
+        NamespaceId namespaceId;
         try {
           namespaceId = Namespaces.getNamespaceId(master.getContext(),
               Tables.qualify(tableName).getFirst());
@@ -516,9 +516,9 @@ class FateServiceHandler implements FateService.Iface {
         String tableName = validateTableNameArgument(arguments.get(0), tableOp, NOT_SYSTEM);
         String exportDir = ByteBufferUtil.toString(arguments.get(1));
 
-        Table.ID tableId = ClientServiceHandler.checkTableId(master.getContext(), tableName,
+        TableId tableId = ClientServiceHandler.checkTableId(master.getContext(), tableName,
             tableOp);
-        Namespace.ID namespaceId = getNamespaceIdFromTableId(tableOp, tableId);
+        NamespaceId namespaceId = getNamespaceIdFromTableId(tableOp, tableId);
 
         final boolean canExport;
         try {
@@ -538,12 +538,12 @@ class FateServiceHandler implements FateService.Iface {
       }
       case TABLE_BULK_IMPORT2: {
         TableOperation tableOp = TableOperation.BULK_IMPORT;
-        Table.ID tableId = validateTableIdArgument(arguments.get(0), tableOp, NOT_ROOT_ID);
+        TableId tableId = validateTableIdArgument(arguments.get(0), tableOp, NOT_ROOT_ID);
         String dir = ByteBufferUtil.toString(arguments.get(1));
 
         boolean setTime = Boolean.parseBoolean(ByteBufferUtil.toString(arguments.get(2)));
 
-        Namespace.ID namespaceId = getNamespaceIdFromTableId(tableOp, tableId);
+        NamespaceId namespaceId = getNamespaceIdFromTableId(tableOp, tableId);
 
         final boolean canBulkImport;
         try {
@@ -554,7 +554,7 @@ class FateServiceHandler implements FateService.Iface {
           throwIfTableMissingSecurityException(e, tableId, "", TableOperation.BULK_IMPORT);
           throw e;
         } catch (TableNotFoundException e) {
-          throw new ThriftTableOperationException(tableId.canonicalID(), null,
+          throw new ThriftTableOperationException(tableId.canonical(), null,
               TableOperation.BULK_IMPORT, TableOperationExceptionType.NOTFOUND,
               "Table no longer exists");
         }
@@ -571,13 +571,13 @@ class FateServiceHandler implements FateService.Iface {
     }
   }
 
-  private Namespace.ID getNamespaceIdFromTableId(TableOperation tableOp, Table.ID tableId)
+  private NamespaceId getNamespaceIdFromTableId(TableOperation tableOp, TableId tableId)
       throws ThriftTableOperationException {
-    Namespace.ID namespaceId;
+    NamespaceId namespaceId;
     try {
       namespaceId = Tables.getNamespaceId(master.getContext(), tableId);
     } catch (TableNotFoundException e) {
-      throw new ThriftTableOperationException(tableId.canonicalID(), null, tableOp,
+      throw new ThriftTableOperationException(tableId.canonical(), null, tableOp,
           TableOperationExceptionType.NOTFOUND, e.getMessage());
     }
     return namespaceId;
@@ -603,11 +603,11 @@ class FateServiceHandler implements FateService.Iface {
    * @throws ThriftTableOperationException
    *           Thrown if {@code e} was thrown because {@link SecurityErrorCode#TABLE_DOESNT_EXIST}
    */
-  private void throwIfTableMissingSecurityException(ThriftSecurityException e, Table.ID tableId,
+  private void throwIfTableMissingSecurityException(ThriftSecurityException e, TableId tableId,
       String tableName, TableOperation op) throws ThriftTableOperationException {
     // ACCUMULO-3135 Table can be deleted after we get table ID but before we can check permission
     if (e.isSetCode() && e.getCode() == SecurityErrorCode.TABLE_DOESNT_EXIST) {
-      throw new ThriftTableOperationException(tableId.canonicalID(), tableName, op,
+      throw new ThriftTableOperationException(tableId.canonical(), tableName, op,
           TableOperationExceptionType.NOTFOUND, "Table no longer exists");
     }
   }
@@ -653,9 +653,9 @@ class FateServiceHandler implements FateService.Iface {
   }
 
   // Verify table name arguments are valid, and match any additional restrictions
-  private Table.ID validateTableIdArgument(ByteBuffer tableIdArg, TableOperation op,
-      Validator<Table.ID> userValidator) throws ThriftTableOperationException {
-    Table.ID tableId = tableIdArg == null ? null : ByteBufferUtil.toTableId(tableIdArg);
+  private TableId validateTableIdArgument(ByteBuffer tableIdArg, TableOperation op,
+      Validator<TableId> userValidator) throws ThriftTableOperationException {
+    TableId tableId = tableIdArg == null ? null : ByteBufferUtil.toTableId(tableIdArg);
     try {
       return VALID_ID.and(userValidator).validate(tableId);
     } catch (IllegalArgumentException e) {
@@ -663,7 +663,7 @@ class FateServiceHandler implements FateService.Iface {
       // Information provided by a client should generate a user-level exception, not a system-level
       // warning.
       log.debug(why);
-      throw new ThriftTableOperationException(tableId.canonicalID(), null, op,
+      throw new ThriftTableOperationException(tableId.canonical(), null, op,
           TableOperationExceptionType.INVALID_NAME, why);
     }
   }
diff --git a/server/master/src/main/java/org/apache/accumulo/master/Master.java b/server/master/src/main/java/org/apache/accumulo/master/Master.java
index 4d422c1..0f6dddd 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/Master.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/Master.java
@@ -48,7 +48,6 @@ import org.apache.accumulo.core.client.Scanner;
 import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.clientImpl.Namespace;
 import org.apache.accumulo.core.clientImpl.Namespaces;
-import org.apache.accumulo.core.clientImpl.Table;
 import org.apache.accumulo.core.clientImpl.Tables;
 import org.apache.accumulo.core.clientImpl.ThriftTransportPool;
 import org.apache.accumulo.core.clientImpl.thrift.TableOperation;
@@ -57,6 +56,8 @@ import org.apache.accumulo.core.clientImpl.thrift.ThriftTableOperationException;
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.data.Key;
+import org.apache.accumulo.core.data.NamespaceId;
+import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.dataImpl.KeyExtent;
 import org.apache.accumulo.core.master.state.tables.TableState;
@@ -404,11 +405,11 @@ public class Master
         // create initial namespaces
         String namespaces = getZooKeeperRoot() + Constants.ZNAMESPACES;
         zoo.putPersistentData(namespaces, new byte[0], NodeExistsPolicy.SKIP);
-        for (Pair<String,Namespace.ID> namespace : Iterables.concat(
-            Collections.singleton(new Pair<>(Namespace.ACCUMULO, Namespace.ID.ACCUMULO)),
-            Collections.singleton(new Pair<>(Namespace.DEFAULT, Namespace.ID.DEFAULT)))) {
+        for (Pair<String,NamespaceId> namespace : Iterables.concat(
+            Collections.singleton(new Pair<>(Namespace.ACCUMULO.name(), Namespace.ACCUMULO.id())),
+            Collections.singleton(new Pair<>(Namespace.DEFAULT.name(), Namespace.DEFAULT.id())))) {
           String ns = namespace.getFirst();
-          Namespace.ID id = namespace.getSecond();
+          NamespaceId id = namespace.getSecond();
           log.debug("Upgrade creating namespace \"{}\" (ID: {})", ns, id);
           if (!Namespaces.exists(context, id))
             TableManager.prepareNewNamespaceState(zoo, getInstanceID(), id, ns,
@@ -418,30 +419,30 @@ public class Master
         // create replication table in zk
         log.debug("Upgrade creating table {} (ID: {})", ReplicationTable.NAME, ReplicationTable.ID);
         TableManager.prepareNewTableState(zoo, getInstanceID(), ReplicationTable.ID,
-            Namespace.ID.ACCUMULO, ReplicationTable.NAME, TableState.OFFLINE,
+            Namespace.ACCUMULO.id(), ReplicationTable.NAME, TableState.OFFLINE,
             NodeExistsPolicy.SKIP);
 
         // create root table
         log.debug("Upgrade creating table {} (ID: {})", RootTable.NAME, RootTable.ID);
-        TableManager.prepareNewTableState(zoo, getInstanceID(), RootTable.ID, Namespace.ID.ACCUMULO,
-            RootTable.NAME, TableState.ONLINE, NodeExistsPolicy.SKIP);
+        TableManager.prepareNewTableState(zoo, getInstanceID(), RootTable.ID,
+            Namespace.ACCUMULO.id(), RootTable.NAME, TableState.ONLINE, NodeExistsPolicy.SKIP);
         Initialize.initSystemTablesConfig(context.getZooReaderWriter(), context.getZooKeeperRoot(),
             context.getHadoopConf());
         // ensure root user can flush root table
         security.grantTablePermission(context.rpcCreds(), security.getRootUsername(), RootTable.ID,
-            TablePermission.ALTER_TABLE, Namespace.ID.ACCUMULO);
+            TablePermission.ALTER_TABLE, Namespace.ACCUMULO.id());
 
         // put existing tables in the correct namespaces
         String tables = getZooKeeperRoot() + Constants.ZTABLES;
         for (String tableId : zoo.getChildren(tables)) {
-          Namespace.ID targetNamespace = (MetadataTable.ID.canonicalID().equals(tableId)
-              || RootTable.ID.canonicalID().equals(tableId)) ? Namespace.ID.ACCUMULO
-                  : Namespace.ID.DEFAULT;
+          NamespaceId targetNamespace = (MetadataTable.ID.canonical().equals(tableId)
+              || RootTable.ID.canonical().equals(tableId)) ? Namespace.ACCUMULO.id()
+                  : Namespace.DEFAULT.id();
           log.debug("Upgrade moving table {} (ID: {}) into namespace with ID {}",
               new String(zoo.getData(tables + "/" + tableId + Constants.ZTABLE_NAME, null), UTF_8),
               tableId, targetNamespace);
           zoo.putPersistentData(tables + "/" + tableId + Constants.ZTABLE_NAMESPACE,
-              targetNamespace.getUtf8(), NodeExistsPolicy.SKIP);
+              targetNamespace.canonical().getBytes(UTF_8), NodeExistsPolicy.SKIP);
         }
 
         // rename metadata table
@@ -466,7 +467,7 @@ public class Master
         for (String user : zoo.getChildren(users)) {
           zoo.putPersistentData(users + "/" + user + "/Namespaces", new byte[0],
               NodeExistsPolicy.SKIP);
-          perm.grantNamespacePermission(user, Namespace.ID.ACCUMULO.canonicalID(),
+          perm.grantNamespacePermission(user, Namespace.ACCUMULO.id().canonical(),
               NamespacePermission.READ);
         }
         // because we need to refer to the root username, we can't use the
@@ -476,7 +477,7 @@ public class Master
         // fail. Instead we should be able to use the security object since
         // the loop above should have made the needed structure in ZK.
         security.grantNamespacePermission(context.rpcCreds(), security.getRootUsername(),
-            Namespace.ID.ACCUMULO, NamespacePermission.ALTER_TABLE);
+            Namespace.ACCUMULO.id(), NamespacePermission.ALTER_TABLE);
 
         // add the currlog location for root tablet current logs
         zoo.putPersistentData(getZooKeeperRoot() + RootTable.ZROOT_TABLET_CURRENT_LOGS, new byte[0],
@@ -570,7 +571,7 @@ public class Master
     }
   }
 
-  private int assignedOrHosted(Table.ID tableId) {
+  private int assignedOrHosted(TableId tableId) {
     int result = 0;
     for (TabletGroupWatcher watcher : watchers) {
       TableCounts count = watcher.getStats(tableId);
@@ -612,8 +613,8 @@ public class Master
         // Count offline tablets for online tables
         for (TabletGroupWatcher watcher : watchers) {
           TableManager manager = context.getTableManager();
-          for (Entry<Table.ID,TableCounts> entry : watcher.getStats().entrySet()) {
-            Table.ID tableId = entry.getKey();
+          for (Entry<TableId,TableCounts> entry : watcher.getStats().entrySet()) {
+            TableId tableId = entry.getKey();
             TableCounts counts = entry.getValue();
             TableState tableState = manager.getTableState(tableId);
             if (tableState != null && tableState.equals(TableState.ONLINE)) {
@@ -643,10 +644,10 @@ public class Master
     return result;
   }
 
-  public void mustBeOnline(final Table.ID tableId) throws ThriftTableOperationException {
+  public void mustBeOnline(final TableId tableId) throws ThriftTableOperationException {
     Tables.clearCache(context);
     if (!Tables.getTableState(context, tableId).equals(TableState.ONLINE))
-      throw new ThriftTableOperationException(tableId.canonicalID(), null, TableOperation.MERGE,
+      throw new ThriftTableOperationException(tableId.canonical(), null, TableOperation.MERGE,
           TableOperationExceptionType.OFFLINE, "table is not online");
   }
 
@@ -731,7 +732,7 @@ public class Master
     return tserverSet.getConnection(server);
   }
 
-  public MergeInfo getMergeInfo(Table.ID tableId) {
+  public MergeInfo getMergeInfo(TableId tableId) {
     synchronized (mergeLock) {
       try {
         String path = getZooKeeperRoot() + Constants.ZTABLES + "/" + tableId + "/merge";
@@ -777,7 +778,7 @@ public class Master
     nextEvent.event("Merge state of %s set to %s", info.getExtent(), state);
   }
 
-  public void clearMergeState(Table.ID tableId) throws KeeperException, InterruptedException {
+  public void clearMergeState(TableId tableId) throws KeeperException, InterruptedException {
     synchronized (mergeLock) {
       String path = getZooKeeperRoot() + Constants.ZTABLES + "/" + tableId + "/merge";
       context.getZooReaderWriter().recursiveDelete(path, NodeMissingPolicy.SKIP);
@@ -817,7 +818,7 @@ public class Master
     return true;
   }
 
-  public void clearMigrations(Table.ID tableId) {
+  public void clearMigrations(TableId tableId) {
     synchronized (migrations) {
       Iterator<KeyExtent> iterator = migrations.keySet().iterator();
       while (iterator.hasNext()) {
@@ -977,7 +978,7 @@ public class Master
      */
     private void cleanupOfflineMigrations() {
       TableManager manager = context.getTableManager();
-      for (Table.ID tableId : Tables.getIdToNameMap(context).keySet()) {
+      for (TableId tableId : Tables.getIdToNameMap(context).keySet()) {
         TableState state = manager.getTableState(tableId);
         if (state == TableState.OFFLINE) {
           clearMigrations(tableId);
@@ -1673,7 +1674,7 @@ public class Master
   }
 
   @Override
-  public void stateChanged(Table.ID tableId, TableState state) {
+  public void stateChanged(TableId tableId, TableState state) {
     nextEvent.event("Table state in zookeeper changed for %s to %s", tableId, state);
     if (state == TableState.OFFLINE) {
       clearMigrations(tableId);
@@ -1687,8 +1688,8 @@ public class Master
   public void sessionExpired() {}
 
   @Override
-  public Set<Table.ID> onlineTables() {
-    Set<Table.ID> result = new HashSet<>();
+  public Set<TableId> onlineTables() {
+    Set<TableId> result = new HashSet<>();
     if (getMasterState() != MasterState.NORMAL) {
       if (getMasterState() != MasterState.UNLOAD_METADATA_TABLETS)
         result.add(MetadataTable.ID);
@@ -1698,7 +1699,7 @@ public class Master
     }
     TableManager manager = context.getTableManager();
 
-    for (Table.ID tableId : Tables.getIdToNameMap(context).keySet()) {
+    for (TableId tableId : Tables.getIdToNameMap(context).keySet()) {
       TableState state = manager.getTableState(tableId);
       if (state != null) {
         if (state == TableState.ONLINE)
@@ -1716,7 +1717,7 @@ public class Master
   @Override
   public Collection<MergeInfo> merges() {
     List<MergeInfo> result = new ArrayList<>();
-    for (Table.ID tableId : Tables.getIdToNameMap(context).keySet()) {
+    for (TableId tableId : Tables.getIdToNameMap(context).keySet()) {
       result.add(getMergeInfo(tableId));
     }
     return result;
diff --git a/server/master/src/main/java/org/apache/accumulo/master/MasterClientServiceHandler.java b/server/master/src/main/java/org/apache/accumulo/master/MasterClientServiceHandler.java
index e40ed52..c49d325 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/MasterClientServiceHandler.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/MasterClientServiceHandler.java
@@ -35,8 +35,6 @@ import org.apache.accumulo.core.client.admin.DelegationTokenConfig;
 import org.apache.accumulo.core.clientImpl.AuthenticationTokenIdentifier;
 import org.apache.accumulo.core.clientImpl.ClientContext;
 import org.apache.accumulo.core.clientImpl.DelegationTokenConfigSerializer;
-import org.apache.accumulo.core.clientImpl.Namespace;
-import org.apache.accumulo.core.clientImpl.Table;
 import org.apache.accumulo.core.clientImpl.Tables;
 import org.apache.accumulo.core.clientImpl.thrift.SecurityErrorCode;
 import org.apache.accumulo.core.clientImpl.thrift.TableOperation;
@@ -46,7 +44,9 @@ import org.apache.accumulo.core.clientImpl.thrift.ThriftTableOperationException;
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.data.Key;
+import org.apache.accumulo.core.data.NamespaceId;
 import org.apache.accumulo.core.data.Range;
+import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.dataImpl.KeyExtent;
 import org.apache.accumulo.core.dataImpl.thrift.TKeyExtent;
@@ -110,8 +110,8 @@ public class MasterClientServiceHandler extends FateServiceHandler
   @Override
   public long initiateFlush(TInfo tinfo, TCredentials c, String tableIdStr)
       throws ThriftSecurityException, ThriftTableOperationException {
-    Table.ID tableId = Table.ID.of(tableIdStr);
-    Namespace.ID namespaceId = getNamespaceIdFromTableId(TableOperation.FLUSH, tableId);
+    TableId tableId = TableId.of(tableIdStr);
+    NamespaceId namespaceId = getNamespaceIdFromTableId(TableOperation.FLUSH, tableId);
     master.security.canFlush(c, tableId, namespaceId);
 
     String zTablePath = Constants.ZROOT + "/" + master.getInstanceID() + Constants.ZTABLES + "/"
@@ -129,11 +129,11 @@ public class MasterClientServiceHandler extends FateServiceHandler
         }
       });
     } catch (NoNodeException nne) {
-      throw new ThriftTableOperationException(tableId.canonicalID(), null, TableOperation.FLUSH,
+      throw new ThriftTableOperationException(tableId.canonical(), null, TableOperation.FLUSH,
           TableOperationExceptionType.NOTFOUND, null);
     } catch (Exception e) {
       Master.log.warn("{}", e.getMessage(), e);
-      throw new ThriftTableOperationException(tableId.canonicalID(), null, TableOperation.FLUSH,
+      throw new ThriftTableOperationException(tableId.canonical(), null, TableOperation.FLUSH,
           TableOperationExceptionType.OTHER, null);
     }
     return Long.parseLong(new String(fid));
@@ -143,15 +143,15 @@ public class MasterClientServiceHandler extends FateServiceHandler
   public void waitForFlush(TInfo tinfo, TCredentials c, String tableIdStr, ByteBuffer startRowBB,
       ByteBuffer endRowBB, long flushID, long maxLoops)
       throws ThriftSecurityException, ThriftTableOperationException {
-    Table.ID tableId = Table.ID.of(tableIdStr);
-    Namespace.ID namespaceId = getNamespaceIdFromTableId(TableOperation.FLUSH, tableId);
+    TableId tableId = TableId.of(tableIdStr);
+    NamespaceId namespaceId = getNamespaceIdFromTableId(TableOperation.FLUSH, tableId);
     master.security.canFlush(c, tableId, namespaceId);
 
     Text startRow = ByteBufferUtil.toText(startRowBB);
     Text endRow = ByteBufferUtil.toText(endRowBB);
 
     if (endRow != null && startRow != null && startRow.compareTo(endRow) >= 0)
-      throw new ThriftTableOperationException(tableId.canonicalID(), null, TableOperation.FLUSH,
+      throw new ThriftTableOperationException(tableId.canonical(), null, TableOperation.FLUSH,
           TableOperationExceptionType.BAD_RANGE, "start row must be less than end row");
 
     Set<TServerInstance> serversToFlush = new HashSet<>(master.tserverSet.getCurrentServers());
@@ -204,7 +204,7 @@ public class MasterClientServiceHandler extends FateServiceHandler
         // TODO detect case of table offline AND tablets w/ logs? - ACCUMULO-1296
 
         if (tabletCount == 0 && !Tables.exists(master.getContext(), tableId))
-          throw new ThriftTableOperationException(tableId.canonicalID(), null, TableOperation.FLUSH,
+          throw new ThriftTableOperationException(tableId.canonical(), null, TableOperation.FLUSH,
               TableOperationExceptionType.NOTFOUND, null);
 
       } catch (TabletDeletedException e) {
@@ -215,13 +215,13 @@ public class MasterClientServiceHandler extends FateServiceHandler
 
   }
 
-  private Namespace.ID getNamespaceIdFromTableId(TableOperation tableOp, Table.ID tableId)
+  private NamespaceId getNamespaceIdFromTableId(TableOperation tableOp, TableId tableId)
       throws ThriftTableOperationException {
-    Namespace.ID namespaceId;
+    NamespaceId namespaceId;
     try {
       namespaceId = Tables.getNamespaceId(master.getContext(), tableId);
     } catch (TableNotFoundException e) {
-      throw new ThriftTableOperationException(tableId.canonicalID(), null, tableOp,
+      throw new ThriftTableOperationException(tableId.canonical(), null, tableOp,
           TableOperationExceptionType.NOTFOUND, e.getMessage());
     }
     return namespaceId;
@@ -385,7 +385,7 @@ public class MasterClientServiceHandler extends FateServiceHandler
       String value, TableOperation op)
       throws ThriftSecurityException, ThriftTableOperationException {
 
-    Namespace.ID namespaceId = null;
+    NamespaceId namespaceId = null;
     namespaceId = ClientServiceHandler.checkNamespaceId(master.getContext(), namespace, op);
 
     if (!master.security.canAlterNamespace(c, namespaceId))
@@ -402,19 +402,19 @@ public class MasterClientServiceHandler extends FateServiceHandler
       // namespace was deleted:
       ClientServiceHandler.checkNamespaceId(master.getContext(), namespace, op);
       log.info("Error altering namespace property", e);
-      throw new ThriftTableOperationException(namespaceId.canonicalID(), namespace, op,
+      throw new ThriftTableOperationException(namespaceId.canonical(), namespace, op,
           TableOperationExceptionType.OTHER, "Problem altering namespaceproperty");
     } catch (Exception e) {
       log.error("Problem altering namespace property", e);
-      throw new ThriftTableOperationException(namespaceId.canonicalID(), namespace, op,
+      throw new ThriftTableOperationException(namespaceId.canonical(), namespace, op,
           TableOperationExceptionType.OTHER, "Problem altering namespace property");
     }
   }
 
   private void alterTableProperty(TCredentials c, String tableName, String property, String value,
       TableOperation op) throws ThriftSecurityException, ThriftTableOperationException {
-    final Table.ID tableId = ClientServiceHandler.checkTableId(master.getContext(), tableName, op);
-    Namespace.ID namespaceId = getNamespaceIdFromTableId(op, tableId);
+    final TableId tableId = ClientServiceHandler.checkTableId(master.getContext(), tableName, op);
+    NamespaceId namespaceId = getNamespaceIdFromTableId(op, tableId);
     if (!master.security.canAlterTable(c, tableId, namespaceId))
       throw new ThriftSecurityException(c.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
 
@@ -429,11 +429,11 @@ public class MasterClientServiceHandler extends FateServiceHandler
       // was deleted:
       ClientServiceHandler.checkTableId(master.getContext(), tableName, op);
       log.info("Error altering table property", e);
-      throw new ThriftTableOperationException(tableId.canonicalID(), tableName, op,
+      throw new ThriftTableOperationException(tableId.canonical(), tableName, op,
           TableOperationExceptionType.OTHER, "Problem altering table property");
     } catch (Exception e) {
       log.error("Problem altering table property", e);
-      throw new ThriftTableOperationException(tableId.canonicalID(), tableName, op,
+      throw new ThriftTableOperationException(tableId.canonical(), tableName, op,
           TableOperationExceptionType.OTHER, "Problem altering table property");
     }
   }
@@ -497,7 +497,7 @@ public class MasterClientServiceHandler extends FateServiceHandler
       Set<String> logsToWatch) throws TException {
     AccumuloClient client = master.getContext();
 
-    final Text tableId = new Text(getTableId(master.getContext(), tableName).getUtf8());
+    final Text tableId = new Text(getTableId(master.getContext(), tableName).canonical());
 
     drainLog.trace("Waiting for {} to be replicated for {}", logsToWatch, tableId);
 
@@ -535,7 +535,7 @@ public class MasterClientServiceHandler extends FateServiceHandler
     }
   }
 
-  protected Table.ID getTableId(ClientContext context, String tableName)
+  protected TableId getTableId(ClientContext context, String tableName)
       throws ThriftTableOperationException {
     return ClientServiceHandler.checkTableId(context, tableName, null);
   }
diff --git a/server/master/src/main/java/org/apache/accumulo/master/TabletGroupWatcher.java b/server/master/src/main/java/org/apache/accumulo/master/TabletGroupWatcher.java
index 0a1437d..ef512cd 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/TabletGroupWatcher.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/TabletGroupWatcher.java
@@ -43,12 +43,12 @@ import org.apache.accumulo.core.client.MutationsRejectedException;
 import org.apache.accumulo.core.client.RowIterator;
 import org.apache.accumulo.core.client.Scanner;
 import org.apache.accumulo.core.client.TableNotFoundException;
-import org.apache.accumulo.core.clientImpl.Table;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.data.Key;
 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.TableId;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.dataImpl.KeyExtent;
 import org.apache.accumulo.core.master.state.tables.TableState;
@@ -116,11 +116,11 @@ abstract class TabletGroupWatcher extends Daemon {
   /** Should this {@code TabletGroupWatcher} suspend tablets? */
   abstract boolean canSuspendTablets();
 
-  Map<Table.ID,TableCounts> getStats() {
+  Map<TableId,TableCounts> getStats() {
     return stats.getLast();
   }
 
-  TableCounts getStats(Table.ID tableId) {
+  TableCounts getStats(TableId tableId) {
     return stats.getLast(tableId);
   }
 
@@ -148,8 +148,8 @@ abstract class TabletGroupWatcher extends Daemon {
       int unloaded = 0;
       ClosableIterator<TabletLocationState> iter = null;
       try {
-        Map<Table.ID,MergeStats> mergeStatsCache = new HashMap<>();
-        Map<Table.ID,MergeStats> currentMerges = new HashMap<>();
+        Map<TableId,MergeStats> mergeStatsCache = new HashMap<>();
+        Map<TableId,MergeStats> currentMerges = new HashMap<>();
         for (MergeInfo merge : master.merges()) {
           if (merge.getExtent() != null) {
             currentMerges.put(merge.getExtent().getTableId(), new MergeStats(merge));
@@ -213,7 +213,7 @@ abstract class TabletGroupWatcher extends Daemon {
             unloaded = 0;
             eventListener.waitForEvents(Master.TIME_TO_WAIT_BETWEEN_SCANS);
           }
-          Table.ID tableId = tls.extent.getTableId();
+          TableId tableId = tls.extent.getTableId();
           TableConfiguration tableConf = this.master.getConfigurationFactory()
               .getTableConfiguration(tableId);
 
@@ -543,7 +543,7 @@ abstract class TabletGroupWatcher extends Daemon {
     }
   }
 
-  private void updateMergeState(Map<Table.ID,MergeStats> mergeStatsCache) {
+  private void updateMergeState(Map<TableId,MergeStats> mergeStatsCache) {
     for (MergeStats stats : mergeStatsCache.values()) {
       try {
         MergeState update = stats.nextMergeState(this.master.getContext(), this.master);
diff --git a/server/master/src/main/java/org/apache/accumulo/master/replication/FinishedWorkUpdater.java b/server/master/src/main/java/org/apache/accumulo/master/replication/FinishedWorkUpdater.java
index 9eb749a..38b8e78 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/replication/FinishedWorkUpdater.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/replication/FinishedWorkUpdater.java
@@ -28,10 +28,10 @@ import org.apache.accumulo.core.client.BatchScanner;
 import org.apache.accumulo.core.client.BatchWriter;
 import org.apache.accumulo.core.client.IteratorSetting;
 import org.apache.accumulo.core.client.MutationsRejectedException;
-import org.apache.accumulo.core.clientImpl.Table;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.data.Range;
+import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.iterators.user.WholeRowIterator;
 import org.apache.accumulo.core.protobuf.ProtobufUtil;
@@ -99,7 +99,7 @@ public class FinishedWorkUpdater implements Runnable {
         log.debug("Processing work progress for {} with {} columns",
             serializedRow.getKey().getRow(), wholeRow.size());
 
-        Map<Table.ID,Long> tableIdToProgress = new HashMap<>();
+        Map<TableId,Long> tableIdToProgress = new HashMap<>();
         boolean error = false;
         Text buffer = new Text();
 
@@ -134,7 +134,7 @@ public class FinishedWorkUpdater implements Runnable {
         }
 
         // Update the replication table for each source table we found work records for
-        for (Entry<Table.ID,Long> entry : tableIdToProgress.entrySet()) {
+        for (Entry<TableId,Long> entry : tableIdToProgress.entrySet()) {
           // If the progress is 0, then no one has replicated anything, and we don't need to update
           // anything
           if (entry.getValue() == 0) {
@@ -153,7 +153,7 @@ public class FinishedWorkUpdater implements Runnable {
           Value serializedUpdatedStatus = ProtobufUtil.toValue(updatedStatus);
 
           // Pull the sourceTableId into a Text
-          Table.ID srcTableId = entry.getKey();
+          TableId srcTableId = entry.getKey();
 
           // Make the mutation
           StatusSection.add(replMutation, srcTableId, serializedUpdatedStatus);
diff --git a/server/master/src/main/java/org/apache/accumulo/master/replication/RemoveCompleteReplicationRecords.java b/server/master/src/main/java/org/apache/accumulo/master/replication/RemoveCompleteReplicationRecords.java
index 47bc64d..9fe182a 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/replication/RemoveCompleteReplicationRecords.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/replication/RemoveCompleteReplicationRecords.java
@@ -30,10 +30,10 @@ import org.apache.accumulo.core.client.BatchScanner;
 import org.apache.accumulo.core.client.BatchWriter;
 import org.apache.accumulo.core.client.IteratorSetting;
 import org.apache.accumulo.core.client.MutationsRejectedException;
-import org.apache.accumulo.core.clientImpl.Table;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.data.Range;
+import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.iterators.user.WholeRowIterator;
 import org.apache.accumulo.core.replication.ReplicationSchema.OrderSection;
@@ -154,7 +154,7 @@ public class RemoveCompleteReplicationRecords implements Runnable {
     }
 
     Mutation m = new Mutation(row);
-    Map<Table.ID,Long> tableToTimeCreated = new HashMap<>();
+    Map<TableId,Long> tableToTimeCreated = new HashMap<>();
     for (Entry<Key,Value> entry : columns.entrySet()) {
       Status status = null;
       try {
@@ -178,9 +178,9 @@ public class RemoveCompleteReplicationRecords implements Runnable {
 
       m.putDelete(colf, colq);
 
-      Table.ID tableId;
+      TableId tableId;
       if (StatusSection.NAME.equals(colf)) {
-        tableId = Table.ID.of(colq.toString());
+        tableId = TableId.of(colq.toString());
       } else if (WorkSection.NAME.equals(colf)) {
         ReplicationTarget target = ReplicationTarget.from(colq);
         tableId = target.getSourceTableId();
@@ -203,11 +203,11 @@ public class RemoveCompleteReplicationRecords implements Runnable {
 
     List<Mutation> mutations = new ArrayList<>();
     mutations.add(m);
-    for (Entry<Table.ID,Long> entry : tableToTimeCreated.entrySet()) {
+    for (Entry<TableId,Long> entry : tableToTimeCreated.entrySet()) {
       log.info("Removing order mutation for table {} at {} for {}", entry.getKey(),
           entry.getValue(), row);
       Mutation orderMutation = OrderSection.createMutation(row.toString(), entry.getValue());
-      orderMutation.putDelete(OrderSection.NAME, new Text(entry.getKey().getUtf8()));
+      orderMutation.putDelete(OrderSection.NAME, new Text(entry.getKey().canonical()));
       mutations.add(orderMutation);
     }
 
diff --git a/server/master/src/main/java/org/apache/accumulo/master/replication/SequentialWorkAssigner.java b/server/master/src/main/java/org/apache/accumulo/master/replication/SequentialWorkAssigner.java
index 8210847..31e74d6 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/replication/SequentialWorkAssigner.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/replication/SequentialWorkAssigner.java
@@ -25,8 +25,8 @@ import java.util.Map.Entry;
 import java.util.Set;
 
 import org.apache.accumulo.core.client.AccumuloClient;
-import org.apache.accumulo.core.clientImpl.Table;
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
+import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.replication.ReplicationConstants;
 import org.apache.accumulo.core.replication.ReplicationTarget;
 import org.apache.accumulo.fate.zookeeper.ZooUtil;
@@ -56,7 +56,7 @@ public class SequentialWorkAssigner extends DistributedWorkQueueWorkAssigner {
    * }
    */
   // @formatter:on
-  private Map<String,Map<Table.ID,String>> queuedWorkByPeerName;
+  private Map<String,Map<TableId,String>> queuedWorkByPeerName;
 
   public SequentialWorkAssigner() {}
 
@@ -69,7 +69,7 @@ public class SequentialWorkAssigner extends DistributedWorkQueueWorkAssigner {
     return NAME;
   }
 
-  protected void setQueuedWork(Map<String,Map<Table.ID,String>> queuedWork) {
+  protected void setQueuedWork(Map<String,Map<TableId,String>> queuedWork) {
     this.queuedWorkByPeerName = queuedWork;
   }
 
@@ -97,12 +97,12 @@ public class SequentialWorkAssigner extends DistributedWorkQueueWorkAssigner {
           .fromQueueKey(work);
       String filename = entry.getKey();
       String peerName = entry.getValue().getPeerName();
-      Table.ID sourceTableId = entry.getValue().getSourceTableId();
+      TableId sourceTableId = entry.getValue().getSourceTableId();
 
       log.debug("In progress replication of {} from table with ID {} to peer {}", filename,
           sourceTableId, peerName);
 
-      Map<Table.ID,String> replicationForPeer = queuedWorkByPeerName.get(peerName);
+      Map<TableId,String> replicationForPeer = queuedWorkByPeerName.get(peerName);
       if (replicationForPeer == null) {
         replicationForPeer = new HashMap<>();
         queuedWorkByPeerName.put(peerName, replicationForPeer);
@@ -117,7 +117,7 @@ public class SequentialWorkAssigner extends DistributedWorkQueueWorkAssigner {
    */
   @Override
   protected void cleanupFinishedWork() {
-    final Iterator<Entry<String,Map<Table.ID,String>>> queuedWork = queuedWorkByPeerName.entrySet()
+    final Iterator<Entry<String,Map<TableId,String>>> queuedWork = queuedWorkByPeerName.entrySet()
         .iterator();
     final String instanceId = client.instanceOperations().getInstanceID();
 
@@ -125,17 +125,17 @@ public class SequentialWorkAssigner extends DistributedWorkQueueWorkAssigner {
     // Check the status of all the work we've queued up
     while (queuedWork.hasNext()) {
       // {peer -> {tableId -> workKey, tableId -> workKey, ... }, peer -> ...}
-      Entry<String,Map<Table.ID,String>> workForPeer = queuedWork.next();
+      Entry<String,Map<TableId,String>> workForPeer = queuedWork.next();
 
       // TableID to workKey (filename and ReplicationTarget)
-      Map<Table.ID,String> queuedReplication = workForPeer.getValue();
+      Map<TableId,String> queuedReplication = workForPeer.getValue();
 
-      Iterator<Entry<Table.ID,String>> iter = queuedReplication.entrySet().iterator();
+      Iterator<Entry<TableId,String>> iter = queuedReplication.entrySet().iterator();
       // Loop over every target we need to replicate this file to, removing the target when
       // the replication task has finished
       while (iter.hasNext()) {
         // tableID -> workKey
-        Entry<Table.ID,String> entry = iter.next();
+        Entry<TableId,String> entry = iter.next();
         // Null equates to the work for this target was finished
         if (zooCache.get(ZooUtil.getRoot(instanceId) + ReplicationConstants.ZOO_WORK_QUEUE + "/"
             + entry.getValue()) == null) {
@@ -157,7 +157,7 @@ public class SequentialWorkAssigner extends DistributedWorkQueueWorkAssigner {
 
   @Override
   protected boolean shouldQueueWork(ReplicationTarget target) {
-    Map<Table.ID,String> queuedWorkForPeer = this.queuedWorkByPeerName.get(target.getPeerName());
+    Map<TableId,String> queuedWorkForPeer = this.queuedWorkByPeerName.get(target.getPeerName());
     if (queuedWorkForPeer == null) {
       return true;
     }
@@ -171,7 +171,7 @@ public class SequentialWorkAssigner extends DistributedWorkQueueWorkAssigner {
   @Override
   protected boolean queueWork(Path path, ReplicationTarget target) {
     String queueKey = DistributedWorkQueueWorkAssignerHelper.getQueueKey(path.getName(), target);
-    Map<Table.ID,String> workForPeer = this.queuedWorkByPeerName.get(target.getPeerName());
+    Map<TableId,String> workForPeer = this.queuedWorkByPeerName.get(target.getPeerName());
     if (workForPeer == null) {
       workForPeer = new HashMap<>();
       this.queuedWorkByPeerName.put(target.getPeerName(), workForPeer);
@@ -201,7 +201,7 @@ public class SequentialWorkAssigner extends DistributedWorkQueueWorkAssigner {
 
   @Override
   protected Set<String> getQueuedWork(ReplicationTarget target) {
-    Map<Table.ID,String> queuedWorkForPeer = this.queuedWorkByPeerName.get(target.getPeerName());
+    Map<TableId,String> queuedWorkForPeer = this.queuedWorkByPeerName.get(target.getPeerName());
     if (queuedWorkForPeer == null) {
       return Collections.emptySet();
     }
@@ -216,7 +216,7 @@ public class SequentialWorkAssigner extends DistributedWorkQueueWorkAssigner {
 
   @Override
   protected void removeQueuedWork(ReplicationTarget target, String queueKey) {
-    Map<Table.ID,String> queuedWorkForPeer = this.queuedWorkByPeerName.get(target.getPeerName());
+    Map<TableId,String> queuedWorkForPeer = this.queuedWorkByPeerName.get(target.getPeerName());
     if (queuedWorkForPeer == null) {
       log.warn("removeQueuedWork called when no work was queued for {}", target.getPeerName());
       return;
diff --git a/server/master/src/main/java/org/apache/accumulo/master/replication/StatusMaker.java b/server/master/src/main/java/org/apache/accumulo/master/replication/StatusMaker.java
index d20b22a..7ea2a0d 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/replication/StatusMaker.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/replication/StatusMaker.java
@@ -27,9 +27,9 @@ import org.apache.accumulo.core.client.BatchWriterConfig;
 import org.apache.accumulo.core.client.MutationsRejectedException;
 import org.apache.accumulo.core.client.Scanner;
 import org.apache.accumulo.core.client.TableNotFoundException;
-import org.apache.accumulo.core.clientImpl.Table;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Mutation;
+import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.metadata.MetadataTable;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema;
@@ -113,7 +113,7 @@ public class StatusMaker {
         }
         // Extract the useful bits from the status key
         MetadataSchema.ReplicationSection.getFile(entry.getKey(), file);
-        Table.ID tableId = MetadataSchema.ReplicationSection.getTableId(entry.getKey());
+        TableId tableId = MetadataSchema.ReplicationSection.getTableId(entry.getKey());
 
         Status status;
         try {
@@ -162,10 +162,10 @@ public class StatusMaker {
   /**
    * Create a status record in the replication table
    */
-  protected boolean addStatusRecord(Text file, Table.ID tableId, Value v) {
+  protected boolean addStatusRecord(Text file, TableId tableId, Value v) {
     try {
       Mutation m = new Mutation(file);
-      m.put(StatusSection.NAME, new Text(tableId.getUtf8()), v);
+      m.put(StatusSection.NAME, new Text(tableId.canonical()), v);
 
       try {
         replicationWriter.addMutation(m);
@@ -199,7 +199,7 @@ public class StatusMaker {
    * @param value
    *          Serialized version of the Status msg
    */
-  protected boolean addOrderRecord(Text file, Table.ID tableId, Status stat, Value value) {
+  protected boolean addOrderRecord(Text file, TableId tableId, Status stat, Value value) {
     try {
       if (!stat.hasCreatedTime()) {
         try {
diff --git a/server/master/src/main/java/org/apache/accumulo/master/replication/WorkMaker.java b/server/master/src/main/java/org/apache/accumulo/master/replication/WorkMaker.java
index 309378f..f867aa7 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/replication/WorkMaker.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/replication/WorkMaker.java
@@ -25,10 +25,10 @@ import org.apache.accumulo.core.client.AccumuloClient;
 import org.apache.accumulo.core.client.BatchWriter;
 import org.apache.accumulo.core.client.MutationsRejectedException;
 import org.apache.accumulo.core.client.Scanner;
-import org.apache.accumulo.core.clientImpl.Table;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Mutation;
+import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.replication.ReplicationSchema;
 import org.apache.accumulo.core.replication.ReplicationSchema.StatusSection;
@@ -95,7 +95,7 @@ public class WorkMaker {
       for (Entry<Key,Value> entry : s) {
         // Extract the useful bits from the status key
         ReplicationSchema.StatusSection.getFile(entry.getKey(), file);
-        Table.ID tableId = ReplicationSchema.StatusSection.getTableId(entry.getKey());
+        TableId tableId = ReplicationSchema.StatusSection.getTableId(entry.getKey());
         log.debug("Processing replication status record for {} on table {}", file, tableId);
 
         Status status;
@@ -172,7 +172,7 @@ public class WorkMaker {
   }
 
   protected void addWorkRecord(Text file, Value v, Map<String,String> targets,
-      Table.ID sourceTableId) {
+      TableId sourceTableId) {
     log.info("Adding work records for {} to targets {}", file, targets);
     try {
       Mutation m = new Mutation(file);
diff --git a/server/master/src/main/java/org/apache/accumulo/master/state/MergeStats.java b/server/master/src/main/java/org/apache/accumulo/master/state/MergeStats.java
index e5cda40..82a374f 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/state/MergeStats.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/state/MergeStats.java
@@ -24,9 +24,9 @@ import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.AccumuloClient;
 import org.apache.accumulo.core.client.Scanner;
 import org.apache.accumulo.core.client.TableNotFoundException;
-import org.apache.accumulo.core.clientImpl.Table;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Range;
+import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.dataImpl.KeyExtent;
 import org.apache.accumulo.core.metadata.MetadataTable;
@@ -197,7 +197,7 @@ public class MergeStats {
     if (start == null) {
       start = new Text();
     }
-    Table.ID tableId = extent.getTableId();
+    TableId tableId = extent.getTableId();
     Text first = TabletsSection.getRow(tableId, start);
     Range range = new Range(first, false, null, true);
     scanner.setRange(range.clip(MetadataSchema.TabletsSection.getRange()));
diff --git a/server/master/src/main/java/org/apache/accumulo/master/state/TableStats.java b/server/master/src/main/java/org/apache/accumulo/master/state/TableStats.java
index 0452be6..441842c 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/state/TableStats.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/state/TableStats.java
@@ -19,13 +19,13 @@ package org.apache.accumulo.master.state;
 import java.util.HashMap;
 import java.util.Map;
 
-import org.apache.accumulo.core.clientImpl.Table;
+import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.master.thrift.MasterState;
 import org.apache.accumulo.server.master.state.TabletState;
 
 public class TableStats {
-  private Map<Table.ID,TableCounts> last = new HashMap<>();
-  private Map<Table.ID,TableCounts> next;
+  private Map<TableId,TableCounts> last = new HashMap<>();
+  private Map<TableId,TableCounts> next;
   private long startScan = 0;
   private long endScan = 0;
   private MasterState state;
@@ -35,7 +35,7 @@ public class TableStats {
     startScan = System.currentTimeMillis();
   }
 
-  public synchronized void update(Table.ID tableId, TabletState state) {
+  public synchronized void update(TableId tableId, TabletState state) {
     TableCounts counts = next.get(tableId);
     if (counts == null) {
       counts = new TableCounts();
@@ -51,7 +51,7 @@ public class TableStats {
     this.state = state;
   }
 
-  public synchronized Map<Table.ID,TableCounts> getLast() {
+  public synchronized Map<TableId,TableCounts> getLast() {
     return last;
   }
 
@@ -59,7 +59,7 @@ public class TableStats {
     return state;
   }
 
-  public synchronized TableCounts getLast(Table.ID tableId) {
+  public synchronized TableCounts getLast(TableId tableId) {
     TableCounts result = last.get(tableId);
     if (result == null)
       return new TableCounts();
diff --git a/server/master/src/main/java/org/apache/accumulo/master/tableOps/ChangeTableState.java b/server/master/src/main/java/org/apache/accumulo/master/tableOps/ChangeTableState.java
index 8452009..1aef0ba 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/tableOps/ChangeTableState.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/tableOps/ChangeTableState.java
@@ -16,9 +16,9 @@
  */
 package org.apache.accumulo.master.tableOps;
 
-import org.apache.accumulo.core.clientImpl.Namespace;
-import org.apache.accumulo.core.clientImpl.Table;
 import org.apache.accumulo.core.clientImpl.thrift.TableOperation;
+import org.apache.accumulo.core.data.NamespaceId;
+import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.master.state.tables.TableState;
 import org.apache.accumulo.fate.Repo;
 import org.apache.accumulo.master.Master;
@@ -27,11 +27,11 @@ import org.slf4j.LoggerFactory;
 public class ChangeTableState extends MasterRepo {
 
   private static final long serialVersionUID = 1L;
-  private Table.ID tableId;
-  private Namespace.ID namespaceId;
+  private TableId tableId;
+  private NamespaceId namespaceId;
   private TableOperation top;
 
-  public ChangeTableState(Namespace.ID namespaceId, Table.ID tableId, TableOperation top) {
+  public ChangeTableState(NamespaceId namespaceId, TableId tableId, TableOperation top) {
     this.tableId = tableId;
     this.namespaceId = namespaceId;
     this.top = top;
diff --git a/server/master/src/main/java/org/apache/accumulo/master/tableOps/TableInfo.java b/server/master/src/main/java/org/apache/accumulo/master/tableOps/TableInfo.java
index a0831a3..c108dd4 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/tableOps/TableInfo.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/tableOps/TableInfo.java
@@ -20,17 +20,16 @@ import java.io.Serializable;
 import java.util.Map;
 
 import org.apache.accumulo.core.client.admin.InitialTableState;
-import org.apache.accumulo.core.clientImpl.Namespace;
-import org.apache.accumulo.core.clientImpl.Table;
-import org.apache.accumulo.core.clientImpl.Table.ID;
+import org.apache.accumulo.core.data.NamespaceId;
+import org.apache.accumulo.core.data.TableId;
 
 public class TableInfo implements Serializable {
 
   private static final long serialVersionUID = 1L;
 
   private String tableName;
-  private Table.ID tableId;
-  private Namespace.ID namespaceId;
+  private TableId tableId;
+  private NamespaceId namespaceId;
 
   private char timeType;
   private String user;
@@ -54,19 +53,19 @@ public class TableInfo implements Serializable {
     this.tableName = tableName;
   }
 
-  public ID getTableId() {
+  public TableId getTableId() {
     return tableId;
   }
 
-  public void setTableId(ID tableId) {
+  public void setTableId(TableId tableId) {
     this.tableId = tableId;
   }
 
-  public Namespace.ID getNamespaceId() {
+  public NamespaceId getNamespaceId() {
     return namespaceId;
   }
 
-  public void setNamespaceId(Namespace.ID namespaceId) {
+  public void setNamespaceId(NamespaceId namespaceId) {
     this.namespaceId = namespaceId;
   }
 
diff --git a/server/master/src/main/java/org/apache/accumulo/master/tableOps/Utils.java b/server/master/src/main/java/org/apache/accumulo/master/tableOps/Utils.java
index 34f3f35..61c3488 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/tableOps/Utils.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/tableOps/Utils.java
@@ -30,14 +30,14 @@ import java.util.concurrent.locks.ReentrantLock;
 import java.util.function.Function;
 
 import org.apache.accumulo.core.Constants;
-import org.apache.accumulo.core.clientImpl.AbstractId;
 import org.apache.accumulo.core.clientImpl.AcceptableThriftTableOperationException;
-import org.apache.accumulo.core.clientImpl.Namespace;
 import org.apache.accumulo.core.clientImpl.Namespaces;
-import org.apache.accumulo.core.clientImpl.Table;
 import org.apache.accumulo.core.clientImpl.Tables;
 import org.apache.accumulo.core.clientImpl.thrift.TableOperation;
 import org.apache.accumulo.core.clientImpl.thrift.TableOperationExceptionType;
+import org.apache.accumulo.core.data.AbstractId;
+import org.apache.accumulo.core.data.NamespaceId;
+import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.fate.zookeeper.DistributedReadWriteLock;
 import org.apache.accumulo.fate.zookeeper.IZooReaderWriter;
 import org.apache.accumulo.fate.zookeeper.ZooQueueLock;
@@ -56,16 +56,16 @@ public class Utils {
   private static final Logger log = LoggerFactory.getLogger(Utils.class);
 
   public static void checkTableDoesNotExist(ServerContext context, String tableName,
-      Table.ID tableId, TableOperation operation) throws AcceptableThriftTableOperationException {
+      TableId tableId, TableOperation operation) throws AcceptableThriftTableOperationException {
 
-    Table.ID id = Tables.getNameToIdMap(context).get(tableName);
+    TableId id = Tables.getNameToIdMap(context).get(tableName);
 
     if (id != null && !id.equals(tableId))
       throw new AcceptableThriftTableOperationException(null, tableName, operation,
           TableOperationExceptionType.EXISTS, null);
   }
 
-  public static <T extends AbstractId> T getNextId(String name, ServerContext context,
+  public static <T extends AbstractId<T>> T getNextId(String name, ServerContext context,
       Function<String,T> newIdFunction) throws AcceptableThriftTableOperationException {
     try {
       IZooReaderWriter zoo = context.getZooReaderWriter();
@@ -86,13 +86,13 @@ public class Utils {
   static final Lock tableNameLock = new ReentrantLock();
   static final Lock idLock = new ReentrantLock();
 
-  public static long reserveTable(Master env, Table.ID tableId, long tid, boolean writeLock,
+  public static long reserveTable(Master env, TableId tableId, long tid, boolean writeLock,
       boolean tableMustExist, TableOperation op) throws Exception {
     if (getLock(env.getContext(), tableId, tid, writeLock).tryLock()) {
       if (tableMustExist) {
         IZooReaderWriter zk = env.getContext().getZooReaderWriter();
         if (!zk.exists(env.getContext().getZooKeeperRoot() + Constants.ZTABLES + "/" + tableId))
-          throw new AcceptableThriftTableOperationException(tableId.canonicalID(), "", op,
+          throw new AcceptableThriftTableOperationException(tableId.canonical(), "", op,
               TableOperationExceptionType.NOTFOUND, "Table does not exist");
       }
       log.info("table {} ({}) locked for {} operation: {}", tableId, Long.toHexString(tid),
@@ -102,27 +102,27 @@ public class Utils {
       return 100;
   }
 
-  public static void unreserveTable(Master env, Table.ID tableId, long tid, boolean writeLock) {
+  public static void unreserveTable(Master env, TableId tableId, long tid, boolean writeLock) {
     getLock(env.getContext(), tableId, tid, writeLock).unlock();
     log.info("table {} ({}) unlocked for ", tableId, Long.toHexString(tid),
         (writeLock ? "write" : "read"));
   }
 
-  public static void unreserveNamespace(Master env, Namespace.ID namespaceId, long id,
+  public static void unreserveNamespace(Master env, NamespaceId namespaceId, long id,
       boolean writeLock) {
     getLock(env.getContext(), namespaceId, id, writeLock).unlock();
     log.info("namespace {} ({}) unlocked for {}", namespaceId, Long.toHexString(id),
         (writeLock ? "write" : "read"));
   }
 
-  public static long reserveNamespace(Master env, Namespace.ID namespaceId, long id,
+  public static long reserveNamespace(Master env, NamespaceId namespaceId, long id,
       boolean writeLock, boolean mustExist, TableOperation op) throws Exception {
     if (getLock(env.getContext(), namespaceId, id, writeLock).tryLock()) {
       if (mustExist) {
         IZooReaderWriter zk = env.getContext().getZooReaderWriter();
         if (!zk.exists(
             env.getContext().getZooKeeperRoot() + Constants.ZNAMESPACES + "/" + namespaceId))
-          throw new AcceptableThriftTableOperationException(namespaceId.canonicalID(), "", op,
+          throw new AcceptableThriftTableOperationException(namespaceId.canonical(), "", op,
               TableOperationExceptionType.NAMESPACE_NOTFOUND, "Namespace does not exist");
       }
       log.info("namespace {} ({}) locked for {} operation: {}", namespaceId, Long.toHexString(id),
@@ -153,10 +153,11 @@ public class Utils {
         String.format("%016x", tid));
   }
 
-  private static Lock getLock(ServerContext context, AbstractId id, long tid, boolean writeLock) {
+  private static Lock getLock(ServerContext context, AbstractId<?> id, long tid,
+      boolean writeLock) {
     byte[] lockData = String.format("%016x", tid).getBytes(UTF_8);
     ZooQueueLock qlock = new ZooQueueLock(context.getZooReaderWriter(),
-        context.getZooKeeperRoot() + Constants.ZTABLE_LOCKS + "/" + id, false);
+        context.getZooKeeperRoot() + Constants.ZTABLE_LOCKS + "/" + id.canonical(), false);
     Lock lock = DistributedReadWriteLock.recoverLock(qlock, lockData);
     if (lock == null) {
       DistributedReadWriteLock locker = new DistributedReadWriteLock(qlock, lockData);
@@ -176,15 +177,15 @@ public class Utils {
     return tableNameLock;
   }
 
-  public static Lock getReadLock(Master env, AbstractId tableId, long tid) {
-    return Utils.getLock(env.getContext(), tableId, tid, false);
+  public static Lock getReadLock(Master env, AbstractId<?> id, long tid) {
+    return Utils.getLock(env.getContext(), id, tid, false);
   }
 
   public static void checkNamespaceDoesNotExist(ServerContext context, String namespace,
-      Namespace.ID namespaceId, TableOperation operation)
+      NamespaceId namespaceId, TableOperation operation)
       throws AcceptableThriftTableOperationException {
 
-    Namespace.ID n = Namespaces.lookupNamespaceId(context, namespace);
+    NamespaceId n = Namespaces.lookupNamespaceId(context, namespace);
 
     if (n != null && !n.equals(namespaceId))
       throw new AcceptableThriftTableOperationException(null, namespace, operation,
diff --git a/server/master/src/main/java/org/apache/accumulo/master/tableOps/bulkVer1/BulkImport.java b/server/master/src/main/java/org/apache/accumulo/master/tableOps/bulkVer1/BulkImport.java
index e98885b..83a59f6 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/tableOps/bulkVer1/BulkImport.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/tableOps/bulkVer1/BulkImport.java
@@ -27,11 +27,11 @@ import java.util.concurrent.TimeUnit;
... 5484 lines suppressed ...