You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by ap...@apache.org on 2017/07/17 17:59:24 UTC

[91/94] [abbrv] hbase git commit: HBASE-14070 - Core HLC (Sai Teja Ranuva) Rebased by Amit Patel

http://git-wip-us.apache.org/repos/asf/hbase/blob/d53fbc74/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 be725fe..b403560 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,6 +52,8 @@ 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;
@@ -59,6 +61,7 @@ import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.ResultScanner;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
 import org.apache.hadoop.hbase.master.HMaster;
 import org.apache.hadoop.hbase.monitoring.MonitoredTask;
 import org.apache.hadoop.hbase.regionserver.CompactingMemStore;
@@ -650,20 +653,34 @@ public abstract class AbstractTestWALReplay {
 
   };
 
+  @Test
+  public void testReplayEditsAfterAbortingFlushWithHybridLogicalClock() throws Exception {
+    testReplayEditsAfterAbortingFlush(new Clock.HLC());
+  }
+
+  @Test
+  public void testReplayEditsAfterAbortingFlushWithSystemMonotonicClock() throws Exception {
+    testReplayEditsAfterAbortingFlush(new Clock.SystemMonotonic());
+  }
+
+  @Test
+  public void testReplayEditsAfterAbortingFlushWithSystemClock() throws Exception {
+    testReplayEditsAfterAbortingFlush(new Clock.System());
+  }
+
   /**
    * Test that we could recover the data correctly after aborting flush. In the
    * test, first we abort flush after writing some data, then writing more data
    * and flush again, at last verify the data.
    * @throws IOException
    */
-  @Test
-  public void testReplayEditsAfterAbortingFlush() throws IOException {
+  public void testReplayEditsAfterAbortingFlush(Clock clock) 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);
+    final HTableDescriptor htd = createBasic3FamilyHTD(tableName, clock.getClockType());
     HRegion region3 = HBaseTestingUtility.createRegionAndWAL(hri, hbaseRootDir, this.conf, htd);
     HBaseTestingUtility.closeRegionAndWAL(region3);
     // Write countPerFamily edits into the three families. Do a flush on one
