You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@accumulo.apache.org by kt...@apache.org on 2019/07/18 19:21:47 UTC

[accumulo] branch master updated: Fix #1210 Create type for time in metadata (#1275)

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

kturner 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 b890a97  Fix #1210 Create type for time in metadata (#1275)
b890a97 is described below

commit b890a97694a8ebf68ede6a2c99e53a48b69391db
Author: hkeebler <49...@users.noreply.github.com>
AuthorDate: Thu Jul 18 15:21:42 2019 -0400

    Fix #1210 Create type for time in metadata (#1275)
    
    Created a MetadataTime object utilizing the TimeType instead of a single
    char M and L; Also removed the MetadataTableUtil.getDataFileSizes which
    was only used by tests.  It was replaced with using
    getFileAndLogEntries(..).getSecond();
---
 .../accumulo/core/metadata/schema/Ample.java       |   2 +-
 .../core/metadata/schema/MetadataTime.java         | 113 +++++++++++++++++++++
 .../core/metadata/schema/TabletMetadata.java       |   6 +-
 .../core/metadata/schema/MetadataTimeTest.java     | 103 +++++++++++++++++++
 .../core/metadata/schema/TabletMetadataTest.java   |   2 +-
 .../apache/accumulo/server/init/Initialize.java    |   6 +-
 .../server/metadata/TabletMutatorBase.java         |   5 +-
 .../apache/accumulo/server/tablets/TabletTime.java |  73 +++++--------
 .../accumulo/server/util/MasterMetadataUtil.java   |   7 +-
 .../accumulo/server/util/MetadataTableUtil.java    |  40 ++------
 .../accumulo/server/tablets/LogicalTimeTest.java   |  10 +-
 .../accumulo/server/tablets/MillisTimeTest.java    |   6 +-
 .../accumulo/server/tablets/TabletTimeTest.java    |  30 +-----
 .../apache/accumulo/master/TabletGroupWatcher.java |   7 +-
 .../apache/accumulo/master/tableOps/TableInfo.java |   7 +-
 .../master/tableOps/create/CreateTable.java        |   3 +-
 .../master/tableOps/create/PopulateMetadata.java   |   6 +-
 .../org/apache/accumulo/tserver/tablet/Tablet.java |  14 +--
 .../apache/accumulo/tserver/tablet/TabletData.java |  15 +--
 .../accumulo/test/functional/SplitRecoveryIT.java  |  12 ++-
 .../test/performance/scan/CollectTabletStats.java  |   6 +-
 21 files changed, 319 insertions(+), 154 deletions(-)

