You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by st...@apache.org on 2017/07/06 03:11:34 UTC

[2/3] hbase git commit: Revert "HBASE-14070 - Core HLC" Revert a push too-early

http://git-wip-us.apache.org/repos/asf/hbase/blob/c5abb6ca/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java
index 4dd62f9..11301d8 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java
@@ -31,9 +31,14 @@ import java.util.concurrent.locks.ReentrantLock;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hbase.*;
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.CellComparator;
+import org.apache.hadoop.hbase.CellUtil;
+import org.apache.hadoop.hbase.DoNotRetryIOException;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.KeyValueUtil;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.client.HTable;
 import org.apache.hadoop.hbase.client.IsolationLevel;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.executor.ExecutorService;
@@ -89,7 +94,6 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
   private final int minVersions;
   private final long maxRowSize;
   private final long cellsPerHeartbeatCheck;
-  private final TimestampType timestampType;
 
   // 1) Collects all the KVHeap that are eagerly getting closed during the
   //    course of a scan
@@ -172,17 +176,8 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
     int numCol = columns == null ? 0 : columns.size();
     explicitColumnQuery = numCol > 0;
     this.scan = scan;
-
-    this.now = this.store != null ? this.store.getClock().now() :
-        new Clock.System().now();
-    this.timestampType = this.store != null ?
-        this.store.getClock().getTimestampType() : TimestampType.PHYSICAL;
-    // Convert to milliseconds before subtracting time
-    long diff = this.timestampType.toEpochTimeMillisFromTimestamp(now) - scanInfo.getTtl();
-    // Prevent overflow if diff is negative and timestampType is HYBRID
-    diff = diff > 0 ? timestampType.fromEpochTimeMillisToTimestamp(diff) : 0L;
-    this.oldestUnexpiredTS = scan.isRaw() ? 0L : diff;
-
+    this.now = EnvironmentEdgeManager.currentTime();
+    this.oldestUnexpiredTS = scan.isRaw() ? 0L : now - scanInfo.getTtl();
     this.minVersions = scanInfo.getMinVersions();
 
      // We look up row-column Bloom filters for multi-column queries as part of
@@ -210,7 +205,6 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
       // readType is default if the scan keeps running for a long time.
       this.scanUsePread = this.readType != Scan.ReadType.STREAM;
     }
-
     this.preadMaxBytes = scanInfo.getPreadMaxBytes();
     this.cellsPerHeartbeatCheck = scanInfo.getCellsPerTimeoutCheck();
     // Parallel seeking is on if the config allows and more there is more than one store file.
@@ -243,8 +237,8 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
     if (columns != null && scan.isRaw()) {
       throw new DoNotRetryIOException("Cannot specify any column for a raw scan");
     }
-    matcher = UserScanQueryMatcher.create(scan, scanInfo, columns, oldestUnexpiredTS, now, store
-        .getCoprocessorHost());
+    matcher = UserScanQueryMatcher.create(scan, scanInfo, columns, oldestUnexpiredTS, now,
+      store.getCoprocessorHost());
 
     this.store.addChangedReaderObserver(this);
 
@@ -320,12 +314,12 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
       // use legacy query matcher since we do not consider the scan object in our code. Only used to
       // keep compatibility for coprocessor.
       matcher = LegacyScanQueryMatcher.create(scan, scanInfo, null, scanType, smallestReadPoint,
-        earliestPutTs, oldestUnexpiredTS, now, dropDeletesFromRow,
-          dropDeletesToRow, store.getCoprocessorHost());
+        earliestPutTs, oldestUnexpiredTS, now, dropDeletesFromRow, dropDeletesToRow,
+        store.getCoprocessorHost());
     } else {
       matcher = CompactionScanQueryMatcher.create(scanInfo, scanType, smallestReadPoint,
-        earliestPutTs, oldestUnexpiredTS, now, dropDeletesFromRow,
-          dropDeletesToRow, store.getCoprocessorHost());
+        earliestPutTs, oldestUnexpiredTS, now, dropDeletesFromRow, dropDeletesToRow,
+        store.getCoprocessorHost());
     }
 
     // Filter the list of scanners using Bloom filters, time range, TTL, etc.
@@ -349,26 +343,6 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
   }
 
   @VisibleForTesting
-  StoreScanner(final Store store, final Scan scan, ScanInfo scanInfo,
-      ScanType scanType, final NavigableSet<byte[]> columns,
-      final List<KeyValueScanner> scanners) throws IOException {
-    this(store, scan, scanInfo, scanType, columns, scanners,
-        HConstants.LATEST_TIMESTAMP,
-        // 0 is passed as readpoint because the test bypasses Store
-        0);
-  }
-
-  @VisibleForTesting
-  StoreScanner(final Store store, final Scan scan, ScanInfo scanInfo,
-      ScanType scanType, final NavigableSet<byte[]> columns,
-      final List<KeyValueScanner> scanners, long earliestPutTs)
-      throws IOException {
-    this(store, scan, scanInfo, scanType, columns, scanners, earliestPutTs,
-        // 0 is passed as readpoint because the test bypasses Store
-        0);
-  }
-
-  @VisibleForTesting
   StoreScanner(final Scan scan, ScanInfo scanInfo,
     ScanType scanType, final NavigableSet<byte[]> columns,
     final List<? extends KeyValueScanner> scanners, long earliestPutTs)
@@ -378,41 +352,14 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
       0);
   }
 
-  public StoreScanner(final Store store, final Scan scan, ScanInfo scanInfo, ScanType scanType,
-      final NavigableSet<byte[]> columns, final List<KeyValueScanner> scanners, long earliestPutTs,
-      long readPt) throws IOException {
-    this(store, scan, scanInfo, columns, readPt, scan.getCacheBlocks(), scanType);
-    if (scanType == ScanType.USER_SCAN) {
-      this.matcher = UserScanQueryMatcher.create(scan, scanInfo, columns, oldestUnexpiredTS, now, null);
-    } else {
-      if (scan.hasFilter() || (scan.getStartRow() != null && scan.getStartRow().length > 0)
-          || (scan.getStopRow() != null && scan.getStopRow().length > 0)
-          || !scan.getTimeRange().isAllTime() || columns != null) {
-        // use legacy query matcher since we do not consider the scan object in our code. Only used
-        // to keep compatibility for coprocessor.
-        matcher = LegacyScanQueryMatcher.create(scan, scanInfo, columns, scanType, Long.MAX_VALUE,
-            earliestPutTs, oldestUnexpiredTS, now, null, null,
-            store.getCoprocessorHost());
-      } else {
-        this.matcher = CompactionScanQueryMatcher.create(scanInfo, scanType, Long.MAX_VALUE,
-            earliestPutTs, oldestUnexpiredTS, now, null, null,
-            null);
-      }
-    }
-
-    // Seek all scanners to the initial key
-    seekScanners(scanners, matcher.getStartKey(), false, parallelSeekEnabled);
-    addCurrentScanners(scanners);
-    resetKVHeap(scanners, scanInfo.getComparator());
-  }
-
   public StoreScanner(final Scan scan, ScanInfo scanInfo, ScanType scanType,
       final NavigableSet<byte[]> columns, final List<? extends KeyValueScanner> scanners, long earliestPutTs,
       long readPt) throws IOException {
     this(null, scan, scanInfo, columns, readPt,
         scanType == ScanType.USER_SCAN ? scan.getCacheBlocks() : false, scanType);
     if (scanType == ScanType.USER_SCAN) {
-      this.matcher = UserScanQueryMatcher.create(scan, scanInfo, columns, oldestUnexpiredTS, now, null);
+      this.matcher = UserScanQueryMatcher.create(scan, scanInfo, columns, oldestUnexpiredTS, now,
+        null);
     } else {
       if (scan.hasFilter() || (scan.getStartRow() != null && scan.getStartRow().length > 0)
           || (scan.getStopRow() != null && scan.getStopRow().length > 0)
@@ -420,12 +367,10 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
         // use legacy query matcher since we do not consider the scan object in our code. Only used
         // to keep compatibility for coprocessor.
         matcher = LegacyScanQueryMatcher.create(scan, scanInfo, columns, scanType, Long.MAX_VALUE,
-          earliestPutTs, oldestUnexpiredTS, now, null, null,
-            store.getCoprocessorHost());
+          earliestPutTs, oldestUnexpiredTS, now, null, null, store.getCoprocessorHost());
       } else {
         this.matcher = CompactionScanQueryMatcher.create(scanInfo, scanType, Long.MAX_VALUE,
-          earliestPutTs, oldestUnexpiredTS, now, null, null,
-            null);
+          earliestPutTs, oldestUnexpiredTS, now, null, null, null);
       }
     }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/c5abb6ca/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/querymatcher/DropDeletesCompactionScanQueryMatcher.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/querymatcher/DropDeletesCompactionScanQueryMatcher.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/querymatcher/DropDeletesCompactionScanQueryMatcher.java
index 5aa041e..89725fe 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/querymatcher/DropDeletesCompactionScanQueryMatcher.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/querymatcher/DropDeletesCompactionScanQueryMatcher.java
@@ -18,7 +18,6 @@
 package org.apache.hadoop.hbase.regionserver.querymatcher;
 
 import org.apache.hadoop.hbase.Cell;
-import org.apache.hadoop.hbase.TimestampType;
 import org.apache.hadoop.hbase.KeepDeletedCells;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.regionserver.ScanInfo;