@@ -672,6 +689,7 @@ 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.getClockType())).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,
@@ -1211,14 +1229,21 @@ public abstract class AbstractTestWALReplay {
   }
 
   private HTableDescriptor createBasic3FamilyHTD(final TableName tableName) {
-    HTableDescriptor htd = new HTableDescriptor(tableName);
-    HColumnDescriptor a = new HColumnDescriptor(Bytes.toBytes("a"));
-    htd.addFamily(a);
-    HColumnDescriptor b = new HColumnDescriptor(Bytes.toBytes("b"));
-    htd.addFamily(b);
-    HColumnDescriptor c = new HColumnDescriptor(Bytes.toBytes("c"));
-    htd.addFamily(c);
-    return htd;
+    return new HTableDescriptor(TableDescriptorBuilder.newBuilder(tableName)
+      .addColumnFamily(new HColumnDescriptor(Bytes.toBytes("a")))
+      .addColumnFamily(new HColumnDescriptor(Bytes.toBytes("b")))
+      .addColumnFamily(new HColumnDescriptor(Bytes.toBytes("c")))
+      .build());
+  }
+
+  private HTableDescriptor createBasic3FamilyHTD(final TableName tableName,
+      final ClockType clockType) {
+    return new HTableDescriptor(TableDescriptorBuilder.newBuilder(tableName)
+      .addColumnFamily(new HColumnDescriptor(Bytes.toBytes("a")))
+      .addColumnFamily(new HColumnDescriptor(Bytes.toBytes("b")))
+      .addColumnFamily(new HColumnDescriptor(Bytes.toBytes("c")))
+      .setClockType(clockType)
+      .build());
   }
 
   private void writerWALFile(Path file, List<FSWALEntry> entries) throws IOException {

http://git-wip-us.apache.org/repos/asf/hbase/blob/d53fbc74/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 88cdf1d..c552384 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,22 +18,28 @@
 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;
@@ -42,6 +48,7 @@ import org.apache.hadoop.hbase.client.Get;
 import org.apache.hadoop.hbase.client.Increment;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
 import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
 import org.apache.hadoop.hbase.regionserver.RegionServerCoprocessorHost;
 import org.apache.hadoop.hbase.security.User;
@@ -49,9 +56,12 @@ 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;
@@ -61,9 +71,25 @@ 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 {
@@ -72,6 +98,9 @@ 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();
@@ -93,6 +122,8 @@ 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 {
@@ -136,15 +167,17 @@ public class TestCellACLWithMultipleVersions extends SecureTestUtil {
 
   @Before
   public void setUp() throws Exception {
-    HTableDescriptor htd = new HTableDescriptor(TEST_TABLE.getTableName());
-    HColumnDescriptor hcd = new HColumnDescriptor(TEST_FAMILY1);
-    hcd.setMaxVersions(4);
-    htd.setOwner(USER_OWNER);
-    htd.addFamily(hcd);
-    hcd = new HColumnDescriptor(TEST_FAMILY2);
-    hcd.setMaxVersions(4);
-    htd.setOwner(USER_OWNER);
-    htd.addFamily(hcd);
+    EnvironmentEdgeManager.injectEdge(new DefaultEnvironmentEdge());
+
+    HTableDescriptor htd = new HTableDescriptor(TableDescriptorBuilder
+      .newBuilder(TEST_TABLE.getTableName())
+      .addColumnFamily(new HColumnDescriptor(TEST_FAMILY1)
+        .setMaxVersions(4))
+      .addColumnFamily(new HColumnDescriptor(TEST_FAMILY2)
+        .setMaxVersions(4))
+      .setOwner(USER_OWNER)
+      .setClockType(clockType)
+      .build());
     // Create the test table (owner added to the _acl_ table)
     try (Connection connection = ConnectionFactory.createConnection(TEST_UTIL.getConfiguration())) {
       try (Admin admin = connection.getAdmin()) {
@@ -385,6 +418,12 @@ 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 {
@@ -484,8 +523,11 @@ 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);
-            p.addColumn(TEST_FAMILY1, TEST_Q1, 123L, ZERO);
-            p.addColumn(TEST_FAMILY1, TEST_Q2, 123L, ZERO);
+            EnvironmentEdgeManager.injectEdge(mee);
+            now += 123;
+            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.setACL(prepareCellPermissions(
               new String[] { USER_OTHER.getShortName(), AuthUtil.toGroupEntry(GROUP),
                   USER_OTHER2.getShortName() }, Permission.Action.READ, Permission.Action.WRITE));
@@ -493,8 +535,9 @@ public class TestCellACLWithMultipleVersions extends SecureTestUtil {
 
             // This version (TS = 125) with rw ACL for USER_OTHER
             p = new Put(TEST_ROW);
-            p.addColumn(TEST_FAMILY1, TEST_Q1, 125L, ONE);
-            p.addColumn(TEST_FAMILY1, TEST_Q2, 125L, ONE);
+            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.setACL(prepareCellPermissions(
               new String[] { USER_OTHER.getShortName(), AuthUtil.toGroupEntry(GROUP) },
               Action.READ, Action.WRITE));
@@ -502,8 +545,9 @@ public class TestCellACLWithMultipleVersions extends SecureTestUtil {
 
             // This version (TS = 127) with rw ACL for USER_OTHER
             p = new Put(TEST_ROW);
-            p.addColumn(TEST_FAMILY1, TEST_Q1, 127L, TWO);
-            p.addColumn(TEST_FAMILY1, TEST_Q2, 127L, TWO);
+            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.setACL(prepareCellPermissions(
               new String[] { USER_OTHER.getShortName(), AuthUtil.toGroupEntry(GROUP) },
               Action.READ, Action.WRITE));
@@ -521,7 +565,8 @@ 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, 124L);
+            Delete d = new Delete(TEST_ROW, timestampType.toTimestamp(TimeUnit.MILLISECONDS,
+                now+1, timestampType.getMaxLogicalTime())); //124
             d.addColumns(TEST_FAMILY1, TEST_Q1);
             t.delete(d);
           }
@@ -537,7 +582,8 @@ 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, 124L);
+            d.addColumns(TEST_FAMILY1, TEST_Q2, timestampType.toTimestamp(TimeUnit.MILLISECONDS,
+              now+1, timestampType.getMaxLogicalTime())); // 124
             t.delete(d);
           }
         }
@@ -569,37 +615,45 @@ 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 += 123;
+            mee.setValue(now);
             Put p = new Put(TEST_ROW1);
-            p.addColumn(TEST_FAMILY1, TEST_Q1, (long) 123, ZERO);
+            p.addColumn(TEST_FAMILY1, TEST_Q1, HConstants.LATEST_TIMESTAMP, ZERO); //123
             p.setACL(permsU1andOwner);
             t.put(p);
             p = new Put(TEST_ROW1);
-            p.addColumn(TEST_FAMILY1, TEST_Q2, (long) 123, ZERO);
+            p.addColumn(TEST_FAMILY1, TEST_Q2, HConstants.LATEST_TIMESTAMP, ZERO); //123
             p.setACL(permsU2andGUandOwner);
             t.put(p);
             p = new Put(TEST_ROW1);
-            p.addColumn(TEST_FAMILY2, TEST_Q1, (long) 123, ZERO);
-            p.addColumn(TEST_FAMILY2, TEST_Q2, (long) 123, ZERO);
+            p.addColumn(TEST_FAMILY2, TEST_Q1, HConstants.LATEST_TIMESTAMP, ZERO); //123
+            p.addColumn(TEST_FAMILY2, TEST_Q2, HConstants.LATEST_TIMESTAMP, ZERO); //123
             p.setACL(permsU2andGUandOwner);
             t.put(p);
 
+            mee.setValue(now+2);
             p = new Put(TEST_ROW1);
-            p.addColumn(TEST_FAMILY2, TEST_Q1, (long) 125, ZERO);
-            p.addColumn(TEST_FAMILY2, TEST_Q2, (long) 125, ZERO);
+            p.addColumn(TEST_FAMILY2, TEST_Q1, HConstants.LATEST_TIMESTAMP, ZERO); //125
+            p.addColumn(TEST_FAMILY2, TEST_Q2, HConstants.LATEST_TIMESTAMP, ZERO); //125
             p.setACL(permsU1andOwner);
             t.put(p);
 
+            mee.setValue(now+4);
             p = new Put(TEST_ROW1);
-            p.addColumn(TEST_FAMILY1, TEST_Q1, (long) 127, ZERO);
+            p.addColumn(TEST_FAMILY1, TEST_Q1, HConstants.LATEST_TIMESTAMP, ZERO); //127
             p.setACL(permsU2andGUandOwner);
             t.put(p);
             p = new Put(TEST_ROW1);
-            p.addColumn(TEST_FAMILY1, TEST_Q2, (long) 127, ZERO);
+            p.addColumn(TEST_FAMILY1, TEST_Q2, HConstants.LATEST_TIMESTAMP, ZERO); //127
             p.setACL(permsU1andOwner);
             t.put(p);
+
+            mee.setValue(now+6);
             p = new Put(TEST_ROW1);
-            p.addColumn(TEST_FAMILY2, TEST_Q1, (long) 129, ZERO);
-            p.addColumn(TEST_FAMILY2, TEST_Q2, (long) 129, ZERO);
+            p.addColumn(TEST_FAMILY2, TEST_Q1, HConstants.LATEST_TIMESTAMP, ZERO); //129
+            p.addColumn(TEST_FAMILY2, TEST_Q2, HConstants.LATEST_TIMESTAMP, ZERO); //129
             p.setACL(permsU1andOwner);
             t.put(p);
           }
@@ -616,9 +670,11 @@ 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, 123);
+            d.addColumn(TEST_FAMILY1, TEST_Q1, timestampType.toTimestamp(TimeUnit.MILLISECONDS,
+              now, timestampType.getMaxLogicalTime())); //123
             d.addColumn(TEST_FAMILY1, TEST_Q2);
-            d.addFamilyVersion(TEST_FAMILY2, 125);
+            d.addFamilyVersion(TEST_FAMILY2, timestampType.toTimestamp(TimeUnit.MILLISECONDS,
+              now+2, timestampType.getMaxLogicalTime())); //125
             t.delete(d);
           }
         }