diff --git a/core/src/main/java/org/apache/accumulo/core/metadata/schema/Ample.java b/core/src/main/java/org/apache/accumulo/core/metadata/schema/Ample.java
index 1205a9d..88fefa1 100644
--- a/core/src/main/java/org/apache/accumulo/core/metadata/schema/Ample.java
+++ b/core/src/main/java/org/apache/accumulo/core/metadata/schema/Ample.java
@@ -156,7 +156,7 @@ public interface Ample {
 
     public TabletMutator deleteWal(LogEntry logEntry);
 
-    public TabletMutator putTime(String time);
+    public TabletMutator putTime(MetadataTime time);
 
     public TabletMutator putBulkFile(Ample.FileMeta bulkref, long tid);
 
diff --git a/core/src/main/java/org/apache/accumulo/core/metadata/schema/MetadataTime.java b/core/src/main/java/org/apache/accumulo/core/metadata/schema/MetadataTime.java
new file mode 100644
index 0000000..ed06a29
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/metadata/schema/MetadataTime.java
@@ -0,0 +1,113 @@
+/*
+ * 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.metadata.schema;
+
+import java.util.Objects;
+
+import org.apache.accumulo.core.client.admin.TimeType;
+
+/**
+ * Immutable metadata time object
+ */
+public final class MetadataTime {
+  private final long time;
+  private final TimeType type;
+
+  public MetadataTime(long time, TimeType type) {
+    this.time = time;
+    this.type = type;
+  }
+
+  /**
+   * Creates a MetadataTime object from a string
+   *
+   * @param timestr
+   *          string representation of a metatdata time, ex. "M12345678"
+   * @return a MetadataTime object represented by string
+   */
+
+  public static MetadataTime parse(String timestr) throws IllegalArgumentException {
+
+    if (timestr != null && timestr.length() > 1) {
+      return new MetadataTime(Long.parseLong(timestr.substring(1)), valueOf(timestr.charAt(0)));
+    } else
+      throw new IllegalArgumentException("Unknown metadata time value " + timestr);
+  }
+
+  /**
+   * Converts timetypes to data codes used in the table data implementation
+   *
+   * @param code
+   *          character M or L otherwise exception thrown
+   * @return a TimeType {@link TimeType} represented by code.
+   */
+  public static TimeType valueOf(char code) {
+    switch (code) {
+      case 'M':
+        return TimeType.MILLIS;
+      case 'L':
+        return TimeType.LOGICAL;
+      default:
+        throw new IllegalArgumentException("Unknown time type code : " + code);
+    }
+  }
+
+  /**
+   * @return the single char code of this objects timeType
+   */
+  public static char getCode(TimeType type) {
+    switch (type) {
+      case MILLIS:
+        return 'M';
+      case LOGICAL:
+        return 'L';
+      default: // this should never happen
+        throw new IllegalArgumentException("Unknown time type: " + type);
+    }
+  }
+
+  public char getCode() {
+    return getCode(this.type);
+  }
+
+  public String encode() {
+    return "" + getCode() + time;
+  }
+
+  public TimeType getType() {
+    return type;
+  }
+
+  public long getTime() {
+    return time;
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (o instanceof MetadataTime) {
+      MetadataTime t = (MetadataTime) o;
+      return time == t.getTime() && type == t.getType();
+    }
+    return false;
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(time, type);
+  }
+
+}
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 04923f4..dc30c46 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
@@ -79,7 +79,7 @@ public class TabletMetadata {
   private KeyExtent extent;
   private Location last;
   private String dir;
-  private String time;
+  private MetadataTime time;
   private String cloned;
   private SortedMap<Key,Value> keyValues;
   private OptionalLong flush = OptionalLong.empty();
@@ -208,7 +208,7 @@ public class TabletMetadata {
     return dir;
   }
 
-  public String getTime() {
+  public MetadataTime getTime() {
     ensureFetched(ColumnType.TIME);
     return time;
   }
@@ -283,7 +283,7 @@ public class TabletMetadata {
               te.dir = val;
               break;
             case TIME_QUAL:
-              te.time = val;
+              te.time = MetadataTime.parse(val);
               break;
             case FLUSH_QUAL:
               te.flush = OptionalLong.of(Long.parseLong(val));
diff --git a/core/src/test/java/org/apache/accumulo/core/metadata/schema/MetadataTimeTest.java b/core/src/test/java/org/apache/accumulo/core/metadata/schema/MetadataTimeTest.java
new file mode 100644
index 0000000..e609fd3
--- /dev/null
+++ b/core/src/test/java/org/apache/accumulo/core/metadata/schema/MetadataTimeTest.java
@@ -0,0 +1,103 @@
+/*
+ * 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.metadata.schema;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotEquals;
+
+import org.apache.accumulo.core.client.admin.TimeType;
+import org.junit.Test;
+
+public class MetadataTimeTest {
+
+  @Test(expected = IllegalArgumentException.class)
+  public void testGetInstance_InvalidType() {
+    MetadataTime mTime = MetadataTime.parse("X1234");
+  }
+
+  @Test(expected = IllegalArgumentException.class)
+  public void testGetInstance_Logical_ParseFailure() {
+    MetadataTime mTime = MetadataTime.parse("LABCD");
+  }
+
+  @Test(expected = IllegalArgumentException.class)
+  public void testGetInstance_Millis_ParseFailure() {
+    MetadataTime mTime = MetadataTime.parse("MABCD");
+  }
+
+  @Test
+  public void testGetInstance_Millis() {
+    MetadataTime mTime = new MetadataTime(1234, TimeType.MILLIS);
+    assertEquals(1234, mTime.getTime());
+    assertEquals(TimeType.MILLIS, mTime.getType());
+  }
+
+  @Test
+  public void testGetInstance_Logical() {
+    MetadataTime mTime = new MetadataTime(1234, TimeType.LOGICAL);
+    assertEquals(1234, mTime.getTime());
+    assertEquals(TimeType.LOGICAL, mTime.getType());
+
+  }
+
+  @Test
+  public void testEquality() {
+    assertEquals(new MetadataTime(21, TimeType.MILLIS), MetadataTime.parse("M21"));
+    assertNotEquals(new MetadataTime(21, TimeType.MILLIS), MetadataTime.parse("L21"));
+    assertNotEquals(new MetadataTime(21, TimeType.LOGICAL), new MetadataTime(44, TimeType.LOGICAL));
+  }
+
+  @Test
+  public void testValueOfM() {
+    assertEquals(TimeType.MILLIS, MetadataTime.valueOf('M'));
+  }
+
+  @Test
+  public void testValueOfL() {
+    assertEquals(TimeType.LOGICAL, MetadataTime.valueOf('L'));
+  }
+
+  @Test(expected = IllegalArgumentException.class)
+  public void testValueOfOtherChar() {
+    MetadataTime.valueOf('x');
+  }
+
+  @Test
+  public void testgetCodeforTimeType() {
+    assertEquals('M', MetadataTime.getCode(TimeType.MILLIS));
+    assertEquals('L', MetadataTime.getCode(TimeType.LOGICAL));
+  }
+
+  @Test
+  public void testgetCodeforMillis() {
+    MetadataTime mTime = new MetadataTime(0, TimeType.MILLIS);
+    assertEquals('M', mTime.getCode());
+  }
+
+  @Test
+  public void testgetCodeforLogical() {
+    MetadataTime mTime = new MetadataTime(0, TimeType.LOGICAL);
+    assertEquals('L', mTime.getCode());
+  }
+
+  @Test
+  public void testenCode() {
+    assertEquals("M21", new MetadataTime(21, TimeType.MILLIS).encode());
+    assertEquals("L45678", new MetadataTime(45678, TimeType.LOGICAL).encode());
+  }
+
+}
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 953bd32..c7a06d4 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
@@ -115,7 +115,7 @@ public class TabletMetadataTest {
     assertEquals(extent.getPrevEndRow(), tm.getPrevEndRow());
     assertEquals(extent.getTableId(), tm.getTableId());
     assertTrue(tm.sawPrevEndRow());
-    assertEquals("M123456789", tm.getTime());
+    assertEquals("M123456789", tm.getTime().encode());
     assertEquals(Set.of("sf1", "sf2"), Set.copyOf(tm.getScans()));
   }
 
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 0646cfa..a19b47b 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
@@ -40,6 +40,7 @@ import org.apache.accumulo.core.cli.Help;
 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.client.admin.TimeType;
 import org.apache.accumulo.core.clientImpl.Namespace;
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
 import org.apache.accumulo.core.conf.DefaultConfiguration;
@@ -69,6 +70,7 @@ import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.Fu
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.LogColumnFamily;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.ServerColumnFamily;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.TabletColumnFamily;
+import org.apache.accumulo.core.metadata.schema.MetadataTime;
 import org.apache.accumulo.core.metadata.schema.RootTabletMetadata;
 import org.apache.accumulo.core.replication.ReplicationConstants;
 import org.apache.accumulo.core.replication.ReplicationSchema.StatusSection;
@@ -99,7 +101,6 @@ import org.apache.accumulo.server.replication.StatusCombiner;
 import org.apache.accumulo.server.security.AuditedSecurityOperation;
 import org.apache.accumulo.server.security.SecurityUtil;
 import org.apache.accumulo.server.tables.TableManager;
-import org.apache.accumulo.server.tablets.TabletTime;
 import org.apache.accumulo.server.util.ReplicationTableUtil;
 import org.apache.accumulo.server.util.SystemPropUtil;
 import org.apache.accumulo.server.util.TablePropUtil;
@@ -571,8 +572,7 @@ public class Initialize implements KeywordExecutable {
     Value EMPTY_SIZE = new DataFileValue(0, 0).encodeAsValue();
     Text extent = new Text(TabletsSection.getRow(tablet.tableId, tablet.endRow));
     addEntry(map, extent, DIRECTORY_COLUMN, new Value(tablet.dir.getBytes(UTF_8)));
-    addEntry(map, extent, TIME_COLUMN,
-        new Value((TabletTime.LOGICAL_TIME_ID + "0").getBytes(UTF_8)));
+    addEntry(map, extent, TIME_COLUMN, new Value(new MetadataTime(0, TimeType.LOGICAL).encode()));
     addEntry(map, extent, PREV_ROW_COLUMN, KeyExtent.encodePrevEndRow(tablet.prevEndRow));
     for (String file : tablet.files) {
       addEntry(map, extent, new ColumnFQ(DataFileColumnFamily.NAME, new Text(file)), EMPTY_SIZE);
diff --git a/server/base/src/main/java/org/apache/accumulo/server/metadata/TabletMutatorBase.java b/server/base/src/main/java/org/apache/accumulo/server/metadata/TabletMutatorBase.java
index db7dce7..1eda116 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/metadata/TabletMutatorBase.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/metadata/TabletMutatorBase.java
@@ -26,6 +26,7 @@ import org.apache.accumulo.core.metadata.schema.MetadataSchema;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.DataFileColumnFamily;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.ScanFileColumnFamily;
+import org.apache.accumulo.core.metadata.schema.MetadataTime;
 import org.apache.accumulo.core.metadata.schema.TabletMetadata.LocationType;
 import org.apache.accumulo.core.tabletserver.log.LogEntry;
 import org.apache.accumulo.fate.FateTxId;
@@ -108,9 +109,9 @@ public abstract class TabletMutatorBase implements Ample.TabletMutator {
   }
 
   @Override
-  public Ample.TabletMutator putTime(String time) {
+  public Ample.TabletMutator putTime(MetadataTime time) {
     Preconditions.checkState(updatesEnabled, "Cannot make updates after calling mutate.");
-    TabletsSection.ServerColumnFamily.TIME_COLUMN.put(mutation, new Value(time));
+    TabletsSection.ServerColumnFamily.TIME_COLUMN.put(mutation, new Value(time.encode()));
     return this;
   }
 
diff --git a/server/base/src/main/java/org/apache/accumulo/server/tablets/TabletTime.java b/server/base/src/main/java/org/apache/accumulo/server/tablets/TabletTime.java
index dc5268b..28eddef 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/tablets/TabletTime.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/tablets/TabletTime.java
@@ -21,29 +21,17 @@ import java.util.concurrent.atomic.AtomicLong;
 
 import org.apache.accumulo.core.client.admin.TimeType;
 import org.apache.accumulo.core.data.Mutation;
+import org.apache.accumulo.core.metadata.schema.MetadataTime;
 import org.apache.accumulo.server.data.ServerMutation;
 import org.apache.accumulo.server.util.time.RelativeTime;
 
 public abstract class TabletTime {
-  public static final char LOGICAL_TIME_ID = 'L';
-  public static final char MILLIS_TIME_ID = 'M';
-
-  public static char getTimeID(TimeType timeType) {
-    switch (timeType) {
-      case LOGICAL:
-        return LOGICAL_TIME_ID;
-      case MILLIS:
-        return MILLIS_TIME_ID;
-    }
-
-    throw new IllegalArgumentException("Unknown time type " + timeType);
-  }
 
   public abstract void useMaxTimeFromWALog(long time);
 
-  public abstract String getMetadataValue(long time);
+  public abstract MetadataTime getMetadataTime();
 
-  public abstract String getMetadataValue();
+  public abstract MetadataTime getMetadataTime(long time);
 
   public abstract long setUpdateTimes(List<Mutation> mutations);
 
@@ -56,51 +44,42 @@ public abstract class TabletTime {
     m.setSystemTimestamp(lastCommitTime);
   }
 
-  public static TabletTime getInstance(String metadataValue) {
-    if (metadataValue.charAt(0) == LOGICAL_TIME_ID) {
-      return new LogicalTime(Long.parseLong(metadataValue.substring(1)));
-    } else if (metadataValue.charAt(0) == MILLIS_TIME_ID) {
-      return new MillisTime(Long.parseLong(metadataValue.substring(1)));
-    }
-
-    throw new IllegalArgumentException("Time type unknown : " + metadataValue);
+  public static TabletTime getInstance(MetadataTime metadataTime) throws IllegalArgumentException {
 
+    if (metadataTime.getType().equals(TimeType.LOGICAL)) {
+      return new LogicalTime(metadataTime.getTime());
+    } else if (metadataTime.getType().equals(TimeType.MILLIS)) {
+      return new MillisTime(metadataTime.getTime());
+    } else // this should really never happen here
+      throw new IllegalArgumentException("Time type unknown : " + metadataTime);
   }
 
   public static String maxMetadataTime(String mv1, String mv2) {
     if (mv1 == null && mv2 == null) {
       return null;
     }
-
+    // the parse is used to validate the string
     if (mv1 == null) {
-      checkType(mv2);
-      return mv2;
+      return MetadataTime.parse(mv2).encode();
     }
 
     if (mv2 == null) {
-      checkType(mv1);
-      return mv1;
+      return MetadataTime.parse(mv1).encode();
     }
 
-    if (mv1.charAt(0) != mv2.charAt(0))
-      throw new IllegalArgumentException("Time types differ " + mv1 + " " + mv2);
-    checkType(mv1);
+    MetadataTime mv1Time = MetadataTime.parse(mv1);
+    MetadataTime mv2Time = MetadataTime.parse(mv2);
 
-    long t1 = Long.parseLong(mv1.substring(1));
-    long t2 = Long.parseLong(mv2.substring(1));
+    if (mv1Time.getType() != mv2Time.getType())
+      throw new IllegalArgumentException("Time types differ " + mv1 + " " + mv2);
 
-    if (t1 < t2)
+    if (mv1Time.getTime() < mv2Time.getTime())
       return mv2;
     else
       return mv1;
 
   }
 
-  private static void checkType(String mv1) {
-    if (mv1.charAt(0) != LOGICAL_TIME_ID && mv1.charAt(0) != MILLIS_TIME_ID)
-      throw new IllegalArgumentException("Invalid time type " + mv1);
-  }
-
   static class MillisTime extends TabletTime {
 
     private long lastTime;
@@ -111,13 +90,13 @@ public abstract class TabletTime {
     }
 
     @Override
-    public String getMetadataValue(long time) {
-      return MILLIS_TIME_ID + "" + time;
+    public MetadataTime getMetadataTime() {
+      return getMetadataTime(lastTime);
     }
 
     @Override
-    public String getMetadataValue() {
-      return getMetadataValue(lastTime);
+    public MetadataTime getMetadataTime(long time) {
+      return new MetadataTime(time, TimeType.MILLIS);
     }
 
     @Override
@@ -196,13 +175,13 @@ public abstract class TabletTime {
     }
 
     @Override
-    public String getMetadataValue() {
-      return getMetadataValue(getTime());
+    public MetadataTime getMetadataTime() {
+      return getMetadataTime(getTime());
     }
 
     @Override
-    public String getMetadataValue(long time) {
-      return LOGICAL_TIME_ID + "" + time;
+    public MetadataTime getMetadataTime(long time) {
+      return new MetadataTime(time, TimeType.LOGICAL);
     }
 
     @Override
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 7a88ba6..5fd79a2 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
@@ -45,6 +45,7 @@ import org.apache.accumulo.core.metadata.schema.Ample.TabletMutator;
 import org.apache.accumulo.core.metadata.schema.DataFileValue;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.DataFileColumnFamily;
+import org.apache.accumulo.core.metadata.schema.MetadataTime;
 import org.apache.accumulo.core.metadata.schema.TabletMetadata.LocationType;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.util.ColumnFQ;
@@ -63,7 +64,7 @@ public class MasterMetadataUtil {
 
   public static void addNewTablet(ServerContext context, KeyExtent extent, String path,
       TServerInstance location, Map<FileRef,DataFileValue> datafileSizes,
-      Map<Long,? extends Collection<FileRef>> bulkLoadedFiles, String time, long lastFlushID,
+      Map<Long,? extends Collection<FileRef>> bulkLoadedFiles, MetadataTime time, long lastFlushID,
       long lastCompactID, ZooLock zooLock) {
 
     TabletMutator tablet = context.getAmple().mutateTablet(extent);
@@ -239,7 +240,7 @@ public class MasterMetadataUtil {
    *
    */
   public static void updateTabletDataFile(ServerContext context, KeyExtent extent, FileRef path,
-      FileRef mergeFile, DataFileValue dfv, String time, Set<FileRef> filesInUseByScans,
+      FileRef mergeFile, DataFileValue dfv, MetadataTime time, Set<FileRef> filesInUseByScans,
       String address, ZooLock zooLock, Set<String> unusedWalLogs, TServerInstance lastLocation,
       long flushId) {
     if (extent.isRootTablet()) {
@@ -267,7 +268,7 @@ public class MasterMetadataUtil {
    *
    */
   private static void updateForTabletDataFile(ServerContext context, KeyExtent extent, FileRef path,
-      FileRef mergeFile, DataFileValue dfv, String time, Set<FileRef> filesInUseByScans,
+      FileRef mergeFile, DataFileValue dfv, MetadataTime time, Set<FileRef> filesInUseByScans,
       String address, ZooLock zooLock, Set<String> unusedWalLogs, TServerInstance lastLocation,
       long flushId) {
 
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 18d50ca..1dedfc8 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
@@ -51,13 +51,13 @@ 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.client.admin.TimeType;
 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.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;
@@ -71,6 +71,7 @@ import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.ChoppedColumnFamily;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.ClonedColumnFamily;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.DataFileColumnFamily;
+import org.apache.accumulo.core.metadata.schema.MetadataTime;
 import org.apache.accumulo.core.metadata.schema.TabletDeletedException;
 import org.apache.accumulo.core.metadata.schema.TabletMetadata;
 import org.apache.accumulo.core.metadata.schema.TabletsMetadata;
@@ -92,7 +93,6 @@ import org.apache.accumulo.server.metadata.ServerAmpleImpl;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.Text;
-import org.apache.zookeeper.KeeperException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -184,7 +184,8 @@ public class MetadataTableUtil {
   }
 
   public static void updateTabletDataFile(long tid, KeyExtent extent,
-      Map<FileRef,DataFileValue> estSizes, String time, ServerContext context, ZooLock zooLock) {
+      Map<FileRef,DataFileValue> estSizes, MetadataTime time, ServerContext context,
+      ZooLock zooLock) {
     TabletMutator tablet = context.getAmple().mutateTablet(extent);
     tablet.putTime(time);
     estSizes.forEach(tablet::putFile);
@@ -204,12 +205,12 @@ public class MetadataTableUtil {
     tablet.mutate();
   }
 
-  public static void addTablet(KeyExtent extent, String path, ServerContext context, char timeType,
-      ZooLock zooLock) {
+  public static void addTablet(KeyExtent extent, String path, ServerContext context,
+      TimeType timeType, ZooLock zooLock) {
     TabletMutator tablet = context.getAmple().mutateTablet(extent);
     tablet.putPrevEndRow(extent.getPrevEndRow());
     tablet.putDir(path);
-    tablet.putTime(timeType + "0");
+    tablet.putTime(new MetadataTime(0, timeType));
     tablet.putZooLock(zooLock);
     tablet.mutate();
 
@@ -243,30 +244,6 @@ public class MetadataTableUtil {
     tabletMutator.mutate();
   }
 
-  public static SortedMap<FileRef,DataFileValue> getDataFileSizes(KeyExtent extent,
-      ServerContext context) {
-    TreeMap<FileRef,DataFileValue> sizes = new TreeMap<>();
-
-    try (Scanner mdScanner = new ScannerImpl(context, MetadataTable.ID, Authorizations.EMPTY)) {
-      mdScanner.fetchColumnFamily(DataFileColumnFamily.NAME);
-      Text row = extent.getMetadataEntry();
-
-      Key endKey = new Key(row, DataFileColumnFamily.NAME, new Text(""));
-      endKey = endKey.followingKey(PartialKey.ROW_COLFAM);
-
-      mdScanner.setRange(new Range(new Key(row), endKey));
-      for (Entry<Key,Value> entry : mdScanner) {
-
-        if (!entry.getKey().getRow().equals(row))
-          break;
-        DataFileValue dfv = new DataFileValue(entry.getValue().get());
-        sizes.put(new FileRef(context.getVolumeManager(), entry.getKey()), dfv);
-      }
-
-      return sizes;
-    }
-  }
-
   public static void rollBackSplit(Text metadataEntry, Text oldPrevEndRow, ServerContext context,
       ZooLock zooLock) {
     KeyExtent ke = new KeyExtent(metadataEntry, oldPrevEndRow);
@@ -449,8 +426,7 @@ public class MetadataTableUtil {
   }
 
   public static Pair<List<LogEntry>,SortedMap<FileRef,DataFileValue>>
-      getFileAndLogEntries(ServerContext context, KeyExtent extent)
-          throws KeeperException, InterruptedException, IOException {
+      getFileAndLogEntries(ServerContext context, KeyExtent extent) throws IOException {
     ArrayList<LogEntry> result = new ArrayList<>();
     TreeMap<FileRef,DataFileValue> sizes = new TreeMap<>();
 
diff --git a/server/base/src/test/java/org/apache/accumulo/server/tablets/LogicalTimeTest.java b/server/base/src/test/java/org/apache/accumulo/server/tablets/LogicalTimeTest.java
index defb7da..ae560e3 100644
--- a/server/base/src/test/java/org/apache/accumulo/server/tablets/LogicalTimeTest.java
+++ b/server/base/src/test/java/org/apache/accumulo/server/tablets/LogicalTimeTest.java
@@ -24,6 +24,7 @@ import static org.junit.Assert.assertEquals;
 import java.util.List;
 
 import org.apache.accumulo.core.data.Mutation;
+import org.apache.accumulo.core.metadata.schema.MetadataTime;
 import org.apache.accumulo.server.data.ServerMutation;
 import org.apache.accumulo.server.tablets.TabletTime.LogicalTime;
 import org.junit.Before;
@@ -35,24 +36,25 @@ public class LogicalTimeTest {
 
   @Before
   public void setUp() {
-    ltime = (LogicalTime) TabletTime.getInstance("L1234");
+    MetadataTime mTime = MetadataTime.parse("L1234");
+    ltime = (LogicalTime) TabletTime.getInstance(mTime);
   }
 
   @Test
   public void testGetMetadataValue() {
-    assertEquals("L1234", ltime.getMetadataValue());
+    assertEquals("L1234", ltime.getMetadataTime().encode());
   }
 
   @Test
   public void testUseMaxTimeFromWALog_Update() {
     ltime.useMaxTimeFromWALog(5678L);
-    assertEquals("L5678", ltime.getMetadataValue());
+    assertEquals("L5678", ltime.getMetadataTime().encode());
   }
 
   @Test
   public void testUseMaxTimeFromWALog_NoUpdate() {
     ltime.useMaxTimeFromWALog(0L);
-    assertEquals("L1234", ltime.getMetadataValue());
+    assertEquals("L1234", ltime.getMetadataTime().encode());
   }
 
   @Test
diff --git a/server/base/src/test/java/org/apache/accumulo/server/tablets/MillisTimeTest.java b/server/base/src/test/java/org/apache/accumulo/server/tablets/MillisTimeTest.java
index 1fee2a0..272cb85 100644
--- a/server/base/src/test/java/org/apache/accumulo/server/tablets/MillisTimeTest.java
+++ b/server/base/src/test/java/org/apache/accumulo/server/tablets/MillisTimeTest.java
@@ -42,19 +42,19 @@ public class MillisTimeTest {
 
   @Test
   public void testGetMetadataValue() {
-    assertEquals("M1234", mtime.getMetadataValue());
+    assertEquals("M1234", mtime.getMetadataTime().encode());
   }
 
   @Test
   public void testUseMaxTimeFromWALog_Yes() {
     mtime.useMaxTimeFromWALog(5678L);
-    assertEquals("M5678", mtime.getMetadataValue());
+    assertEquals("M5678", mtime.getMetadataTime().encode());
   }
 
   @Test
   public void testUseMaxTimeFromWALog_No() {
     mtime.useMaxTimeFromWALog(0L);
-    assertEquals("M1234", mtime.getMetadataValue());
+    assertEquals("M1234", mtime.getMetadataTime().encode());
   }
 
   @Test
diff --git a/server/base/src/test/java/org/apache/accumulo/server/tablets/TabletTimeTest.java b/server/base/src/test/java/org/apache/accumulo/server/tablets/TabletTimeTest.java
index 1bbf7c2..2ea0295 100644
--- a/server/base/src/test/java/org/apache/accumulo/server/tablets/TabletTimeTest.java
+++ b/server/base/src/test/java/org/apache/accumulo/server/tablets/TabletTimeTest.java
@@ -23,6 +23,7 @@ import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNull;
 
 import org.apache.accumulo.core.client.admin.TimeType;
+import org.apache.accumulo.core.metadata.schema.MetadataTime;
 import org.apache.accumulo.server.data.ServerMutation;
 import org.apache.accumulo.server.tablets.TabletTime.LogicalTime;
 import org.apache.accumulo.server.tablets.TabletTime.MillisTime;
@@ -39,12 +40,6 @@ public class TabletTimeTest {
   }
 
   @Test
-  public void testGetTimeID() {
-    assertEquals('L', TabletTime.getTimeID(TimeType.LOGICAL));
-    assertEquals('M', TabletTime.getTimeID(TimeType.MILLIS));
-  }
-
-  @Test
   public void testSetSystemTimes() {
     ServerMutation m = createMock(ServerMutation.class);
     long lastCommitTime = 1234L;
@@ -56,31 +51,16 @@ public class TabletTimeTest {
 
   @Test
   public void testGetInstance_Logical() {
-    TabletTime t = TabletTime.getInstance("L1234");
+    TabletTime t = TabletTime.getInstance(new MetadataTime(1234, TimeType.LOGICAL));
     assertEquals(LogicalTime.class, t.getClass());
-    assertEquals("L1234", t.getMetadataValue());
+    assertEquals("L1234", t.getMetadataTime().encode());
   }
 
   @Test
   public void testGetInstance_Millis() {
-    TabletTime t = TabletTime.getInstance("M1234");
+    TabletTime t = TabletTime.getInstance(new MetadataTime(1234, TimeType.MILLIS));
     assertEquals(MillisTime.class, t.getClass());
-    assertEquals("M1234", t.getMetadataValue());
-  }
-
-  @Test(expected = IllegalArgumentException.class)
-  public void testGetInstance_InvalidType() {
-    TabletTime.getInstance("X1234");
-  }
-
-  @Test(expected = NumberFormatException.class)
-  public void testGetInstance_Logical_ParseFailure() {
-    TabletTime.getInstance("LABCD");
-  }
-
-  @Test(expected = NumberFormatException.class)
-  public void testGetInstance_Millis_ParseFailure() {
-    TabletTime.getInstance("MABCD");
+    assertEquals("M1234", t.getMetadataTime().encode());
   }
 
   @Test
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 2ff8f8c..8060885 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
@@ -61,6 +61,7 @@ import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.Ch
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.CurrentLocationColumnFamily;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.DataFileColumnFamily;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.FutureLocationColumnFamily;
+import org.apache.accumulo.core.metadata.schema.MetadataTime;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.tabletserver.thrift.NotServingTabletException;
 import org.apache.accumulo.core.util.Daemon;
@@ -582,7 +583,7 @@ abstract class TabletGroupWatcher extends Daemon {
     KeyExtent extent = info.getExtent();
     String targetSystemTable = extent.isMeta() ? RootTable.NAME : MetadataTable.NAME;
     Master.log.debug("Deleting tablets for {}", extent);
-    char timeType = '\0';
+    MetadataTime metadataTime = null;
     KeyExtent followingTablet = null;
     if (extent.getEndRow() != null) {
       Key nextExtent = new Key(extent.getEndRow()).followingKey(PartialKey.ROW);
@@ -615,7 +616,7 @@ abstract class TabletGroupWatcher extends Daemon {
             datafiles.clear();
           }
         } else if (TabletsSection.ServerColumnFamily.TIME_COLUMN.hasColumns(key)) {
-          timeType = entry.getValue().toString().charAt(0);
+          metadataTime = MetadataTime.parse(entry.getValue().toString());
         } else if (key.compareColumnFamily(TabletsSection.CurrentLocationColumnFamily.NAME) == 0) {
           throw new IllegalStateException(
               "Tablet " + key.getRow() + " is assigned during a merge!");
@@ -669,7 +670,7 @@ abstract class TabletGroupWatcher extends Daemon {
             + Path.SEPARATOR + extent.getTableId() + Constants.DEFAULT_TABLET_LOCATION;
         MetadataTableUtil.addTablet(
             new KeyExtent(extent.getTableId(), null, extent.getPrevEndRow()), tdir,
-            master.getContext(), timeType, this.master.masterLock);
+            master.getContext(), metadataTime.getType(), this.master.masterLock);
       }
     } catch (RuntimeException | TableNotFoundException ex) {
       throw new AccumuloException(ex);
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 c108dd4..cf11383 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,6 +20,7 @@ import java.io.Serializable;
 import java.util.Map;
 
 import org.apache.accumulo.core.client.admin.InitialTableState;
+import org.apache.accumulo.core.client.admin.TimeType;
 import org.apache.accumulo.core.data.NamespaceId;
 import org.apache.accumulo.core.data.TableId;
 
@@ -31,7 +32,7 @@ public class TableInfo implements Serializable {
   private TableId tableId;
   private NamespaceId namespaceId;
 
-  private char timeType;
+  private TimeType timeType;
   private String user;
 
   // Record requested initial state at creation
@@ -69,11 +70,11 @@ public class TableInfo implements Serializable {
     this.namespaceId = namespaceId;
   }
 
-  public char getTimeType() {
+  public TimeType getTimeType() {
     return timeType;
   }
 
-  public void setTimeType(char timeType) {
+  public void setTimeType(TimeType timeType) {
     this.timeType = timeType;
   }
 
diff --git a/server/master/src/main/java/org/apache/accumulo/master/tableOps/create/CreateTable.java b/server/master/src/main/java/org/apache/accumulo/master/tableOps/create/CreateTable.java
index 5d2327f..a3e9ab0 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/tableOps/create/CreateTable.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/tableOps/create/CreateTable.java
@@ -28,7 +28,6 @@ import org.apache.accumulo.master.Master;
 import org.apache.accumulo.master.tableOps.MasterRepo;
 import org.apache.accumulo.master.tableOps.TableInfo;
 import org.apache.accumulo.master.tableOps.Utils;
-import org.apache.accumulo.server.tablets.TabletTime;
 
 public class CreateTable extends MasterRepo {
   private static final long serialVersionUID = 1L;
@@ -40,7 +39,7 @@ public class CreateTable extends MasterRepo {
       NamespaceId namespaceId) {
     tableInfo = new TableInfo();
     tableInfo.setTableName(tableName);
-    tableInfo.setTimeType(TabletTime.getTimeID(timeType));
+    tableInfo.setTimeType(timeType);
     tableInfo.setUser(user);
     tableInfo.props = props;
     tableInfo.setNamespaceId(namespaceId);
diff --git a/server/master/src/main/java/org/apache/accumulo/master/tableOps/create/PopulateMetadata.java b/server/master/src/main/java/org/apache/accumulo/master/tableOps/create/PopulateMetadata.java
index 70ce1ed..a5ff669 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/tableOps/create/PopulateMetadata.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/tableOps/create/PopulateMetadata.java
@@ -24,11 +24,13 @@ import java.util.SortedSet;
 
 import org.apache.accumulo.core.client.BatchWriter;
 import org.apache.accumulo.core.client.MutationsRejectedException;
+import org.apache.accumulo.core.client.admin.TimeType;
 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;
+import org.apache.accumulo.core.metadata.schema.MetadataTime;
 import org.apache.accumulo.fate.Repo;
 import org.apache.accumulo.fate.zookeeper.ZooLock;
 import org.apache.accumulo.master.Master;
@@ -78,7 +80,7 @@ class PopulateMetadata extends MasterRepo {
   }
 
   private void writeSplitsToMetadataTable(ServerContext ctx, TableId tableId,
-      SortedSet<Text> splits, Map<Text,Text> data, char timeType, ZooLock lock, BatchWriter bw)
+      SortedSet<Text> splits, Map<Text,Text> data, TimeType timeType, ZooLock lock, BatchWriter bw)
       throws MutationsRejectedException {
     Text prevSplit = null;
     Value dirValue;
@@ -88,7 +90,7 @@ class PopulateMetadata extends MasterRepo {
           (split == null) ? new Value(tableInfo.defaultTabletDir) : new Value(data.get(split));
       MetadataSchema.TabletsSection.ServerColumnFamily.DIRECTORY_COLUMN.put(mut, dirValue);
       MetadataSchema.TabletsSection.ServerColumnFamily.TIME_COLUMN.put(mut,
-          new Value(timeType + "0"));
+          new Value(new MetadataTime(0, timeType).encode()));
       MetadataTableUtil.putLockID(ctx, lock, mut);
       prevSplit = split;
       bw.addMutation(mut);
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Tablet.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Tablet.java
index 360355e..9a23a43 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Tablet.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Tablet.java
@@ -79,6 +79,7 @@ import org.apache.accumulo.core.master.thrift.TabletLoadState;
 import org.apache.accumulo.core.metadata.MetadataTable;
 import org.apache.accumulo.core.metadata.RootTable;
 import org.apache.accumulo.core.metadata.schema.DataFileValue;
+import org.apache.accumulo.core.metadata.schema.MetadataTime;
 import org.apache.accumulo.core.protobuf.ProtobufUtil;
 import org.apache.accumulo.core.replication.ReplicationConfigurationUtil;
 import org.apache.accumulo.core.security.Authorizations;
@@ -301,7 +302,8 @@ public class Tablet {
   }
 
   public Tablet(final TabletServer tabletServer, final KeyExtent extent,
-      final TabletResourceManager trm, TabletData data) throws IOException {
+      final TabletResourceManager trm, TabletData data)
+      throws IOException, IllegalArgumentException {
 
     this.tabletServer = tabletServer;
     this.context = tabletServer.getContext();
@@ -2255,7 +2257,7 @@ public class Tablet {
       log.debug("Files for low split {} {}", low, lowDatafileSizes.keySet());
       log.debug("Files for high split {} {}", high, highDatafileSizes.keySet());
 
-      String time = tabletTime.getMetadataValue();
+      MetadataTime time = tabletTime.getMetadataTime();
 
       MetadataTableUtil.splitTablet(high, extent.getPrevEndRow(), splitRatio,
           getTabletServer().getContext(), getTabletServer().getLock());
@@ -2726,7 +2728,7 @@ public class Tablet {
       }
 
       MetadataTableUtil.updateTabletDataFile(tid, extent, paths,
-          tabletTime.getMetadataValue(persistedTime), getTabletServer().getContext(),
+          tabletTime.getMetadataTime(persistedTime), getTabletServer().getContext(),
           getTabletServer().getLock());
     }
 
@@ -2739,10 +2741,10 @@ public class Tablet {
         persistedTime = maxCommittedTime;
       }
 
-      String time = tabletTime.getMetadataValue(persistedTime);
       MasterMetadataUtil.updateTabletDataFile(getTabletServer().getContext(), extent, newDatafile,
-          absMergeFile, dfv, time, filesInUseByScans, tabletServer.getClientAddressString(),
-          tabletServer.getLock(), unusedWalLogs, lastLocation, flushId);
+          absMergeFile, dfv, tabletTime.getMetadataTime(persistedTime), filesInUseByScans,
+          tabletServer.getClientAddressString(), tabletServer.getLock(), unusedWalLogs,
+          lastLocation, flushId);
     }
 
   }
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/TabletData.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/TabletData.java
index 87b39fd..5aaff34 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/TabletData.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/TabletData.java
@@ -33,6 +33,7 @@ import java.util.Map.Entry;
 import java.util.SortedMap;
 import java.util.TreeMap;
 
+import org.apache.accumulo.core.client.admin.TimeType;
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Value;
@@ -47,6 +48,7 @@ import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.La
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.LogColumnFamily;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.ScanFileColumnFamily;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.ServerColumnFamily;
+import org.apache.accumulo.core.metadata.schema.MetadataTime;
 import org.apache.accumulo.core.metadata.schema.TabletMetadata;
 import org.apache.accumulo.core.tabletserver.log.LogEntry;
 import org.apache.accumulo.server.ServerContext;
@@ -54,7 +56,6 @@ import org.apache.accumulo.server.fs.FileRef;
 import org.apache.accumulo.server.fs.VolumeManager;
 import org.apache.accumulo.server.fs.VolumeUtil;
 import org.apache.accumulo.server.master.state.TServerInstance;
-import org.apache.accumulo.server.tablets.TabletTime;
 import org.apache.accumulo.server.util.MetadataTableUtil;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
@@ -69,7 +70,7 @@ import org.slf4j.LoggerFactory;
 public class TabletData {
   private static Logger log = LoggerFactory.getLogger(TabletData.class);
 
-  private String time = null;
+  private MetadataTime time = null;
   private SortedMap<FileRef,DataFileValue> dataFiles = new TreeMap<>();
   private List<LogEntry> logEntries = new ArrayList<>();
   private HashSet<FileRef> scanFiles = new HashSet<>();
@@ -95,7 +96,7 @@ public class TabletData {
       }
       if (ServerColumnFamily.TIME_COLUMN.hasColumns(entry.getKey())) {
         if (time == null) {
-          time = value.toString();
+          time = MetadataTime.parse(value.toString());
         }
       } else if (DataFileColumnFamily.NAME.equals(family)) {
         FileRef ref = new FileRef(fs, key);
@@ -124,7 +125,7 @@ public class TabletData {
     }
     if (time == null && dataFiles.isEmpty() && extent.equals(RootTable.OLD_EXTENT)) {
       // recovery... old root tablet has no data, so time doesn't matter:
-      time = TabletTime.LOGICAL_TIME_ID + "" + Long.MIN_VALUE;
+      time = new MetadataTime(Long.MIN_VALUE, TimeType.LOGICAL);
     }
   }
 
@@ -160,7 +161,7 @@ public class TabletData {
         }
       }
       if (maxTime > rtime) {
-        time = TabletTime.LOGICAL_TIME_ID + "" + maxTime;
+        time = new MetadataTime(maxTime, TimeType.LOGICAL);
         rtime = maxTime;
       }
     }
@@ -174,7 +175,7 @@ public class TabletData {
 
   // Data pulled from an existing tablet to make a split
   public TabletData(String tabletDirectory, SortedMap<FileRef,DataFileValue> highDatafileSizes,
-      String time, long lastFlushID, long lastCompactID, TServerInstance lastLocation,
+      MetadataTime time, long lastFlushID, long lastCompactID, TServerInstance lastLocation,
       Map<Long,List<FileRef>> bulkIngestedFiles) {
     this.directory = tabletDirectory;
     this.dataFiles = highDatafileSizes;
@@ -186,7 +187,7 @@ public class TabletData {
     this.splitTime = System.currentTimeMillis();
   }
 
-  public String getTime() {
+  public MetadataTime getTime() {
     return time;
   }
 
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/SplitRecoveryIT.java b/test/src/main/java/org/apache/accumulo/test/functional/SplitRecoveryIT.java
index 4ecb5ef..540f673 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/SplitRecoveryIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/SplitRecoveryIT.java
@@ -32,6 +32,7 @@ import java.util.TreeMap;
 
 import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.Scanner;
+import org.apache.accumulo.core.client.admin.TimeType;
 import org.apache.accumulo.core.clientImpl.ScannerImpl;
 import org.apache.accumulo.core.clientImpl.Writer;
 import org.apache.accumulo.core.conf.SiteConfiguration;
@@ -45,6 +46,7 @@ import org.apache.accumulo.core.metadata.MetadataTable;
 import org.apache.accumulo.core.metadata.schema.DataFileValue;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.DataFileColumnFamily;
+import org.apache.accumulo.core.metadata.schema.MetadataTime;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.util.ColumnFQ;
 import org.apache.accumulo.fate.zookeeper.ZooLock;
@@ -57,7 +59,6 @@ import org.apache.accumulo.server.ServerContext;
 import org.apache.accumulo.server.fs.FileRef;
 import org.apache.accumulo.server.master.state.Assignment;
 import org.apache.accumulo.server.master.state.TServerInstance;
-import org.apache.accumulo.server.tablets.TabletTime;
 import org.apache.accumulo.server.util.MasterMetadataUtil;
 import org.apache.accumulo.server.util.MetadataTableUtil;
 import org.apache.accumulo.server.zookeeper.TransactionWatcher;
@@ -147,7 +148,7 @@ public class SplitRecoveryIT extends ConfigurableMacBase {
 
       String tdir =
           ServerConstants.getTablesDirs(context)[0] + "/" + extent.getTableId() + "/dir_" + i;
-      MetadataTableUtil.addTablet(extent, tdir, context, TabletTime.LOGICAL_TIME_ID, zl);
+      MetadataTableUtil.addTablet(extent, tdir, context, TimeType.LOGICAL, zl);
       SortedMap<FileRef,DataFileValue> mapFiles = new TreeMap<>();
       mapFiles.put(new FileRef(tdir + "/" + RFile.EXTENSION + "_000_000"),
           new DataFileValue(1000017 + i, 10000 + i));
@@ -157,7 +158,8 @@ public class SplitRecoveryIT extends ConfigurableMacBase {
       }
       int tid = 0;
       TransactionWatcher.ZooArbitrator.start(context, Constants.BULK_ARBITRATOR_TYPE, tid);
-      MetadataTableUtil.updateTabletDataFile(tid, extent, mapFiles, "L0", context, zl);
+      MetadataTableUtil.updateTabletDataFile(tid, extent, mapFiles,
+          new MetadataTime(0, TimeType.LOGICAL), context, zl);
     }
 
     KeyExtent extent = extents[extentToSplit];
@@ -192,7 +194,7 @@ public class SplitRecoveryIT extends ConfigurableMacBase {
       Map<Long,? extends Collection<FileRef>> bulkFiles =
           MetadataTableUtil.getBulkFilesLoaded(context, extent);
       MasterMetadataUtil.addNewTablet(context, low, "/lowDir", instance, lowDatafileSizes,
-          bulkFiles, TabletTime.LOGICAL_TIME_ID + "0", -1L, -1L, zl);
+          bulkFiles, new MetadataTime(0, TimeType.LOGICAL), -1L, -1L, zl);
     }
     if (steps >= 2) {
       MetadataTableUtil.finishSplit(high, highDatafileSizes, highDatafilesToRemove, context, zl);
@@ -267,7 +269,7 @@ public class SplitRecoveryIT extends ConfigurableMacBase {
       }
 
       SortedMap<FileRef,DataFileValue> fixedMapFiles =
-          MetadataTableUtil.getDataFileSizes(extent, context);
+          MetadataTableUtil.getFileAndLogEntries(context, extent).getSecond();
       verifySame(expectedMapFiles, fixedMapFiles);
     }
   }
diff --git a/test/src/main/java/org/apache/accumulo/test/performance/scan/CollectTabletStats.java b/test/src/main/java/org/apache/accumulo/test/performance/scan/CollectTabletStats.java
index 092c333..118e164 100644
--- a/test/src/main/java/org/apache/accumulo/test/performance/scan/CollectTabletStats.java
+++ b/test/src/main/java/org/apache/accumulo/test/performance/scan/CollectTabletStats.java
@@ -388,8 +388,10 @@ public class CollectTabletStats {
     return tabletsToTest;
   }
 
-  private static List<FileRef> getTabletFiles(ServerContext context, KeyExtent ke) {
-    return new ArrayList<>(MetadataTableUtil.getDataFileSizes(ke, context).keySet());
+  private static List<FileRef> getTabletFiles(ServerContext context, KeyExtent ke)
+      throws IOException {
+    return new ArrayList<>(
+        MetadataTableUtil.getFileAndLogEntries(context, ke).getSecond().keySet());
   }
 
   private static void reportHdfsBlockLocations(ServerContext context, List<FileRef> files)