@@ -63,21 +62,9 @@ public abstract class DropDeletesCompactionScanQueryMatcher extends CompactionSc
   protected final MatchCode tryDropDelete(Cell cell) {
     long timestamp = cell.getTimestamp();
     // If it is not the time to drop the delete marker, just return
-    if (timeToPurgeDeletes > 0) {
-      // Assumes now and timestamp should be of same type. It should be the case.
-      // Else there is something wrong. if it happens in tests, tests should be rewritten.
-      if (TimestampType.HYBRID.isLikelyOfType(now, true)) {
-        if (TimestampType.HYBRID.toEpochTimeMillisFromTimestamp(now) - TimestampType.HYBRID
-          .toEpochTimeMillisFromTimestamp(timestamp) <= timeToPurgeDeletes) {
-          return MatchCode.INCLUDE;
-        }
-      } else {
-        if (now - timestamp <= timeToPurgeDeletes) {
-          return MatchCode.INCLUDE;
-        }
-      }
+    if (timeToPurgeDeletes > 0 && now - timestamp <= timeToPurgeDeletes) {
+      return MatchCode.INCLUDE;
     }
-
     if (keepDeletedCells == KeepDeletedCells.TRUE
         || (keepDeletedCells == KeepDeletedCells.TTL && timestamp >= oldestUnexpiredTS)) {
       // If keepDeletedCell is true, or the delete marker is not expired yet, we should include it

http://git-wip-us.apache.org/repos/asf/hbase/blob/c5abb6ca/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/querymatcher/ScanQueryMatcher.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/querymatcher/ScanQueryMatcher.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/querymatcher/ScanQueryMatcher.java
index 2e59482..e508a9a 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/querymatcher/ScanQueryMatcher.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/querymatcher/ScanQueryMatcher.java
@@ -31,7 +31,6 @@ import org.apache.hadoop.hbase.Tag;
 import org.apache.hadoop.hbase.TagType;
 import org.apache.hadoop.hbase.TagUtil;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.TimestampType;
 import org.apache.hadoop.hbase.filter.Filter;
 import org.apache.hadoop.hbase.regionserver.KeyValueScanner;
 import org.apache.hadoop.hbase.regionserver.RegionCoprocessorHost;
@@ -154,23 +153,8 @@ public abstract class ScanQueryMatcher implements ShipperListener {
         long ts = cell.getTimestamp();
         assert t.getValueLength() == Bytes.SIZEOF_LONG;
         long ttl = TagUtil.getValueAsLong(t);
-        if (TimestampType.HYBRID.isLikelyOfType(ts, true)) {
-          if (TimestampType.HYBRID.isLikelyOfType(now, true)) {
-            if (TimestampType.HYBRID.toEpochTimeMillisFromTimestamp(ts) + ttl < TimestampType.HYBRID
-                .toEpochTimeMillisFromTimestamp(now)) {
-              return true;
-            }
-          }
-          else {
-            if (TimestampType.HYBRID.toEpochTimeMillisFromTimestamp(ts) + ttl < now) {
-              return true;
-            }
-          }
-
-        } else {
-          if (ts + ttl < now) {
-            return true;
-          }
+        if (ts + ttl < now) {
+          return true;
         }
         // Per cell TTLs cannot extend lifetime beyond family settings, so
         // fall through to check that

http://git-wip-us.apache.org/repos/asf/hbase/blob/c5abb6ca/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java
index 1c7b39c..aa0c094 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java
@@ -55,7 +55,6 @@ import org.apache.hadoop.hbase.ProcedureInfo;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.Tag;
-import org.apache.hadoop.hbase.TimestampType;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.client.Append;
 import org.apache.hadoop.hbase.client.Delete;
@@ -778,14 +777,7 @@ public class AccessController implements MasterObserver, RegionObserver, RegionS
     // any cells found there inclusively.
     long latestTs = Math.max(opTs, latestCellTs);
     if (latestTs == 0 || latestTs == HConstants.LATEST_TIMESTAMP) {
-      if (latestCellTs == HConstants.LATEST_TIMESTAMP || latestCellTs == 0) {
-        latestTs = HConstants.LATEST_TIMESTAMP - 1;
-      } else if (TimestampType.HYBRID.isLikelyOfType(latestCellTs, true)) {
-        latestTs = TimestampType.HYBRID.fromEpochTimeMillisToTimestamp(EnvironmentEdgeManager
-          .currentTime());
-      } else {
-        latestTs = EnvironmentEdgeManager.currentTime();
-      }
+      latestTs = EnvironmentEdgeManager.currentTime();
     }
     get.setTimeRange(0, latestTs + 1);
     // In case of Put operation we set to read all versions. This was done to consider the case

http://git-wip-us.apache.org/repos/asf/hbase/blob/c5abb6ca/hbase-server/src/test/java/org/apache/hadoop/hbase/MockRegionServerServices.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/MockRegionServerServices.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/MockRegionServerServices.java
index 7312000..a99345b 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/MockRegionServerServices.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/MockRegionServerServices.java
@@ -252,16 +252,6 @@ public class MockRegionServerServices implements RegionServerServices {
     return null;
   }
 
-  @Override public Clock getRegionServerClock(ClockType clockType) {
-    if (clockType.equals(ClockType.HLC)){
-      return new Clock.HLC();
-    } else if (clockType.equals(ClockType.SYSTEM_MONOTONIC)) {
-      return new Clock.SystemMonotonic();
-    } else {
-      return new Clock.System();
-    }
-  }
-
   @Override
   public ExecutorService getExecutorService() {
     return null;

http://git-wip-us.apache.org/repos/asf/hbase/blob/c5abb6ca/hbase-server/src/test/java/org/apache/hadoop/hbase/TestClockWithCluster.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestClockWithCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestClockWithCluster.java
deleted file mode 100644
index b885a79..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestClockWithCluster.java
+++ /dev/null
@@ -1,127 +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.hadoop.hbase;
-
-import static org.junit.Assert.*;
-
-import java.io.IOException;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hbase.client.Admin;
-import org.apache.hadoop.hbase.client.Connection;
-import org.apache.hadoop.hbase.client.ConnectionFactory;
-import org.apache.hadoop.hbase.client.Get;
-import org.apache.hadoop.hbase.client.Result;
-import org.apache.hadoop.hbase.client.Scan;
-import org.apache.hadoop.hbase.client.Table;
-import org.apache.hadoop.hbase.TimestampType;
-import org.apache.hadoop.hbase.testclassification.MediumTests;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.FSTableDescriptors;
-import org.junit.After;
-import org.junit.AfterClass;
-import org.junit.Before;
-import org.junit.BeforeClass;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-
-@Category({MediumTests.class})
-public class TestClockWithCluster {
-  private static final Log LOG = LogFactory.getLog(TestClockWithCluster.class);
-  private static final  HBaseTestingUtility UTIL = new HBaseTestingUtility();
-  private static Connection connection;
-  private byte[] columnFamily = Bytes.toBytes("testCF");
-  @BeforeClass
-  public static void setupClass() throws Exception {
-    UTIL.startMiniCluster(1);
-    connection = ConnectionFactory.createConnection(UTIL.getConfiguration());
-  }
-
-  @AfterClass
-  public static void tearDownClass() throws Exception {
-    connection.close();
-    UTIL.shutdownMiniCluster();
-  }
-
-  private void verifyTimestamps(Table table, final byte[] f, int startRow, int endRow,
-      TimestampType timestamp, boolean isMonotonic) throws IOException {
-    for (int i = startRow; i < endRow; i++) {
-      String failMsg = "Failed verification of row :" + i;
-      byte[] data = Bytes.toBytes(String.valueOf(i));
-      Get get = new Get(data);
-      Result result = table.get(get);
-      Cell cell = result.getColumnLatestCell(f, null);
-      assertTrue(failMsg, timestamp.isLikelyOfType(cell.getTimestamp(), isMonotonic));
-    }
-  }
-
-  @Test
-  public void testNewTablesAreCreatedWithSystemClock() throws IOException {
-    try {
-      Admin admin = connection.getAdmin();
-      TableName tableName = TableName.valueOf("TestNewTablesAreSystemByDefault");
-      admin.createTable(new HTableDescriptor(tableName).addFamily(new
-          HColumnDescriptor(columnFamily)));
-
-      Table table = connection.getTable(tableName);
-
-      ClockType clockType = admin.getTableDescriptor(tableName).getClockType();
-      assertEquals(ClockType.SYSTEM, clockType);
-      // write
-      UTIL.loadNumericRows(table, columnFamily, 0, 1000);
-      // read , check if the it is same.
-      UTIL.verifyNumericRows(table, Bytes.toBytes("testCF"), 0, 1000, 0);
-
-      // This check will be useful if Clock type were to be system monotonic or HLC.
-      verifyTimestamps(table, columnFamily, 0, 1000, TimestampType.PHYSICAL, false);
-    } catch(Exception e) {
-
-    }
-  }
-
-  @Test
-  public void testMetaTableClockTypeIsSystem() {
-    try {
-      Admin admin = connection.getAdmin();
-      Table table = connection.getTable(TableName.META_TABLE_NAME);
-      ClockType clockType = admin.getTableDescriptor(TableName.META_TABLE_NAME).getClockType();
-      assertEquals(ClockType.SYSTEM, clockType);
-    } catch(IOException ioE) {
-      fail("Execution should not come here");
-    }
-  }
-
-  @Test
-  public void testMetaTableTimestampsAreSystem() {
-    // Checks timestamps of whatever is present in meta table currently.
-    // ToDo: Include complete meta table sample with all column families to check all paths of
-    // meta table modification.
-    try {
-      Table table = connection.getTable(TableName.META_TABLE_NAME);
-      Result result = table.getScanner(new Scan()).next();
-      for (Cell cell : result.rawCells()) {
-        assertTrue(TimestampType.PHYSICAL.isLikelyOfType(cell.getTimestamp(), false));
-      }
-    } catch(IOException ioE) {
-      fail("Execution should not come here");
-    }
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/c5abb6ca/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestIncrementTimeRange.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestIncrementTimeRange.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestIncrementTimeRange.java
index aa0c0a4..8805337 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestIncrementTimeRange.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestIncrementTimeRange.java
@@ -19,24 +19,19 @@
 
 package org.apache.hadoop.hbase.coprocessor;
 
-import static org.bouncycastle.asn1.x500.style.RFC4519Style.name;
 import static org.junit.Assert.assertArrayEquals;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 
 import java.io.IOException;
-import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.List;
 import java.util.Map;
 import java.util.NavigableMap;
 
-import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.Cell;
-import org.apache.hadoop.hbase.ClockType;
-import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Get;
 import org.apache.hadoop.hbase.client.Increment;
 import org.apache.hadoop.hbase.client.Put;
@@ -50,17 +45,12 @@ import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.hbase.util.ManualEnvironmentEdge;
-import org.apache.hadoop.hbase.util.TestTableName;
 import org.junit.After;
 import org.junit.AfterClass;
 import org.junit.Before;
 import org.junit.BeforeClass;
-import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-import org.junit.runners.Parameterized.Parameters;
 
 /**
  * This test runs batch mutation with Increments which have custom TimeRange.
@@ -69,16 +59,14 @@ import org.junit.runners.Parameterized.Parameters;
  * See HBASE-15698
  */
 @Category({CoprocessorTests.class, MediumTests.class})
-@RunWith(Parameterized.class)
 public class TestIncrementTimeRange {
 
   private static final HBaseTestingUtility util = new HBaseTestingUtility();
   private static ManualEnvironmentEdge mee = new ManualEnvironmentEdge();
 
-  @Rule
-  public TestTableName TEST_TABLE = new TestTableName();
+  private static final TableName TEST_TABLE = TableName.valueOf("test");
   private static final byte[] TEST_FAMILY = Bytes.toBytes("f1");
-  private static final byte[][] TEST_FAMILIES = new byte[][]{TEST_FAMILY};
+
   private static final byte[] ROW_A = Bytes.toBytes("aaa");
   private static final byte[] ROW_B = Bytes.toBytes("bbb");
   private static final byte[] ROW_C = Bytes.toBytes("ccc");
@@ -92,18 +80,6 @@ public class TestIncrementTimeRange {
   private Table hTableInterface;
   private Table table;
 
-  private ClockType clockType;
-
-  @Parameters(name = "{0}")
-  public static Iterable<Object> data() {
-    return Arrays.asList(new Object[] {ClockType.HLC, ClockType.SYSTEM_MONOTONIC, ClockType
-        .SYSTEM});
-  }
-
-  public TestIncrementTimeRange(ClockType clockType) {
-    this.clockType = clockType;
-  }
-
   @BeforeClass
   public static void setupBeforeClass() throws Exception {
     util.getConfiguration().set(CoprocessorHost.REGION_COPROCESSOR_CONF_KEY,
@@ -122,8 +98,7 @@ public class TestIncrementTimeRange {
 
   @Before
   public void before() throws Exception {
-    HTableDescriptor htd = util.createTableDescriptor(TEST_TABLE.getTableName()).setClockType(clockType);
-    table = util.createTable(htd, TEST_FAMILIES, new Configuration(HBaseConfiguration.create()));
+    table = util.createTable(TEST_TABLE, TEST_FAMILY);
 
     Put puta = new Put(ROW_A);
     puta.addColumn(TEST_FAMILY, qualifierCol1, bytes1);
@@ -146,7 +121,7 @@ public class TestIncrementTimeRange {
       }
     } finally {
       try {
-        util.deleteTable(TEST_TABLE.getTableName());
+        util.deleteTable(TEST_TABLE);
       } catch (IOException ioe) {
       }
     }
@@ -175,7 +150,7 @@ public class TestIncrementTimeRange {
 
   @Test
   public void testHTableInterfaceMethods() throws Exception {
-    hTableInterface = util.getConnection().getTable(TEST_TABLE.getTableName());
+    hTableInterface = util.getConnection().getTable(TEST_TABLE);
     checkHTableInterfaceMethods();
   }
 
@@ -187,7 +162,7 @@ public class TestIncrementTimeRange {
 
     time = EnvironmentEdgeManager.currentTime();
     mee.setValue(time);
-    TimeRange range10 = new TimeRange(1, time + 10);
+    TimeRange range10 = new TimeRange(1, time+10);
     hTableInterface.increment(new Increment(ROW_A).addColumn(TEST_FAMILY, qualifierCol1, 10L)
         .setTimeRange(range10.getMin(), range10.getMax()));
     checkRowValue(ROW_A, Bytes.toBytes(11L));
@@ -196,7 +171,7 @@ public class TestIncrementTimeRange {
 
     time = EnvironmentEdgeManager.currentTime();
     mee.setValue(time);
-    TimeRange range2 = new TimeRange(1, time + 20);
+    TimeRange range2 = new TimeRange(1, time+20);
     List<Row> actions =
         Arrays.asList(new Row[] { new Increment(ROW_A).addColumn(TEST_FAMILY, qualifierCol1, 2L)
             .setTimeRange(range2.getMin(), range2.getMax()),

http://git-wip-us.apache.org/repos/asf/hbase/blob/c5abb6ca/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestCopyTable.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestCopyTable.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestCopyTable.java
index c29b31f..0bec03b 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestCopyTable.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestCopyTable.java
@@ -17,7 +17,6 @@
  */
 package org.apache.hadoop.hbase.mapreduce;
 
-import static org.apache.hadoop.hbase.constraint.CheckConfigurationConstraint.getConfiguration;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertNull;
@@ -29,9 +28,7 @@ import java.io.PrintStream;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.CellUtil;
-import org.apache.hadoop.hbase.ClockType;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Get;
 import org.apache.hadoop.hbase.client.Put;
@@ -192,24 +189,13 @@ public class TestCopyTable {
    */
   @Test
   public void testRenameFamily() throws Exception {
-    testRenameFamily(ClockType.SYSTEM);
-    testRenameFamily(ClockType.SYSTEM_MONOTONIC);
-    testRenameFamily(ClockType.HLC);
-  }
-
-  public void testRenameFamily(ClockType clockType) throws Exception {
-    TableName sourceTable = TableName.valueOf("sourceTable");
-    HTableDescriptor sourceTableDesc = new HTableDescriptor(sourceTable);
-    sourceTableDesc.setClockType(clockType);
-    TableName targetTable = TableName.valueOf("targetTable");
-    HTableDescriptor targetTableDesc = new HTableDescriptor(targetTable);
-    targetTableDesc.setClockType(clockType);
+    final TableName sourceTable = TableName.valueOf(name.getMethodName() + "source");
+    final TableName targetTable = TableName.valueOf(name.getMethodName() + "-target");
 
     byte[][] families = { FAMILY_A, FAMILY_B };
-    Table t = TEST_UTIL.createTable(sourceTableDesc, families, (byte[][]) null, new Configuration
-        (getConfiguration()));
-    Table t2 = TEST_UTIL.createTable(targetTableDesc, families, (byte[][]) null, new Configuration
-        (getConfiguration()));
+
+    Table t = TEST_UTIL.createTable(sourceTable, families);
+    Table t2 = TEST_UTIL.createTable(targetTable, families);
     Put p = new Put(ROW1);
     p.addColumn(FAMILY_A, QUALIFIER, Bytes.toBytes("Data11"));
     p.addColumn(FAMILY_B, QUALIFIER, Bytes.toBytes("Data12"));
@@ -239,9 +225,6 @@ public class TestCopyTable {
     // Data from the family of B is not copied
     assertNull(b1);
 
-    TEST_UTIL.deleteTable(sourceTable);
-    TEST_UTIL.deleteTable(targetTable);
-
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/c5abb6ca/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat2.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat2.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat2.java
index cda2509..87522b6 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat2.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat2.java
@@ -49,7 +49,6 @@ import org.apache.hadoop.hbase.ArrayBackedTag;
 import org.apache.hadoop.hbase.CategoryBasedTimeout;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellUtil;
-import org.apache.hadoop.hbase.ClockType;
 import org.apache.hadoop.hbase.CompatibilitySingletonFactory;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HBaseTestingUtility;

http://git-wip-us.apache.org/repos/asf/hbase/blob/c5abb6ca/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java
index 8ec05ef..7ac7571 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java
@@ -34,8 +34,6 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.hbase.Abortable;
 import org.apache.hadoop.hbase.CellScannable;
 import org.apache.hadoop.hbase.CellUtil;
-import org.apache.hadoop.hbase.ClockType;
-import org.apache.hadoop.hbase.Clock;
 import org.apache.hadoop.hbase.ChoreService;
 import org.apache.hadoop.hbase.CoordinatedStateManager;
 import org.apache.hadoop.hbase.HRegionInfo;
@@ -581,12 +579,6 @@ ClientProtos.ClientService.BlockingInterface, RegionServerServices {
   }
 
   @Override
-  public Clock getRegionServerClock(ClockType clockType) {
-    Clock systemClock = new Clock.System();
-    return systemClock;
-  }
-
-  @Override
   public ExecutorService getExecutorService() {
     return null;
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/c5abb6ca/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactingMemStore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactingMemStore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactingMemStore.java
index 40ba82d..b0eadb5 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactingMemStore.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactingMemStore.java
@@ -42,12 +42,8 @@ import org.junit.Before;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
-import org.apache.hadoop.hbase.Clock;
-
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
 
 /**
  * compacted memstore test case
@@ -173,29 +169,24 @@ public class TestCompactingMemStore extends TestDefaultMemStore {
   /** Test getNextRow from memstore
    * @throws InterruptedException
    */
+  @Override
   @Test
   public void testGetNextRow() throws Exception {
-    testGetNextRow(new Clock.HLC());
-    testGetNextRow(new Clock.SystemMonotonic());
-    testGetNextRow(new Clock.System());
-  }
-
-  public void testGetNextRow(Clock clock) throws Exception {
-    addRows(this.memstore, clock);
+    addRows(this.memstore);
     // Add more versions to make it a little more interesting.
     Thread.sleep(1);
-    addRows(this.memstore, clock);
+    addRows(this.memstore);
     Cell closestToEmpty = ((CompactingMemStore)this.memstore).getNextRow(KeyValue.LOWESTKEY);
     assertTrue(CellComparator.COMPARATOR.compareRows(closestToEmpty,
-        new KeyValue(Bytes.toBytes(0), clock.now())) == 0);
+        new KeyValue(Bytes.toBytes(0), System.currentTimeMillis())) == 0);
     for (int i = 0; i < ROW_COUNT; i++) {
       Cell nr = ((CompactingMemStore)this.memstore).getNextRow(new KeyValue(Bytes.toBytes(i),
-          clock.now()));
+          System.currentTimeMillis()));
       if (i + 1 == ROW_COUNT) {
         assertEquals(nr, null);
       } else {
         assertTrue(CellComparator.COMPARATOR.compareRows(nr,
-            new KeyValue(Bytes.toBytes(i + 1), clock.now())) == 0);
+            new KeyValue(Bytes.toBytes(i + 1), System.currentTimeMillis())) == 0);
       }
     }
     //starting from each row, validate results should contain the starting row
@@ -204,10 +195,9 @@ public class TestCompactingMemStore extends TestDefaultMemStore {
       ScanInfo scanInfo = new ScanInfo(conf, FAMILY, 0, 1, Integer.MAX_VALUE,
           KeepDeletedCells.FALSE, HConstants.DEFAULT_BLOCKSIZE, 0, this.memstore.getComparator());
       ScanType scanType = ScanType.USER_SCAN;
-      Store mockStore = mock(HStore.class);
-      when(mockStore.getClock()).thenReturn(Clock.getDummyClockOfGivenClockType(clock.clockType));
-      InternalScanner scanner = new StoreScanner(mockStore, new Scan(Bytes.toBytes(startRowId)),
-          scanInfo, scanType, null, memstore.getScanners(0)) ;
+      InternalScanner scanner = new StoreScanner(new Scan(
+          Bytes.toBytes(startRowId)), scanInfo, scanType, null,
+          memstore.getScanners(0));
       List<Cell> results = new ArrayList<>();
       for (int i = 0; scanner.next(results); i++) {
         int rowId = startRowId + i;

http://git-wip-us.apache.org/repos/asf/hbase/blob/c5abb6ca/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultMemStore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultMemStore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultMemStore.java
index a748487..439f3d4 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultMemStore.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultMemStore.java
@@ -30,7 +30,6 @@ import org.apache.hadoop.hbase.CategoryBasedTimeout;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellComparator;
 import org.apache.hadoop.hbase.CellUtil;
-import org.apache.hadoop.hbase.Clock;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HColumnDescriptor;
@@ -61,8 +60,6 @@ import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertNotNull;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
 
 import org.junit.experimental.categories.Category;
 import org.junit.rules.TestName;
@@ -583,22 +580,16 @@ public class TestDefaultMemStore {
    */
   @Test
   public void testGetNextRow() throws Exception {
-    testGetNextRow(new Clock.HLC());
-    testGetNextRow(new Clock.SystemMonotonic());
-    testGetNextRow(new Clock.System());
-  }
-
-  public void testGetNextRow(Clock clock) throws Exception {
-    addRows(this.memstore, clock);
+    addRows(this.memstore);
     // Add more versions to make it a little more interesting.
     Thread.sleep(1);
-    addRows(this.memstore, clock);
+    addRows(this.memstore);
     Cell closestToEmpty = ((DefaultMemStore) this.memstore).getNextRow(KeyValue.LOWESTKEY);
     assertTrue(CellComparator.COMPARATOR.compareRows(closestToEmpty,
         new KeyValue(Bytes.toBytes(0), System.currentTimeMillis())) == 0);
     for (int i = 0; i < ROW_COUNT; i++) {
       Cell nr = ((DefaultMemStore) this.memstore).getNextRow(new KeyValue(Bytes.toBytes(i),
-          clock.now()));
+          System.currentTimeMillis()));
       if (i + 1 == ROW_COUNT) {
         assertEquals(nr, null);
       } else {
@@ -612,11 +603,9 @@ public class TestDefaultMemStore {
       ScanInfo scanInfo = new ScanInfo(conf, FAMILY, 0, 1, Integer.MAX_VALUE,
           KeepDeletedCells.FALSE, HConstants.DEFAULT_BLOCKSIZE, 0, this.memstore.getComparator());
       ScanType scanType = ScanType.USER_SCAN;
-      Store store = mock(HStore.class);
-      when(store.getClock()).thenReturn(Clock.getDummyClockOfGivenClockType(clock.clockType));
-
-      try (InternalScanner scanner = new StoreScanner(store, new Scan(Bytes.toBytes(startRowId)),
-          scanInfo, scanType, null, memstore.getScanners(0))) {
+      try (InternalScanner scanner = new StoreScanner(new Scan(
+          Bytes.toBytes(startRowId)), scanInfo, scanType, null,
+          memstore.getScanners(0))) {
         List<Cell> results = new ArrayList<>();
         for (int i = 0; scanner.next(results); i++) {
           int rowId = startRowId + i;
@@ -1034,24 +1023,6 @@ public class TestDefaultMemStore {
     return ROW_COUNT;
   }
 
-  /**
-   * Adds {@link #ROW_COUNT} rows and {@link #QUALIFIER_COUNT}
-   * @param hmc Instance to add rows to.
-   * @return How many rows we added.
-   * @throws IOException
-   */
-  protected int addRows(final MemStore hmc, Clock clock) {
-    for (int i = 0; i < ROW_COUNT; i++) {
-      long timestamp = clock.now();
-      for (int ii = 0; ii < QUALIFIER_COUNT; ii++) {
-        byte [] row = Bytes.toBytes(i);
-        byte [] qf = makeQualifier(i, ii);
-        hmc.add(new KeyValue(row, FAMILY, qf, timestamp, qf), null);
-      }
-    }
-    return ROW_COUNT;
-  }
-
   private long runSnapshot(final AbstractMemStore hmc) throws UnexpectedStateException {
     // Save off old state.
     int oldHistorySize = hmc.getSnapshot().getCellsCount();

http://git-wip-us.apache.org/repos/asf/hbase/blob/c5abb6ca/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java
index 4d6bb85..4f46c88 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java
@@ -43,7 +43,6 @@ import static org.mockito.Mockito.when;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
 
-import java.io.File;
 import java.io.IOException;
 import java.io.InterruptedIOException;
 import java.security.PrivilegedExceptionAction;
@@ -78,9 +77,6 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.ArrayBackedTag;
 import org.apache.hadoop.hbase.CategoryBasedTimeout;
 import org.apache.hadoop.hbase.Cell;
-import org.apache.hadoop.hbase.Clock;
-import org.apache.hadoop.hbase.ClockType;
-import org.apache.hadoop.hbase.TimestampType;
 import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.CompatibilitySingletonFactory;
 import org.apache.hadoop.hbase.DroppedSnapshotException;
@@ -6011,12 +6007,9 @@ public class TestHRegion {
 
   @Test
   public void testCellTTLs() throws IOException {
-    testCellTTLs(ClockType.SYSTEM);
-    testCellTTLs(ClockType.SYSTEM_MONOTONIC);
-    testCellTTLs(ClockType.HLC);
-  }
+    IncrementingEnvironmentEdge edge = new IncrementingEnvironmentEdge();
+    EnvironmentEdgeManager.injectEdge(edge);
 
-  public void testCellTTLs(ClockType clockType) throws IOException {
     final byte[] row = Bytes.toBytes("testRow");
     final byte[] q1 = Bytes.toBytes("q1");
     final byte[] q2 = Bytes.toBytes("q2");
@@ -6027,8 +6020,6 @@ public class TestHRegion {
     HColumnDescriptor hcd = new HColumnDescriptor(fam1);
     hcd.setTimeToLive(10); // 10 seconds
     htd.addFamily(hcd);
-    htd.setClockType(clockType);
-    TimestampType timestampType = clockType.timestampType();
 
     Configuration conf = new Configuration(TEST_UTIL.getConfiguration());
     conf.setInt(HFile.FORMAT_VERSION_KEY, HFile.MIN_FORMAT_VERSION_WITH_TAGS);
@@ -6037,32 +6028,22 @@ public class TestHRegion {
             HConstants.EMPTY_BYTE_ARRAY, HConstants.EMPTY_BYTE_ARRAY),
         TEST_UTIL.getDataTestDir(), conf, htd);
     assertNotNull(region);
-
-    region.setClock(Clock.getDummyClockOfGivenClockType(clockType));
-    long now = timestampType.toEpochTimeMillisFromTimestamp(region.getClock().now());
-    ManualEnvironmentEdge mee = new ManualEnvironmentEdge();
-    EnvironmentEdgeManager.injectEdge(mee);
-    mee.setValue(now);
-
     try {
+      long now = EnvironmentEdgeManager.currentTime();
       // Add a cell that will expire in 5 seconds via cell TTL
-      region.put(new Put(row).add(new KeyValue(row, fam1, q1, timestampType
-          .fromEpochTimeMillisToTimestamp(now),
-          HConstants.EMPTY_BYTE_ARRAY, new ArrayBackedTag[] {
+      region.put(new Put(row).add(new KeyValue(row, fam1, q1, now,
+        HConstants.EMPTY_BYTE_ARRAY, new ArrayBackedTag[] {
           // TTL tags specify ts in milliseconds
           new ArrayBackedTag(TagType.TTL_TAG_TYPE, Bytes.toBytes(5000L)) } )));
       // Add a cell that will expire after 10 seconds via family setting
-      region.put(new Put(row).addColumn(fam1, q2, timestampType
-          .fromEpochTimeMillisToTimestamp(now), HConstants.EMPTY_BYTE_ARRAY));
+      region.put(new Put(row).addColumn(fam1, q2, now, HConstants.EMPTY_BYTE_ARRAY));
       // Add a cell that will expire in 15 seconds via cell TTL
-      region.put(new Put(row).add(new KeyValue(row, fam1, q3, timestampType
-          .fromEpochTimeMillisToTimestamp(now + 10000 - 1),
-          HConstants.EMPTY_BYTE_ARRAY, new ArrayBackedTag[] {
+      region.put(new Put(row).add(new KeyValue(row, fam1, q3, now + 10000 - 1,
+        HConstants.EMPTY_BYTE_ARRAY, new ArrayBackedTag[] {
           // TTL tags specify ts in milliseconds
           new ArrayBackedTag(TagType.TTL_TAG_TYPE, Bytes.toBytes(5000L)) } )));
       // Add a cell that will expire in 20 seconds via family setting
-      region.put(new Put(row).addColumn(fam1, q4, timestampType.fromEpochTimeMillisToTimestamp
-          (now + 10000 - 1), HConstants.EMPTY_BYTE_ARRAY));
+      region.put(new Put(row).addColumn(fam1, q4, now + 10000 - 1, HConstants.EMPTY_BYTE_ARRAY));
 
       // Flush so we are sure store scanning gets this right
       region.flush(true);
@@ -6075,7 +6056,7 @@ public class TestHRegion {
       assertNotNull(r.getValue(fam1, q4));
 
       // Increment time to T+5 seconds
-      mee.setValue(now + 5001);
+      edge.incrementTime(5000);
 
       r = region.get(new Get(row));
       assertNull(r.getValue(fam1, q1));
@@ -6084,7 +6065,7 @@ public class TestHRegion {
       assertNotNull(r.getValue(fam1, q4));
 
       // Increment time to T+10 seconds
-      mee.setValue(now + 10001);
+      edge.incrementTime(5000);
 
       r = region.get(new Get(row));
       assertNull(r.getValue(fam1, q1));
@@ -6093,7 +6074,7 @@ public class TestHRegion {
       assertNotNull(r.getValue(fam1, q4));
 
       // Increment time to T+15 seconds
-      mee.setValue(now + 15000);
+      edge.incrementTime(5000);
 
       r = region.get(new Get(row));
       assertNull(r.getValue(fam1, q1));
@@ -6102,7 +6083,7 @@ public class TestHRegion {
       assertNotNull(r.getValue(fam1, q4));
 
       // Increment time to T+20 seconds
-      mee.setValue(now + 20000);
+      edge.incrementTime(10000);
 
       r = region.get(new Get(row));
       assertNull(r.getValue(fam1, q1));
@@ -6131,12 +6112,7 @@ public class TestHRegion {
       assertEquals(Bytes.toLong(val), 2L);
 
       // Increment time to T+25 seconds
-
-      if (clockType == ClockType.SYSTEM) {
-        mee.setValue(now + 25002);
-      } else {
-        mee.setValue(now + 25001);
-      }
+      edge.incrementTime(5000);
 
       // Value should be back to 1
       r = region.get(new Get(row));
@@ -6145,7 +6121,7 @@ public class TestHRegion {
       assertEquals(Bytes.toLong(val), 1L);
 
       // Increment time to T+30 seconds
-      mee.setValue(now + 30001);
+      edge.incrementTime(5000);
 
       // Original value written at T+20 should be gone now via family TTL
       r = region.get(new Get(row));

http://git-wip-us.apache.org/repos/asf/hbase/blob/c5abb6ca/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionReplayEvents.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionReplayEvents.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionReplayEvents.java
index 95024ab..f115b34 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionReplayEvents.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionReplayEvents.java
@@ -60,8 +60,6 @@ import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.Clock;
-import org.apache.hadoop.hbase.ClockType;
 import org.apache.hadoop.hbase.client.Durability;
 import org.apache.hadoop.hbase.client.Get;
 import org.apache.hadoop.hbase.client.Put;
@@ -101,8 +99,6 @@ import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.junit.rules.TestName;
-import org.apache.hadoop.hbase.Clock;
-import org.apache.hadoop.hbase.ClockType;
 
 /**
  * Tests of HRegion methods for replaying flush, compaction, region open, etc events for secondary
@@ -175,7 +171,6 @@ public class TestHRegionReplayEvents {
     when(rss.getServerName()).thenReturn(ServerName.valueOf("foo", 1, 1));
     when(rss.getConfiguration()).thenReturn(CONF);
     when(rss.getRegionServerAccounting()).thenReturn(new RegionServerAccounting(CONF));
-    when(rss.getRegionServerClock((ClockType)any())).thenReturn(new Clock.System());
     String string = org.apache.hadoop.hbase.executor.EventType.RS_COMPACTED_FILES_DISCHARGER
         .toString();
     ExecutorService es = new ExecutorService(string);

http://git-wip-us.apache.org/repos/asf/hbase/blob/c5abb6ca/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionSplitPolicy.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionSplitPolicy.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionSplitPolicy.java
index e07f042..89f7589 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionSplitPolicy.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionSplitPolicy.java
@@ -21,7 +21,6 @@ import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
-import static org.mockito.Matchers.any;
 
 import java.io.IOException;
 import java.util.ArrayList;
@@ -31,8 +30,6 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.Clock;
-import org.apache.hadoop.hbase.ClockType;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.testclassification.RegionServerTests;
@@ -109,7 +106,6 @@ public class TestRegionSplitPolicy {
     final List<Region> regions = new ArrayList<>();
     Mockito.when(rss.getOnlineRegions(TABLENAME)).thenReturn(regions);
     Mockito.when(mockRegion.getRegionServerServices()).thenReturn(rss);
-    Mockito.when(rss.getRegionServerClock(ClockType.SYSTEM)).thenReturn(new Clock.System());
     // Set max size for this 'table'.
     long maxSplitSize = 1024L;
     htd.setMaxFileSize(maxSplitSize);
@@ -171,7 +167,6 @@ public class TestRegionSplitPolicy {
     Mockito.when(mockRegion.getRegionServerServices()).thenReturn(rss);
     Mockito.when(mockRegion.getBlockedRequestsCount()).thenReturn(0L);
     Mockito.when(mockRegion.getWriteRequestsCount()).thenReturn(0L);
-    Mockito.when(rss.getRegionServerClock(ClockType.SYSTEM)).thenReturn(new Clock.System());
 
 
     BusyRegionSplitPolicy policy =

http://git-wip-us.apache.org/repos/asf/hbase/blob/c5abb6ca/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreScanner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreScanner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreScanner.java
index 4a9415b..10f00a6 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreScanner.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreScanner.java
@@ -21,8 +21,6 @@ package org.apache.hadoop.hbase.regionserver;
 
 import static org.apache.hadoop.hbase.regionserver.KeyValueScanFixture.scanFixture;
 import static org.junit.Assert.*;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
 
 import java.io.IOException;
 import java.util.ArrayList;
@@ -40,14 +38,11 @@ import org.apache.hadoop.hbase.CategoryBasedTimeout;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellComparator;
 import org.apache.hadoop.hbase.CellUtil;
-import org.apache.hadoop.hbase.Clock;
-import org.apache.hadoop.hbase.ClockType;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.KeepDeletedCells;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.KeyValueTestUtil;
-import org.apache.hadoop.hbase.TimestampType;
 import org.apache.hadoop.hbase.client.Get;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.filter.ColumnCountGetFilter;
@@ -821,31 +816,16 @@ public class TestStoreScanner {
    */
   @Test
   public void testWildCardTtlScan() throws IOException {
-    //testWildCardTtlScan(ClockType.SYSTEM);
-    //testWildCardTtlScan(ClockType.SYSTEM_MONOTONIC);
-    testWildCardTtlScan(ClockType.HLC);
-  }
-
-  public void testWildCardTtlScan(ClockType clockType) throws IOException {
     long now = System.currentTimeMillis();
-    TimestampType timestampType = clockType.timestampType();
     KeyValue [] kvs = new KeyValue[] {
-        KeyValueTestUtil.create("R1", "cf", "a", timestampType.fromEpochTimeMillisToTimestamp
-          (now-1000), KeyValue.Type.Put, "dont-care"),
-        KeyValueTestUtil.create("R1", "cf", "b", timestampType.fromEpochTimeMillisToTimestamp
-          (now-10), KeyValue.Type.Put, "dont-care"),
-        KeyValueTestUtil.create("R1", "cf", "c", timestampType.fromEpochTimeMillisToTimestamp
-          (now-200), KeyValue.Type.Put, "dont-care"),
-        KeyValueTestUtil.create("R1", "cf", "d", timestampType.fromEpochTimeMillisToTimestamp
-          (now-10000), KeyValue.Type.Put, "dont-care"),
-        KeyValueTestUtil.create("R2", "cf", "a", timestampType.fromEpochTimeMillisToTimestamp
-          (now), KeyValue.Type.Put, "dont-care"),
-        KeyValueTestUtil.create("R2", "cf", "b", timestampType.fromEpochTimeMillisToTimestamp
-          (now-10), KeyValue.Type.Put, "dont-care"),
-        KeyValueTestUtil.create("R2", "cf", "c", timestampType.fromEpochTimeMillisToTimestamp
-          (now-200), KeyValue.Type.Put, "dont-care"),
-        KeyValueTestUtil.create("R2", "cf", "c", timestampType.fromEpochTimeMillisToTimestamp
-          (now-1000), KeyValue.Type.Put, "dont-care")
+        KeyValueTestUtil.create("R1", "cf", "a", now-1000, KeyValue.Type.Put, "dont-care"),
+        KeyValueTestUtil.create("R1", "cf", "b", now-10, KeyValue.Type.Put, "dont-care"),
+        KeyValueTestUtil.create("R1", "cf", "c", now-200, KeyValue.Type.Put, "dont-care"),
+        KeyValueTestUtil.create("R1", "cf", "d", now-10000, KeyValue.Type.Put, "dont-care"),
+        KeyValueTestUtil.create("R2", "cf", "a", now, KeyValue.Type.Put, "dont-care"),
+        KeyValueTestUtil.create("R2", "cf", "b", now-10, KeyValue.Type.Put, "dont-care"),
+        KeyValueTestUtil.create("R2", "cf", "c", now-200, KeyValue.Type.Put, "dont-care"),
+        KeyValueTestUtil.create("R2", "cf", "c", now-1000, KeyValue.Type.Put, "dont-care")
     };
     List<KeyValueScanner> scanners = scanFixture(kvs);
     Scan scan = new Scan();
@@ -853,9 +833,7 @@ public class TestStoreScanner {
     ScanInfo scanInfo = new ScanInfo(CONF, CF, 0, 1, 500, KeepDeletedCells.FALSE,
         HConstants.DEFAULT_BLOCKSIZE, 0, CellComparator.COMPARATOR);
     ScanType scanType = ScanType.USER_SCAN;
-    Store store = mock(HStore.class);
-    when(store.getClock()).thenReturn(Clock.getDummyClockOfGivenClockType(clockType));
-    try (StoreScanner scanner = new StoreScanner(store, scan, scanInfo, scanType, null, scanners)) {
+    try (StoreScanner scanner = new StoreScanner(scan, scanInfo, scanType, null, scanners)) {
       List<Cell> results = new ArrayList<>();
       Assert.assertEquals(true, scanner.next(results));
       Assert.assertEquals(2, results.size());
@@ -914,19 +892,12 @@ public class TestStoreScanner {
    */
   @Test
   public void testExpiredDeleteFamily() throws Exception {
-    testExpiredDeleteFamily(new Clock.HLC());
-    testExpiredDeleteFamily(new Clock.SystemMonotonic());
-    testExpiredDeleteFamily(new Clock.System());
-  }
-
-  public void testExpiredDeleteFamily(Clock clock) throws Exception {
     long now = System.currentTimeMillis();
-    TimestampType timestampType = clock.getTimestampType();
     KeyValue [] kvs = new KeyValue[] {
-        new KeyValue(Bytes.toBytes("R1"), Bytes.toBytes("cf"), null, timestampType
-          .fromEpochTimeMillisToTimestamp(now-1000), KeyValue.Type.DeleteFamily),
-        KeyValueTestUtil.create("R1", "cf", "a", timestampType.fromEpochTimeMillisToTimestamp
-          (now-10), KeyValue.Type.Put, "dont-care"),
+        new KeyValue(Bytes.toBytes("R1"), Bytes.toBytes("cf"), null, now-1000,
+            KeyValue.Type.DeleteFamily),
+        KeyValueTestUtil.create("R1", "cf", "a", now-10, KeyValue.Type.Put,
+            "dont-care"),
     };
     List<KeyValueScanner> scanners = scanFixture(kvs);
     Scan scan = new Scan();
@@ -935,10 +906,9 @@ public class TestStoreScanner {
     ScanInfo scanInfo = new ScanInfo(CONF, CF, 0, 1, 500, KeepDeletedCells.FALSE,
         HConstants.DEFAULT_BLOCKSIZE, 0, CellComparator.COMPARATOR);
     ScanType scanType = ScanType.USER_SCAN;
-    Store store = mock(HStore.class);
-    when(store.getClock()).thenReturn(clock);
     try (StoreScanner scanner =
-        new StoreScanner(store, scan, scanInfo, scanType, null, scanners)) {
+        new StoreScanner(scan, scanInfo, scanType, null, scanners)) {
+
       List<Cell> results = new ArrayList<>();
       Assert.assertEquals(true, scanner.next(results));
       Assert.assertEquals(1, results.size());
@@ -951,15 +921,8 @@ public class TestStoreScanner {
 
   @Test
   public void testDeleteMarkerLongevity() throws Exception {
-    testDeleteMarkerLongevity(new Clock.HLC());
-    testDeleteMarkerLongevity(new Clock.SystemMonotonic());
-    testDeleteMarkerLongevity(new Clock.System());
-  }
-
-  public void testDeleteMarkerLongevity(Clock clock) throws Exception {
     try {
       final long now = System.currentTimeMillis();
-      TimestampType timestampType = clock.getTimestampType();
       EnvironmentEdgeManagerTestHelper.injectEdge(new EnvironmentEdge() {
         public long currentTime() {
           return now;
@@ -967,40 +930,37 @@ public class TestStoreScanner {
       });
       KeyValue[] kvs = new KeyValue[]{
         /*0*/ new KeyValue(Bytes.toBytes("R1"), Bytes.toBytes("cf"), null,
-          timestampType.fromEpochTimeMillisToTimestamp(now - 100), KeyValue.Type.DeleteFamily),
-          // live
+        now - 100, KeyValue.Type.DeleteFamily), // live
         /*1*/ new KeyValue(Bytes.toBytes("R1"), Bytes.toBytes("cf"), null,
-          timestampType.fromEpochTimeMillisToTimestamp(now - 1000), KeyValue.Type.DeleteFamily),
-          // expired
-        /*2*/ KeyValueTestUtil.create("R1", "cf", "a", timestampType
-              .fromEpochTimeMillisToTimestamp(now - 50), KeyValue.Type.Put, "v3"), // live
-        /*3*/ KeyValueTestUtil.create("R1", "cf", "a", timestampType
-              .fromEpochTimeMillisToTimestamp(now - 55), KeyValue.Type.Delete, "dontcare"), // live
-        /*4*/ KeyValueTestUtil.create("R1", "cf", "a", timestampType
-              .fromEpochTimeMillisToTimestamp(now - 55), KeyValue.Type.Put, "deleted-version v2"), // deleted
-        /*5*/ KeyValueTestUtil.create("R1", "cf", "a", timestampType
-              .fromEpochTimeMillisToTimestamp(now - 60), KeyValue.Type.Put, "v1"), // live
-        /*6*/ KeyValueTestUtil.create("R1", "cf", "a", timestampType
-              .fromEpochTimeMillisToTimestamp(now - 65), KeyValue.Type.Put, "v0"), // max-version reached
+        now - 1000, KeyValue.Type.DeleteFamily), // expired
+        /*2*/ KeyValueTestUtil.create("R1", "cf", "a", now - 50,
+        KeyValue.Type.Put, "v3"), // live
+        /*3*/ KeyValueTestUtil.create("R1", "cf", "a", now - 55,
+        KeyValue.Type.Delete, "dontcare"), // live
+        /*4*/ KeyValueTestUtil.create("R1", "cf", "a", now - 55,
+        KeyValue.Type.Put, "deleted-version v2"), // deleted
+        /*5*/ KeyValueTestUtil.create("R1", "cf", "a", now - 60,
+        KeyValue.Type.Put, "v1"), // live
+        /*6*/ KeyValueTestUtil.create("R1", "cf", "a", now - 65,
+        KeyValue.Type.Put, "v0"), // max-version reached
         /*7*/ KeyValueTestUtil.create("R1", "cf", "a",
-          timestampType.fromEpochTimeMillisToTimestamp(now - 100), KeyValue.Type.DeleteColumn,
-          "dont-care"), // max-version
-        /*8*/ KeyValueTestUtil.create("R1", "cf", "b", timestampType
-              .fromEpochTimeMillisToTimestamp(now - 600), KeyValue.Type.DeleteColumn, "dont-care"), //expired
-        /*9*/ KeyValueTestUtil.create("R1", "cf", "b", timestampType
-              .fromEpochTimeMillisToTimestamp(now - 70), KeyValue.Type.Put, "v2"), //live
-        /*10*/ KeyValueTestUtil.create("R1", "cf", "b", timestampType
-              .fromEpochTimeMillisToTimestamp(now - 750), KeyValue.Type.Put, "v1"), //expired
-        /*11*/ KeyValueTestUtil.create("R1", "cf", "c", timestampType
-              .fromEpochTimeMillisToTimestamp(now - 500), KeyValue.Type.Delete, "dontcare"), //expired
-        /*12*/ KeyValueTestUtil.create("R1", "cf", "c", timestampType
-              .fromEpochTimeMillisToTimestamp(now - 600), KeyValue.Type.Put, "v1"), //expired
-        /*13*/ KeyValueTestUtil.create("R1", "cf", "c", timestampType
-              .fromEpochTimeMillisToTimestamp(now - 1000), KeyValue.Type.Delete, "dontcare"), //expired
-        /*14*/ KeyValueTestUtil.create("R1", "cf", "d", timestampType
-              .fromEpochTimeMillisToTimestamp(now - 60), KeyValue.Type.Put, "expired put"), //live
-        /*15*/ KeyValueTestUtil.create("R1", "cf", "d", timestampType
-          .fromEpochTimeMillisToTimestamp(now - 100), KeyValue.Type.Delete, "not-expired delete"), //live
+        now - 100, KeyValue.Type.DeleteColumn, "dont-care"), // max-version
+        /*8*/ KeyValueTestUtil.create("R1", "cf", "b", now - 600,
+        KeyValue.Type.DeleteColumn, "dont-care"), //expired
+        /*9*/ KeyValueTestUtil.create("R1", "cf", "b", now - 70,
+        KeyValue.Type.Put, "v2"), //live
+        /*10*/ KeyValueTestUtil.create("R1", "cf", "b", now - 750,
+        KeyValue.Type.Put, "v1"), //expired
+        /*11*/ KeyValueTestUtil.create("R1", "cf", "c", now - 500,
+        KeyValue.Type.Delete, "dontcare"), //expired
+        /*12*/ KeyValueTestUtil.create("R1", "cf", "c", now - 600,
+        KeyValue.Type.Put, "v1"), //expired
+        /*13*/ KeyValueTestUtil.create("R1", "cf", "c", now - 1000,
+        KeyValue.Type.Delete, "dontcare"), //expired
+        /*14*/ KeyValueTestUtil.create("R1", "cf", "d", now - 60,
+        KeyValue.Type.Put, "expired put"), //live
+        /*15*/ KeyValueTestUtil.create("R1", "cf", "d", now - 100,
+        KeyValue.Type.Delete, "not-expired delete"), //live
       };
       List<KeyValueScanner> scanners = scanFixture(kvs);
       Scan scan = new Scan();
@@ -1012,10 +972,10 @@ public class TestStoreScanner {
         HConstants.DEFAULT_BLOCKSIZE /* block size */,
         200, /* timeToPurgeDeletes */
         CellComparator.COMPARATOR);
-      Store store = mock(HStore.class);
-      when(store.getClock()).thenReturn(clock);
-      try (StoreScanner scanner = new StoreScanner(store, scan, scanInfo, ScanType
-          .COMPACT_DROP_DELETES, null, scanners, HConstants.OLDEST_TIMESTAMP)) {
+      try (StoreScanner scanner =
+        new StoreScanner(scan, scanInfo,
+          ScanType.COMPACT_DROP_DELETES, null, scanners,
+          HConstants.OLDEST_TIMESTAMP)) {
         List<Cell> results = new ArrayList<>();
         results = new ArrayList<>();
         Assert.assertEquals(true, scanner.next(results));

http://git-wip-us.apache.org/repos/asf/hbase/blob/c5abb6ca/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALLockup.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALLockup.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALLockup.java
index b5c7335..51260a6 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALLockup.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALLockup.java
@@ -19,7 +19,6 @@
 package org.apache.hadoop.hbase.regionserver;
 
 import static org.junit.Assert.assertTrue;
-import static org.mockito.Matchers.any;
 
 import java.io.IOException;
 import java.util.NavigableMap;
@@ -33,8 +32,6 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.CellScanner;
-import org.apache.hadoop.hbase.Clock;
-import org.apache.hadoop.hbase.ClockType;
 import org.apache.hadoop.hbase.ChoreService;
 import org.apache.hadoop.hbase.CoordinatedStateManager;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
@@ -206,7 +203,6 @@ public class TestWALLockup {
     Mockito.when(server.isStopped()).thenReturn(false);
     Mockito.when(server.isAborted()).thenReturn(false);
     RegionServerServices services = Mockito.mock(RegionServerServices.class);
-    Mockito.when(services.getRegionServerClock(ClockType.SYSTEM)).thenReturn(new Clock.System());
 
     // OK. Now I have my mocked up Server & RegionServerServices and dodgy WAL, go ahead with test.
     FileSystem fs = FileSystem.get(CONF);

http://git-wip-us.apache.org/repos/asf/hbase/blob/c5abb6ca/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestWALReplay.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestWALReplay.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestWALReplay.java
index aa571ad..be725fe 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestWALReplay.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestWALReplay.java
@@ -52,8 +52,6 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.PathFilter;
 import org.apache.hadoop.hbase.*;
-import org.apache.hadoop.hbase.Clock;
-import org.apache.hadoop.hbase.ClockType;
 import org.apache.hadoop.hbase.client.Delete;
 import org.apache.hadoop.hbase.client.Get;
 import org.apache.hadoop.hbase.client.Put;
@@ -659,23 +657,13 @@ public abstract class AbstractTestWALReplay {
    * @throws IOException
    */
   @Test
-  public void testReplayEditsAfterAbortingFlush() throws Exception {
-    testReplayEditsAfterAbortingFlush(new Clock.System());
-    setUp();
-    testReplayEditsAfterAbortingFlush(new Clock.SystemMonotonic());
-    tearDown();
-    setUp();
-    testReplayEditsAfterAbortingFlush(new Clock.HLC());
-  }
-
-  public void testReplayEditsAfterAbortingFlush(Clock clock) throws IOException {
+  public void testReplayEditsAfterAbortingFlush() throws IOException {
     final TableName tableName =
         TableName.valueOf("testReplayEditsAfterAbortingFlush");
     final HRegionInfo hri = createBasic3FamilyHRegionInfo(tableName);
     final Path basedir = FSUtils.getTableDir(this.hbaseRootDir, tableName);
     deleteDir(basedir);
     final HTableDescriptor htd = createBasic3FamilyHTD(tableName);
-    htd.setClockType(clock.clockType);
     HRegion region3 = HBaseTestingUtility.createRegionAndWAL(hri, hbaseRootDir, this.conf, htd);
     HBaseTestingUtility.closeRegionAndWAL(region3);
     // Write countPerFamily edits into the three families. Do a flush on one
@@ -684,7 +672,6 @@ public abstract class AbstractTestWALReplay {
     WAL wal = createWAL(this.conf, hbaseRootDir, logName);
     RegionServerServices rsServices = Mockito.mock(RegionServerServices.class);
     Mockito.doReturn(false).when(rsServices).isAborted();
-    when(rsServices.getRegionServerClock(clock.clockType)).thenReturn(clock);
     when(rsServices.getServerName()).thenReturn(ServerName.valueOf("foo", 10, 10));
     Configuration customConf = new Configuration(this.conf);
     customConf.set(DefaultStoreEngine.DEFAULT_STORE_FLUSHER_CLASS_KEY,

http://git-wip-us.apache.org/repos/asf/hbase/blob/c5abb6ca/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestCellACLWithMultipleVersions.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestCellACLWithMultipleVersions.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestCellACLWithMultipleVersions.java
index d39744b..88cdf1d 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestCellACLWithMultipleVersions.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestCellACLWithMultipleVersions.java
@@ -18,28 +18,22 @@
 package org.apache.hadoop.hbase.security.access;
 
 import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
 import java.io.IOException;
 import java.security.PrivilegedExceptionAction;
-import java.util.Arrays;
 import java.util.HashMap;
 import java.util.Map;
-import java.util.concurrent.TimeUnit;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.AuthUtil;
-import org.apache.hadoop.hbase.ClockType;
 import org.apache.hadoop.hbase.Coprocessor;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HColumnDescriptor;
-import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.TableNotFoundException;
-import org.apache.hadoop.hbase.TimestampType;
 import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.ConnectionFactory;
@@ -55,12 +49,9 @@ import org.apache.hadoop.hbase.security.access.Permission.Action;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.testclassification.SecurityTests;
 import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.DefaultEnvironmentEdge;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
-import org.apache.hadoop.hbase.util.ManualEnvironmentEdge;
 import org.apache.hadoop.hbase.util.TestTableName;
 import org.apache.hadoop.hbase.util.Threads;
-import org.apache.hadoop.util.Time;
 import org.apache.log4j.Level;
 import org.apache.log4j.Logger;
 import org.junit.After;
@@ -70,25 +61,9 @@ import org.junit.BeforeClass;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-import org.junit.runners.Parameterized.Parameters;
 
 @Category({SecurityTests.class, MediumTests.class})
-@RunWith(Parameterized.class)
 public class TestCellACLWithMultipleVersions extends SecureTestUtil {
-
-  @Parameters()
-  public static Iterable<Object> data() {
-    return Arrays.asList(new Object[] {ClockType
-        .SYSTEM, ClockType.SYSTEM_MONOTONIC, ClockType.HLC});
-  }
-
-  public TestCellACLWithMultipleVersions(ClockType clockType) {
-    this.clockType = clockType;
-    this.timestampType = clockType.timestampType();
-  }
-
   private static final Log LOG = LogFactory.getLog(TestCellACLWithMultipleVersions.class);
 
   static {
@@ -97,9 +72,6 @@ public class TestCellACLWithMultipleVersions extends SecureTestUtil {
     Logger.getLogger(TableAuthManager.class).setLevel(Level.TRACE);
   }
 
-  private final ClockType clockType;
-  private final TimestampType timestampType;
-
   @Rule
   public TestTableName TEST_TABLE = new TestTableName();
   private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
@@ -121,8 +93,6 @@ public class TestCellACLWithMultipleVersions extends SecureTestUtil {
   private static User USER_OTHER2;
 
   private static String[] usersAndGroups;
-  private static ManualEnvironmentEdge mee = new ManualEnvironmentEdge();
-  private static long now;
 
   @BeforeClass
   public static void setupBeforeClass() throws Exception {
@@ -166,9 +136,7 @@ public class TestCellACLWithMultipleVersions extends SecureTestUtil {
 
   @Before
   public void setUp() throws Exception {
-    EnvironmentEdgeManager.injectEdge(new DefaultEnvironmentEdge());
     HTableDescriptor htd = new HTableDescriptor(TEST_TABLE.getTableName());
-    htd.setClockType(clockType);
     HColumnDescriptor hcd = new HColumnDescriptor(TEST_FAMILY1);
     hcd.setMaxVersions(4);
     htd.setOwner(USER_OWNER);
@@ -417,12 +385,6 @@ public class TestCellACLWithMultipleVersions extends SecureTestUtil {
   public void testDeleteWithFutureTimestamp() throws Exception {
     // Store two values, one in the future
 
-    // Setting of future timestamps is not allowed with System Monotonic and HLC.
-    // So need not run this test against these two clocks.
-    if (clockType == ClockType.HLC || clockType == ClockType.SYSTEM_MONOTONIC) {
-      assertTrue(true);
-      return;
-    }
     verifyAllowed(new AccessTestAction() {
       @Override
       public Object run() throws Exception {
@@ -522,11 +484,8 @@ public class TestCellACLWithMultipleVersions extends SecureTestUtil {
           try (Table t = connection.getTable(TEST_TABLE.getTableName())) {
             // This version (TS = 123) with rw ACL for USER_OTHER and USER_OTHER2
             Put p = new Put(TEST_ROW);
-            EnvironmentEdgeManager.injectEdge(mee);
-            now += 10000;
-            mee.setValue(now);
-            p.addColumn(TEST_FAMILY1, TEST_Q1, HConstants.LATEST_TIMESTAMP, ZERO); //123
-            p.addColumn(TEST_FAMILY1, TEST_Q2, HConstants.LATEST_TIMESTAMP, ZERO); //123
+            p.addColumn(TEST_FAMILY1, TEST_Q1, 123L, ZERO);
+            p.addColumn(TEST_FAMILY1, TEST_Q2, 123L, ZERO);
             p.setACL(prepareCellPermissions(
               new String[] { USER_OTHER.getShortName(), AuthUtil.toGroupEntry(GROUP),
                   USER_OTHER2.getShortName() }, Permission.Action.READ, Permission.Action.WRITE));
@@ -534,9 +493,8 @@ public class TestCellACLWithMultipleVersions extends SecureTestUtil {
 
             // This version (TS = 125) with rw ACL for USER_OTHER
             p = new Put(TEST_ROW);
-            mee.setValue(now+2);
-            p.addColumn(TEST_FAMILY1, TEST_Q1, HConstants.LATEST_TIMESTAMP, ONE); //125
-            p.addColumn(TEST_FAMILY1, TEST_Q2, HConstants.LATEST_TIMESTAMP, ONE); //125
+            p.addColumn(TEST_FAMILY1, TEST_Q1, 125L, ONE);
+            p.addColumn(TEST_FAMILY1, TEST_Q2, 125L, ONE);
             p.setACL(prepareCellPermissions(
               new String[] { USER_OTHER.getShortName(), AuthUtil.toGroupEntry(GROUP) },
               Action.READ, Action.WRITE));
@@ -544,9 +502,8 @@ public class TestCellACLWithMultipleVersions extends SecureTestUtil {
 
             // This version (TS = 127) with rw ACL for USER_OTHER
             p = new Put(TEST_ROW);
-            mee.setValue(now+4);
-            p.addColumn(TEST_FAMILY1, TEST_Q1, HConstants.LATEST_TIMESTAMP, TWO); //127
-            p.addColumn(TEST_FAMILY1, TEST_Q2, HConstants.LATEST_TIMESTAMP, TWO); //127
+            p.addColumn(TEST_FAMILY1, TEST_Q1, 127L, TWO);
+            p.addColumn(TEST_FAMILY1, TEST_Q2, 127L, TWO);
             p.setACL(prepareCellPermissions(
               new String[] { USER_OTHER.getShortName(), AuthUtil.toGroupEntry(GROUP) },
               Action.READ, Action.WRITE));
@@ -564,8 +521,7 @@ public class TestCellACLWithMultipleVersions extends SecureTestUtil {
       public Object run() throws Exception {
         try (Connection connection = ConnectionFactory.createConnection(conf)) {
           try (Table t = connection.getTable(TEST_TABLE.getTableName())) {
-            Delete d = new Delete(TEST_ROW, timestampType.toTimestamp(TimeUnit.MILLISECONDS,
-                now+1, timestampType.getMaxLogicalTime())); //124
+            Delete d = new Delete(TEST_ROW, 124L);
             d.addColumns(TEST_FAMILY1, TEST_Q1);
             t.delete(d);
           }
@@ -581,8 +537,7 @@ public class TestCellACLWithMultipleVersions extends SecureTestUtil {
         try (Connection connection = ConnectionFactory.createConnection(conf)) {
           try (Table t = connection.getTable(TEST_TABLE.getTableName())) {
             Delete d = new Delete(TEST_ROW);
-            d.addColumns(TEST_FAMILY1, TEST_Q2, timestampType.toTimestamp(TimeUnit.MILLISECONDS,
-              now+1, timestampType.getMaxLogicalTime())); // 124
+            d.addColumns(TEST_FAMILY1, TEST_Q2, 124L);
             t.delete(d);
           }
         }
@@ -614,45 +569,37 @@ public class TestCellACLWithMultipleVersions extends SecureTestUtil {
                 prepareCellPermissions(
                   new String[] { user2.getShortName(), AuthUtil.toGroupEntry(GROUP),
                       USER_OWNER.getShortName() }, Action.READ, Action.WRITE);
-            now = EnvironmentEdgeManager.currentTime();
-            EnvironmentEdgeManager.injectEdge(mee);
-            now += 10000;
-            mee.setValue(now);
             Put p = new Put(TEST_ROW1);
-            p.addColumn(TEST_FAMILY1, TEST_Q1, HConstants.LATEST_TIMESTAMP, ZERO); //123
+            p.addColumn(TEST_FAMILY1, TEST_Q1, (long) 123, ZERO);
             p.setACL(permsU1andOwner);
             t.put(p);
             p = new Put(TEST_ROW1);
-            p.addColumn(TEST_FAMILY1, TEST_Q2, HConstants.LATEST_TIMESTAMP, ZERO); //123
+            p.addColumn(TEST_FAMILY1, TEST_Q2, (long) 123, ZERO);
             p.setACL(permsU2andGUandOwner);
             t.put(p);
             p = new Put(TEST_ROW1);
-            p.addColumn(TEST_FAMILY2, TEST_Q1, HConstants.LATEST_TIMESTAMP, ZERO); //123
-            p.addColumn(TEST_FAMILY2, TEST_Q2, HConstants.LATEST_TIMESTAMP, ZERO); //123
+            p.addColumn(TEST_FAMILY2, TEST_Q1, (long) 123, ZERO);
+            p.addColumn(TEST_FAMILY2, TEST_Q2, (long) 123, ZERO);
             p.setACL(permsU2andGUandOwner);
             t.put(p);
 
-            mee.setValue(now+2);
             p = new Put(TEST_ROW1);
-            p.addColumn(TEST_FAMILY2, TEST_Q1, HConstants.LATEST_TIMESTAMP, ZERO); //125
-            p.addColumn(TEST_FAMILY2, TEST_Q2, HConstants.LATEST_TIMESTAMP, ZERO); //125
+            p.addColumn(TEST_FAMILY2, TEST_Q1, (long) 125, ZERO);
+            p.addColumn(TEST_FAMILY2, TEST_Q2, (long) 125, ZERO);
             p.setACL(permsU1andOwner);
             t.put(p);
 
-            mee.setValue(now+4);
             p = new Put(TEST_ROW1);
-            p.addColumn(TEST_FAMILY1, TEST_Q1, HConstants.LATEST_TIMESTAMP, ZERO); //127
+            p.addColumn(TEST_FAMILY1, TEST_Q1, (long) 127, ZERO);
             p.setACL(permsU2andGUandOwner);
             t.put(p);
             p = new Put(TEST_ROW1);
-            p.addColumn(TEST_FAMILY1, TEST_Q2, HConstants.LATEST_TIMESTAMP, ZERO); //127
+            p.addColumn(TEST_FAMILY1, TEST_Q2, (long) 127, ZERO);
             p.setACL(permsU1andOwner);
             t.put(p);
-
-            mee.setValue(now+6);
             p = new Put(TEST_ROW1);
-            p.addColumn(TEST_FAMILY2, TEST_Q1, HConstants.LATEST_TIMESTAMP, ZERO); //129
-            p.addColumn(TEST_FAMILY2, TEST_Q2, HConstants.LATEST_TIMESTAMP, ZERO); //129
+            p.addColumn(TEST_FAMILY2, TEST_Q1, (long) 129, ZERO);
+            p.addColumn(TEST_FAMILY2, TEST_Q2, (long) 129, ZERO);
             p.setACL(permsU1andOwner);
             t.put(p);
           }
@@ -669,11 +616,9 @@ public class TestCellACLWithMultipleVersions extends SecureTestUtil {
         try (Connection connection = ConnectionFactory.createConnection(conf)) {
           try (Table t = connection.getTable(TEST_TABLE.getTableName())) {
             Delete d = new Delete(TEST_ROW1);
-            d.addColumn(TEST_FAMILY1, TEST_Q1, timestampType.toTimestamp(TimeUnit.MILLISECONDS,
-              now, timestampType.getMaxLogicalTime())); //123
+            d.addColumn(TEST_FAMILY1, TEST_Q1, 123);
             d.addColumn(TEST_FAMILY1, TEST_Q2);
-            d.addFamilyVersion(TEST_FAMILY2, timestampType.toTimestamp(TimeUnit.MILLISECONDS,
-              now+2, timestampType.getMaxLogicalTime())); //125
+            d.addFamilyVersion(TEST_FAMILY2, 125);
             t.delete(d);
           }
         }
@@ -692,12 +637,10 @@ public class TestCellACLWithMultipleVersions extends SecureTestUtil {
       public Void run() throws Exception {
         try (Connection connection = ConnectionFactory.createConnection(conf)) {
           try (Table t = connection.getTable(TEST_TABLE.getTableName())) {
-            Delete d = new Delete(row, timestampType.toTimestamp(TimeUnit.MILLISECONDS,
-              now+4, timestampType.getMaxLogicalTime())); //127
+            Delete d = new Delete(row, 127);
             d.addColumns(TEST_FAMILY1, q1);
             d.addColumns(TEST_FAMILY1, q2);
-            d.addFamily(TEST_FAMILY2, timestampType.toTimestamp(TimeUnit.MILLISECONDS,
-              now+6, timestampType.getMaxLogicalTime())); //129
+            d.addFamily(TEST_FAMILY2, 129);
             t.delete(d);
             fail(user.getShortName() + " can not do the delete");
           } catch (Exception e) {
@@ -732,27 +675,21 @@ public class TestCellACLWithMultipleVersions extends SecureTestUtil {
                 prepareCellPermissions(
                   new String[] { user2.getShortName(), AuthUtil.toGroupEntry(GROUP),
                       USER_OWNER.getShortName() }, Action.READ, Action.WRITE);
-            now = EnvironmentEdgeManager.currentTime();
-            EnvironmentEdgeManager.injectEdge(mee);
-            now += 10000;
-            mee.setValue(now);
             Put p = new Put(TEST_ROW1);
-            p.addColumn(TEST_FAMILY1, TEST_Q1, HConstants.LATEST_TIMESTAMP, ZERO); //123
+            p.addColumn(TEST_FAMILY1, TEST_Q1, (long) 123, ZERO);
             p.setACL(permsU1andOwner);
             t.put(p);
             p = new Put(TEST_ROW1);
-            p.addColumn(TEST_FAMILY1, TEST_Q2, HConstants.LATEST_TIMESTAMP, ZERO); //123
+            p.addColumn(TEST_FAMILY1, TEST_Q2, (long) 123, ZERO);
             p.setACL(permsU2andGUandOwner);
             t.put(p);
 
-            System.out.println(now+4);
-            mee.setValue(now+4);
             p = new Put(TEST_ROW1);
-            p.addColumn(TEST_FAMILY1, TEST_Q1, HConstants.LATEST_TIMESTAMP, ZERO); //127
+            p.addColumn(TEST_FAMILY1, TEST_Q1, (long) 127, ZERO);
             p.setACL(permsU2andGUandOwner);
             t.put(p);
             p = new Put(TEST_ROW1);
-            p.addColumn(TEST_FAMILY1, TEST_Q2, HConstants.LATEST_TIMESTAMP, ZERO); //127
+            p.addColumn(TEST_FAMILY1, TEST_Q2, (long) 127, ZERO);
             p.setACL(permsU1andOwner);
             t.put(p);
           }
@@ -768,8 +705,7 @@ public class TestCellACLWithMultipleVersions extends SecureTestUtil {
         try (Connection connection = ConnectionFactory.createConnection(conf)) {
           try (Table t = connection.getTable(TEST_TABLE.getTableName())) {
             Increment inc = new Increment(TEST_ROW1);
-            inc.setTimeRange(0, timestampType.toTimestamp(TimeUnit.MILLISECONDS, now,
-              timestampType.getMaxLogicalTime()));
+            inc.setTimeRange(0, 123);
             inc.addColumn(TEST_FAMILY1, TEST_Q1, 2L);
             t.increment(inc);
             t.incrementColumnValue(TEST_ROW1, TEST_FAMILY1, TEST_Q2, 1L);
@@ -791,9 +727,7 @@ public class TestCellACLWithMultipleVersions extends SecureTestUtil {
         try (Connection connection = ConnectionFactory.createConnection(conf)) {
           try (Table t = connection.getTable(TEST_TABLE.getTableName())) {
             Increment inc = new Increment(row);
-            System.out.println(now+4);
-            inc.setTimeRange(0, timestampType.toTimestamp(TimeUnit.MILLISECONDS, now+4,
-              timestampType.getMaxLogicalTime()));
+            inc.setTimeRange(0, 127);
             inc.addColumn(TEST_FAMILY1, q1, 2L);
             t.increment(inc);
             fail(user.getShortName() + " cannot do the increment.");
@@ -808,14 +742,6 @@ public class TestCellACLWithMultipleVersions extends SecureTestUtil {
 
   @Test
   public void testCellPermissionsForPutWithMultipleVersions() throws Exception {
-
-    // This test relies is dependent on non monotonic timestamp updates which doesn't happen with
-    // HLC and System Monotonic Clocks.
-    if (clockType == ClockType.HLC || clockType == ClockType.SYSTEM_MONOTONIC) {
-      assertTrue(true);
-      return;
-    }
-
     final byte[] TEST_ROW1 = Bytes.toBytes("r1");
     final byte[] TEST_Q1 = Bytes.toBytes("q1");
     final byte[] TEST_Q2 = Bytes.toBytes("q2");
@@ -931,45 +857,38 @@ public class TestCellACLWithMultipleVersions extends SecureTestUtil {
                   Action.WRITE);
             Map<String, Permission> permsU1andU2andGUandOwner =
                 prepareCellPermissions(new String[] { user1.getShortName(), user2.getShortName(),
-                      AuthUtil.toGroupEntry(GROUP), USER_OWNER.getShortName() }, Action.READ,
+                    AuthUtil.toGroupEntry(GROUP), USER_OWNER.getShortName() }, Action.READ,
                   Action.WRITE);
             Map<String, Permission> permsU1_U2andGU =
                 prepareCellPermissions(new String[] { user1.getShortName(), user2.getShortName(),
                     AuthUtil.toGroupEntry(GROUP) }, Action.READ, Action.WRITE);
 
-            now = EnvironmentEdgeManager.currentTime();
-            EnvironmentEdgeManager.injectEdge(mee);
-            now += 5000;
-
-            mee.setValue(now);
             Put p = new Put(TEST_ROW1);
-            p.addColumn(TEST_FAMILY1, TEST_Q1, HConstants.LATEST_TIMESTAMP, ZERO); //120
-            p.addColumn(TEST_FAMILY1, TEST_Q2, HConstants.LATEST_TIMESTAMP, ZERO);
-            p.addColumn(TEST_FAMILY1, TEST_Q3, HConstants.LATEST_TIMESTAMP, ZERO);
+            p.addColumn(TEST_FAMILY1, TEST_Q1, (long) 120, ZERO);
+            p.addColumn(TEST_FAMILY1, TEST_Q2, (long) 120, ZERO);
+            p.addColumn(TEST_FAMILY1, TEST_Q3, (long) 120, ZERO);
             p.setACL(permsU1andU2andGUandOwner);
             t.put(p);
 
-            mee.setValue(now+3);
             p = new Put(TEST_ROW1);
-            p.addColumn(TEST_FAMILY1, TEST_Q1, HConstants.LATEST_TIMESTAMP, ZERO); //123
-            p.addColumn(TEST_FAMILY1, TEST_Q2, HConstants.LATEST_TIMESTAMP, ZERO);
-            p.addColumn(TEST_FAMILY1, TEST_Q3, HConstants.LATEST_TIMESTAMP, ZERO);
+            p.addColumn(TEST_FAMILY1, TEST_Q1, (long) 123, ZERO);
+            p.addColumn(TEST_FAMILY1, TEST_Q2, (long) 123, ZERO);
+            p.addColumn(TEST_FAMILY1, TEST_Q3, (long) 123, ZERO);
             p.setACL(permsU1andOwner);
             t.put(p);
 
-            mee.setValue(now+7);
             p = new Put(TEST_ROW1);
-            p.addColumn(TEST_FAMILY1, TEST_Q1, HConstants.LATEST_TIMESTAMP, ZERO); //127
+            p.addColumn(TEST_FAMILY1, TEST_Q1, (long) 127, ZERO);
             p.setACL(permsU1_U2andGU);
             t.put(p);
 
             p = new Put(TEST_ROW1);
-            p.addColumn(TEST_FAMILY1, TEST_Q2, HConstants.LATEST_TIMESTAMP, ZERO); //127
+            p.addColumn(TEST_FAMILY1, TEST_Q2, (long) 127, ZERO);
             p.setACL(user2.getShortName(), new Permission(Permission.Action.READ));
             t.put(p);
 
             p = new Put(TEST_ROW1);
-            p.addColumn(TEST_FAMILY1, TEST_Q3, HConstants.LATEST_TIMESTAMP, ZERO); //127
+            p.addColumn(TEST_FAMILY1, TEST_Q3, 127, ZERO);
             p.setACL(AuthUtil.toGroupEntry(GROUP), new Permission(Permission.Action.READ));
             t.put(p);
           }
@@ -986,8 +905,7 @@ public class TestCellACLWithMultipleVersions extends SecureTestUtil {
         try (Connection connection = ConnectionFactory.createConnection(conf)) {
           try (Table t = connection.getTable(TEST_TABLE.getTableName())) {
             Delete d = new Delete(TEST_ROW1);
-            d.addColumns(TEST_FAMILY1, TEST_Q1, timestampType.toTimestamp(TimeUnit.MILLISECONDS,
-              now, timestampType.getMaxLogicalTime())); //120
+            d.addColumns(TEST_FAMILY1, TEST_Q1, 120);
             t.checkAndDelete(TEST_ROW1, TEST_FAMILY1, TEST_Q1, ZERO, d);
           }
         }
@@ -1023,8 +941,7 @@ public class TestCellACLWithMultipleVersions extends SecureTestUtil {
         try (Connection connection = ConnectionFactory.createConnection(conf)) {
           try (Table t = connection.getTable(TEST_TABLE.getTableName())) {
             Delete d = new Delete(row);
-            d.addColumn(TEST_FAMILY1, q1, timestampType.toTimestamp(TimeUnit.MILLISECONDS,
-              now, timestampType.getMaxLogicalTime()));
+            d.addColumn(TEST_FAMILY1, q1, 120);
             t.checkAndDelete(row, TEST_FAMILY1, q1, value, d);
           }
         }