@@ -637,10 +693,12 @@ 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, 127);
+            Delete d = new Delete(row, timestampType.toTimestamp(TimeUnit.MILLISECONDS,
+              now+4, timestampType.getMaxLogicalTime())); //127
             d.addColumns(TEST_FAMILY1, q1);
             d.addColumns(TEST_FAMILY1, q2);
-            d.addFamily(TEST_FAMILY2, 129);
+            d.addFamily(TEST_FAMILY2, timestampType.toTimestamp(TimeUnit.MILLISECONDS,
+              now+6, timestampType.getMaxLogicalTime())); //129
             t.delete(d);
             fail(user.getShortName() + " can not do the delete");
           } catch (Exception e) {
@@ -675,21 +733,26 @@ 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 += 123;
+            mee.setValue(now);
             Put p = new Put(TEST_ROW1);
-            p.addColumn(TEST_FAMILY1, TEST_Q1, (long) 123, ZERO);
+            p.addColumn(TEST_FAMILY1, TEST_Q1, HConstants.LATEST_TIMESTAMP, ZERO); //123
             p.setACL(permsU1andOwner);
             t.put(p);
             p = new Put(TEST_ROW1);
-            p.addColumn(TEST_FAMILY1, TEST_Q2, (long) 123, ZERO);
+            p.addColumn(TEST_FAMILY1, TEST_Q2, HConstants.LATEST_TIMESTAMP, ZERO); //123
             p.setACL(permsU2andGUandOwner);
             t.put(p);
 
+            mee.setValue(now+4);
             p = new Put(TEST_ROW1);
-            p.addColumn(TEST_FAMILY1, TEST_Q1, (long) 127, ZERO);
+            p.addColumn(TEST_FAMILY1, TEST_Q1, HConstants.LATEST_TIMESTAMP, ZERO); //127
             p.setACL(permsU2andGUandOwner);
             t.put(p);
             p = new Put(TEST_ROW1);
-            p.addColumn(TEST_FAMILY1, TEST_Q2, (long) 127, ZERO);
+            p.addColumn(TEST_FAMILY1, TEST_Q2, HConstants.LATEST_TIMESTAMP, ZERO); //127
             p.setACL(permsU1andOwner);
             t.put(p);
           }
@@ -705,7 +768,8 @@ 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, 123);
+            inc.setTimeRange(0, timestampType.toTimestamp(TimeUnit.MILLISECONDS, now,
+              timestampType.getMaxLogicalTime()));
             inc.addColumn(TEST_FAMILY1, TEST_Q1, 2L);
             t.increment(inc);
             t.incrementColumnValue(TEST_ROW1, TEST_FAMILY1, TEST_Q2, 1L);
@@ -727,7 +791,8 @@ public class TestCellACLWithMultipleVersions extends SecureTestUtil {
         try (Connection connection = ConnectionFactory.createConnection(conf)) {
           try (Table t = connection.getTable(TEST_TABLE.getTableName())) {
             Increment inc = new Increment(row);
-            inc.setTimeRange(0, 127);
+            inc.setTimeRange(0, timestampType.toTimestamp(TimeUnit.MILLISECONDS, now+4,
+              timestampType.getMaxLogicalTime()));
             inc.addColumn(TEST_FAMILY1, q1, 2L);
             t.increment(inc);
             fail(user.getShortName() + " cannot do the increment.");
@@ -742,6 +807,14 @@ 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");
@@ -857,38 +930,45 @@ 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 += 120;
+
+            mee.setValue(now);
             Put p = new Put(TEST_ROW1);
-            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.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.setACL(permsU1andU2andGUandOwner);
             t.put(p);
 
+            mee.setValue(now+3);
             p = new Put(TEST_ROW1);
-            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.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.setACL(permsU1andOwner);
             t.put(p);
 
+            mee.setValue(now+7);
             p = new Put(TEST_ROW1);
-            p.addColumn(TEST_FAMILY1, TEST_Q1, (long) 127, ZERO);
+            p.addColumn(TEST_FAMILY1, TEST_Q1, HConstants.LATEST_TIMESTAMP, ZERO); //127
             p.setACL(permsU1_U2andGU);
             t.put(p);
 
             p = new Put(TEST_ROW1);
-            p.addColumn(TEST_FAMILY1, TEST_Q2, (long) 127, ZERO);
+            p.addColumn(TEST_FAMILY1, TEST_Q2, HConstants.LATEST_TIMESTAMP, ZERO); //127
             p.setACL(user2.getShortName(), new Permission(Permission.Action.READ));
             t.put(p);
 
             p = new Put(TEST_ROW1);
-            p.addColumn(TEST_FAMILY1, TEST_Q3, 127, ZERO);
+            p.addColumn(TEST_FAMILY1, TEST_Q3, HConstants.LATEST_TIMESTAMP, ZERO); //127
             p.setACL(AuthUtil.toGroupEntry(GROUP), new Permission(Permission.Action.READ));
             t.put(p);
           }
@@ -905,7 +985,8 @@ 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, 120);
+            d.addColumns(TEST_FAMILY1, TEST_Q1, timestampType.toTimestamp(TimeUnit.MILLISECONDS,
+              now, timestampType.getMaxLogicalTime())); //120
             t.checkAndDelete(TEST_ROW1, TEST_FAMILY1, TEST_Q1, ZERO, d);
           }
         }
@@ -941,7 +1022,8 @@ 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, 120);
+            d.addColumn(TEST_FAMILY1, q1, timestampType.toTimestamp(TimeUnit.MILLISECONDS,
+              now, timestampType.getMaxLogicalTime()));
             t.checkAndDelete(row, TEST_FAMILY1, q1, value, d);
           }
         }

http://git-wip-us.apache.org/repos/asf/hbase/blob/d53fbc74/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestCoprocessorScanPolicy.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestCoprocessorScanPolicy.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestCoprocessorScanPolicy.java
index 920d3ac..dfd34a4 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestCoprocessorScanPolicy.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestCoprocessorScanPolicy.java
@@ -31,6 +31,7 @@ import java.util.NavigableSet;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellUtil;
+import org.apache.hadoop.hbase.ClockType;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HConstants;
@@ -45,6 +46,7 @@ import org.apache.hadoop.hbase.client.Put;
 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.client.TableDescriptorBuilder;
 import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
 import org.apache.hadoop.hbase.coprocessor.ObserverContext;
 import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
@@ -148,25 +150,32 @@ public class TestCoprocessorScanPolicy {
   }
 
   @Test
-  public void testTTL() throws Exception {
+  public void testTTl() throws Exception {
+    testTTL(ClockType.HLC);
+    testTTL(ClockType.SYSTEM_MONOTONIC);
+    testTTL(ClockType.SYSTEM);
+  }
+
+  public void testTTL(ClockType clockType) throws Exception {
     TableName tableName =
         TableName.valueOf("testTTL");
     if (TEST_UTIL.getAdmin().tableExists(tableName)) {
       TEST_UTIL.deleteTable(tableName);
     }
-    HTableDescriptor desc = new HTableDescriptor(tableName);
-    HColumnDescriptor hcd = new HColumnDescriptor(F)
-    .setMaxVersions(10)
-    .setTimeToLive(1);
-    desc.addFamily(hcd);
+    HTableDescriptor desc = new HTableDescriptor(TableDescriptorBuilder.newBuilder(tableName)
+      .addColumnFamily(new HColumnDescriptor(F)
+        .setMaxVersions(10)
+        .setTimeToLive(1))
+      .setClockType(clockType)
+      .build());
     TEST_UTIL.getAdmin().createTable(desc);
     Table t = TEST_UTIL.getConnection().getTable(tableName);
     long now = EnvironmentEdgeManager.currentTime();
     ManualEnvironmentEdge me = new ManualEnvironmentEdge();
-    me.setValue(now);
+    me.setValue(now-2000);
     EnvironmentEdgeManagerTestHelper.injectEdge(me);
     // 2s in the past
-    long ts = now - 2000;
+    long ts = Long.MAX_VALUE;
     // Set the TTL override to 3s
     Put p = new Put(R);
     p.setAttribute("ttl", new byte[]{});
@@ -176,12 +185,15 @@ public class TestCoprocessorScanPolicy {
     p = new Put(R);
     p.addColumn(F, Q, ts, Q);
     t.put(p);
+
+    me.setValue(now-1999);
     p = new Put(R);
-    p.addColumn(F, Q, ts + 1, Q);
+    p.addColumn(F, Q, ts , Q);
     t.put(p);
 
     // these two should be expired but for the override
     // (their ts was 2s in the past)
+    me.setValue(now);
     Get g = new Get(R);
     g.setMaxVersions(10);
     Result r = t.get(g);

http://git-wip-us.apache.org/repos/asf/hbase/blob/d53fbc74/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestTableName.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestTableName.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestTableName.java
index f585f47..4284338 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestTableName.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestTableName.java
@@ -46,7 +46,10 @@ public class TestTableName extends TestWatcher {
    */
   @Override
   protected void starting(Description description) {
-    tableName = TableName.valueOf(description.getMethodName());
+    // Parameterized test names have brackets, which are are invalid characters for table names
+    tableName = TableName.valueOf(description.getMethodName()
+        .replace("[", ".")
+        .replace("]", ""));
   }
 
   public TableName getTableName() {