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/08/16 00:41:16 UTC

hbase git commit: HBASE-18509 Cleanup Clock interface.

Repository: hbase
Updated Branches:
  refs/heads/HBASE-14070.HLC 035dd8b53 -> 82a9cec59


HBASE-18509 Cleanup Clock interface.

- Moved implementations out and renamed to a more consistent naming (SystemClock, SystemMonotonicClock, HybridLogicalClock)
- Moved ClockException out
- Added InterfaceAudience.Private to all classes.

Change-Id: Icb9ed6c5411d140c9fa08af5f8bda15ba7ad0092
Signed-off-by: Apekshit Sharma <ap...@apache.org>


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

Branch: refs/heads/HBASE-14070.HLC
Commit: 82a9cec595d165a715a0406e5b280bec8fcf0e88
Parents: 035dd8b
Author: Amit Patel <ia...@gmail.com>
Authored: Tue Aug 15 08:32:54 2017 -0700
Committer: Apekshit Sharma <ap...@apache.org>
Committed: Tue Aug 15 17:26:57 2017 -0700

----------------------------------------------------------------------
 .../hbase/client/TableDescriptorBuilder.java    |   4 +-
 .../hbase/shaded/protobuf/ProtobufUtil.java     |   4 +-
 .../java/org/apache/hadoop/hbase/Clock.java     | 267 +------------------
 .../org/apache/hadoop/hbase/ClockException.java |  33 +++
 .../java/org/apache/hadoop/hbase/ClockType.java |   2 +-
 .../apache/hadoop/hbase/HybridLogicalClock.java | 134 ++++++++++
 .../apache/hadoop/hbase/SettableTimestamp.java  |   2 +-
 .../org/apache/hadoop/hbase/SystemClock.java    |  44 +++
 .../hadoop/hbase/SystemMonotonicClock.java      |  83 ++++++
 .../org/apache/hadoop/hbase/TimestampType.java  |   1 -
 .../java/org/apache/hadoop/hbase/TestClock.java |  51 ++--
 .../apache/hadoop/hbase/TestTimestampType.java  |   1 -
 .../master/procedure/RSProcedureDispatcher.java |  10 +-
 .../hadoop/hbase/regionserver/HRegion.java      |   5 +-
 .../hbase/regionserver/HRegionServer.java       |  14 +-
 .../hbase/regionserver/RSRpcServices.java       |  10 +-
 .../hadoop/hbase/regionserver/StoreScanner.java |   4 +-
 .../hadoop/hbase/TestClockWithCluster.java      |  22 +-
 .../coprocessor/TestIncrementTimeRange.java     |   4 +-
 .../hadoop/hbase/mapreduce/TestCopyTable.java   |   2 +-
 .../hadoop/hbase/master/MockRegionServer.java   |   4 +-
 .../regionserver/TestCompactingMemStore.java    |   6 +-
 .../hbase/regionserver/TestDefaultMemStore.java |   9 +-
 .../hadoop/hbase/regionserver/TestHRegion.java  |  13 +-
 .../regionserver/TestHRegionReplayEvents.java   |   4 +-
 .../regionserver/TestRegionSplitPolicy.java     |   6 +-
 .../hbase/regionserver/TestStoreScanner.java    |  17 +-
 .../hbase/regionserver/TestWALLockup.java       |   4 +-
 .../regionserver/wal/AbstractTestWALReplay.java |   6 +-
 .../access/TestCellACLWithMultipleVersions.java |  11 +-
 .../hbase/util/TestCoprocessorScanPolicy.java   |   2 +-
 31 files changed, 412 insertions(+), 367 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/82a9cec5/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableDescriptorBuilder.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableDescriptorBuilder.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableDescriptorBuilder.java
index d40ce2b..8b3a13d 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableDescriptorBuilder.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableDescriptorBuilder.java
@@ -165,9 +165,9 @@ public class TableDescriptorBuilder {
   public static final ClockType DEFAULT_CLOCK_TYPE = ClockType.SYSTEM;
 
   /**
-   * Default clock type for HTD is HLC
+   * Default clock type for HTD is the hybrid logical clock
    */
-  public static final ClockType DEFAULT_META_CLOCK_TYPE = ClockType.HLC;
+  public static final ClockType DEFAULT_META_CLOCK_TYPE = ClockType.HYBRID_LOGICAL;
 
   @InterfaceAudience.Private
   public static final String PRIORITY = "PRIORITY";

http://git-wip-us.apache.org/repos/asf/hbase/blob/82a9cec5/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
index 3fad23e..a2ed93b 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
@@ -3325,7 +3325,7 @@ public final class ProtobufUtil {
     case SYSTEM_MONOTONIC:
       return ClockType.SYSTEM_MONOTONIC;
     case HLC:
-      return ClockType.HLC;
+      return ClockType.HYBRID_LOGICAL;
     default:
       throw new IllegalArgumentException("Unknown clock type: " + proto);
     }
@@ -3343,7 +3343,7 @@ public final class ProtobufUtil {
       return NodeTime.ClockType.SYSTEM;
     case SYSTEM_MONOTONIC:
       return NodeTime.ClockType.SYSTEM_MONOTONIC;
-    case HLC:
+    case HYBRID_LOGICAL:
       return NodeTime.ClockType.HLC;
     default:
       throw new IllegalArgumentException("Unknown clock type: " + clockType);

http://git-wip-us.apache.org/repos/asf/hbase/blob/82a9cec5/hbase-common/src/main/java/org/apache/hadoop/hbase/Clock.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/Clock.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/Clock.java
index 6a0374e..8387a40 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/Clock.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/Clock.java
@@ -17,14 +17,9 @@
 package org.apache.hadoop.hbase;
 
 import com.google.common.annotations.VisibleForTesting;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 
 import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicLong;
-
-import static org.apache.hadoop.hbase.util.AtomicUtils.updateMax;
 
 /**
  * A clock is an implementation of an algorithm to get timestamps corresponding to one of the
@@ -54,7 +49,6 @@ import static org.apache.hadoop.hbase.util.AtomicUtils.updateMax;
  */
 
 @InterfaceAudience.Private
-@InterfaceStability.Evolving
 public interface Clock {
   long DEFAULT_MAX_CLOCK_SKEW_IN_MS = 30000;
 
@@ -95,241 +89,6 @@ public interface Clock {
    */
   TimeUnit getTimeUnit();
 
-  /**
-   * Indicates that Physical Time or Logical Time component has overflowed. This extends
-   * RuntimeException.
-   */
-  @SuppressWarnings("serial")
-  class ClockException extends RuntimeException {
-    public ClockException(String msg) {
-      super(msg);
-    }
-  }
-
-  Clock SYSTEM_CLOCK = new System();
-
-  //////////////////////////////////////////////////////////////////
-  // Implementation of clocks
-  //////////////////////////////////////////////////////////////////
-
-  /**
-   * System clock is an implementation of clock which doesn't give any monotonic guarantees.
-   * Since it directly represents system's actual clock which cannot be changed, update() function
-   * is no-op.
-   */
-  class System implements Clock {
-    @Override
-    public long now() {
-      return EnvironmentEdgeManager.currentTime();
-    }
-
-    @Override
-    public long update(long timestamp) {
-      return EnvironmentEdgeManager.currentTime();
-    }
-
-    @Override
-    public boolean isMonotonic() {
-      return false;
-    }
-
-    @Override
-    public TimeUnit getTimeUnit() {
-      return TimeUnit.MILLISECONDS;
-    }
-
-    @Override
-    public TimestampType getTimestampType() {
-      return TimestampType.PHYSICAL;
-    }
-
-    @Override
-    public ClockType getClockType() {
-      return ClockType.SYSTEM;
-    }
-  }
-
-  /**
-   * System clock is an implementation of clock which guarantees monotonically non-decreasing
-   * timestamps.
-   */
-  class SystemMonotonic implements Clock {
-    private final long maxClockSkewInMs;
-    private final Clock systemClock;
-    private final AtomicLong physicalTime = new AtomicLong();
-
-    public SystemMonotonic(long maxClockSkewInMs) {
-      this(SYSTEM_CLOCK, maxClockSkewInMs);
-    }
-
-    @VisibleForTesting
-    public SystemMonotonic() {
-      this(DEFAULT_MAX_CLOCK_SKEW_IN_MS);
-    }
-
-    @VisibleForTesting
-    public SystemMonotonic(Clock systemClock) {
-      this(systemClock, DEFAULT_MAX_CLOCK_SKEW_IN_MS);
-    }
-
-    @VisibleForTesting
-    public SystemMonotonic(Clock systemClock, long maxClockSkewInMs) {
-      this.systemClock = systemClock;
-      this.maxClockSkewInMs = maxClockSkewInMs > 0 ?
-          maxClockSkewInMs : DEFAULT_MAX_CLOCK_SKEW_IN_MS;
-    }
-
-    @Override
-    public long now() {
-      updateMax(physicalTime, systemClock.now());
-      return physicalTime.get();
-    }
-
-    /**
-     * @throws ClockException If timestamp exceeds max clock skew allowed.
-     */
-    public long update(long targetTimestamp) throws ClockException {
-      final long systemTime = systemClock.now();
-      if (maxClockSkewInMs > 0 && (targetTimestamp - systemTime) > maxClockSkewInMs) {
-        throw new ClockException(
-            "Received event with timestamp:" + getTimestampType().toString(targetTimestamp)
-                + " which is greater than allowed clock skew ");
-      }
-      final long oldPhysicalTime = systemTime > targetTimestamp ? systemTime : targetTimestamp;
-      updateMax(physicalTime, oldPhysicalTime);
-      return physicalTime.get();
-    }
-
-    @Override
-    public boolean isMonotonic() {
-      return true;
-    }
-
-    @Override
-    public TimeUnit getTimeUnit() {
-      return systemClock.getTimeUnit();
-    }
-
-    @Override
-    public TimestampType getTimestampType() {
-      return TimestampType.PHYSICAL;
-    }
-
-    @Override
-    public ClockType getClockType() {
-      return ClockType.SYSTEM_MONOTONIC;
-    }
-  }
-
-  /**
-   * HLC clock implementation.
-   * Monotonicity guarantee of physical component of time comes from {@link SystemMonotonic} clock.
-   */
-  class HLC implements Clock {
-    private static final TimestampType TIMESTAMP_TYPE = TimestampType.HYBRID;
-    private static final long MAX_PHYSICAL_TIME = TIMESTAMP_TYPE.getMaxPhysicalTime();
-    private static final long MAX_LOGICAL_TIME = TIMESTAMP_TYPE.getMaxLogicalTime();
-    private final Clock systemMonotonicClock;
-    private long currentPhysicalTime = 0;
-    private long currentLogicalTime = 0;
-
-    public HLC(long maxClockSkewInMs) {
-      this(new SystemMonotonic(maxClockSkewInMs));
-    }
-
-    @VisibleForTesting
-    public HLC() {
-      this(DEFAULT_MAX_CLOCK_SKEW_IN_MS);
-    }
-
-    /**
-     * @param systemMonotonicClock Clock to get physical component of time. Should be monotonic
-     *                             clock.
-     */
-    @VisibleForTesting
-    public HLC(Clock systemMonotonicClock) {
-      assert(systemMonotonicClock.isMonotonic());
-      this.systemMonotonicClock = systemMonotonicClock;
-    }
-
-    @Override
-    public synchronized long now() throws ClockException {
-      final long newSystemTime = systemMonotonicClock.now();
-      if (newSystemTime <= currentPhysicalTime) {
-        currentLogicalTime++;
-      } else if (newSystemTime > currentPhysicalTime) {
-        currentLogicalTime = 0;
-        currentPhysicalTime = newSystemTime;
-      }
-      checkPhysicalTimeOverflow(newSystemTime, MAX_PHYSICAL_TIME);
-      checkLogicalTimeOverflow(currentLogicalTime, MAX_LOGICAL_TIME);
-
-      return toTimestamp();
-    }
-
-    /**
-     * Updates {@link HLC} with the given time received from elsewhere (possibly some other node).
-     *
-     * @param targetTime time from the external message.
-     * @return a hybrid timestamp of HLC that is strict greater than given {@code targetTime} and
-     * previously returned times.
-     * @throws ClockException If timestamp exceeds max clock skew allowed.
-     */
-    @Override
-    public synchronized long update(long targetTime)
-        throws ClockException {
-      final long targetPhysicalTime = TIMESTAMP_TYPE.getPhysicalTime(targetTime);
-      final long targetLogicalTime = TIMESTAMP_TYPE.getLogicalTime(targetTime);
-      final long oldPhysicalTime = currentPhysicalTime;
-      currentPhysicalTime = systemMonotonicClock.update(targetPhysicalTime);
-
-      checkPhysicalTimeOverflow(currentPhysicalTime, MAX_PHYSICAL_TIME);
-
-      if (currentPhysicalTime == targetPhysicalTime && currentPhysicalTime == oldPhysicalTime) {
-        currentLogicalTime = Math.max(currentLogicalTime, targetLogicalTime) + 1;
-      } else if (currentPhysicalTime == targetPhysicalTime) {
-        currentLogicalTime = targetLogicalTime + 1;
-      } else if (currentPhysicalTime == oldPhysicalTime) {
-        currentLogicalTime++;
-      } else {
-        currentLogicalTime = 0;
-      }
-
-      checkLogicalTimeOverflow(currentLogicalTime, MAX_LOGICAL_TIME);
-      return toTimestamp();
-    }
-
-    @Override
-    public boolean isMonotonic() {
-      return true;
-    }
-
-    public TimeUnit getTimeUnit() {
-      return systemMonotonicClock.getTimeUnit();
-    }
-
-    private long toTimestamp() {
-      return TIMESTAMP_TYPE.toTimestamp(TimeUnit.MILLISECONDS, currentPhysicalTime,
-          currentLogicalTime);
-    }
-
-    @VisibleForTesting
-    synchronized long getLogicalTime() { return currentLogicalTime; }
-
-    @VisibleForTesting
-    synchronized long getPhysicalTime() { return currentPhysicalTime; }
-
-    @Override
-    public TimestampType getTimestampType() {
-      return TIMESTAMP_TYPE;
-    }
-
-    @Override
-    public ClockType getClockType() {
-      return ClockType.HLC;
-    }
-  }
-
   //////////////////////////////////////////////////////////////////
   // Utility functions
   //////////////////////////////////////////////////////////////////
@@ -337,30 +96,12 @@ public interface Clock {
   // Only for testing.
   @VisibleForTesting
   static Clock getDummyClockOfGivenClockType(ClockType clockType) {
-    if (clockType == ClockType.HLC) {
-      return new Clock.HLC();
+    if (clockType == ClockType.HYBRID_LOGICAL) {
+      return new HybridLogicalClock();
     } else if (clockType == ClockType.SYSTEM_MONOTONIC) {
-      return new Clock.SystemMonotonic();
+      return new SystemMonotonicClock();
     } else {
-      return new Clock.System();
-    }
-  }
-
-  static void checkLogicalTimeOverflow(long logicalTime, long maxLogicalTime) {
-    if (logicalTime >= maxLogicalTime) {
-      // highly unlikely to happen, when it happens, we throw exception for the above layer to
-      // handle it the way they wish to.
-      throw new Clock.ClockException(
-          "Logical Time Overflowed: " + logicalTime + "max " + "logical time: " + maxLogicalTime);
+      return new SystemClock();
     }
   }
-
-  static void checkPhysicalTimeOverflow(long physicalTime, long maxPhysicalTime) {
-    if (physicalTime >= maxPhysicalTime) {
-      // Extremely unlikely to happen, if this happens upper layers may have to kill the server.
-      throw new Clock.ClockException(
-          "Physical Time overflowed: " + physicalTime + " and max physical time:" + maxPhysicalTime);
-    }
-  }
-
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/82a9cec5/hbase-common/src/main/java/org/apache/hadoop/hbase/ClockException.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/ClockException.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/ClockException.java
new file mode 100644
index 0000000..0de1b11
--- /dev/null
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/ClockException.java
@@ -0,0 +1,33 @@
+/**
+ * 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 org.apache.hadoop.hbase.classification.InterfaceAudience;
+
+/**
+ * Indicates that Physical Time or Logical Time component has overflowed. This extends
+ * RuntimeException.
+ */
+@InterfaceAudience.Private
+@SuppressWarnings("serial")
+public class ClockException extends RuntimeException {
+  public ClockException(String msg) {
+    super(msg);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/82a9cec5/hbase-common/src/main/java/org/apache/hadoop/hbase/ClockType.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/ClockType.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/ClockType.java
index aee8e43..36f7224 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/ClockType.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/ClockType.java
@@ -29,7 +29,7 @@ public enum ClockType {
     public TimestampType timestampType() {
       return TimestampType.PHYSICAL;
     }
-  }, HLC {
+  }, HYBRID_LOGICAL {
     public TimestampType timestampType() {
       return TimestampType.HYBRID;
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/82a9cec5/hbase-common/src/main/java/org/apache/hadoop/hbase/HybridLogicalClock.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/HybridLogicalClock.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/HybridLogicalClock.java
new file mode 100644
index 0000000..f61c13d
--- /dev/null
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/HybridLogicalClock.java
@@ -0,0 +1,134 @@
+package org.apache.hadoop.hbase;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+
+import java.util.concurrent.TimeUnit;
+
+/**
+ * Hybrid logical clock implementation.
+ * Monotonicity guarantee of physical component of time comes from {@link SystemMonotonicClock}.
+ */
+@InterfaceAudience.Private
+public class HybridLogicalClock implements Clock {
+  private static final TimestampType TIMESTAMP_TYPE = TimestampType.HYBRID;
+  private static final long MAX_PHYSICAL_TIME = TIMESTAMP_TYPE.getMaxPhysicalTime();
+  private static final long MAX_LOGICAL_TIME = TIMESTAMP_TYPE.getMaxLogicalTime();
+  private final Clock systemMonotonicClock;
+  private long currentPhysicalTime = 0;
+  private long currentLogicalTime = 0;
+
+  public HybridLogicalClock(long maxClockSkewInMs) {
+    this(new SystemMonotonicClock(maxClockSkewInMs));
+  }
+
+  @VisibleForTesting
+  public HybridLogicalClock() {
+    this(DEFAULT_MAX_CLOCK_SKEW_IN_MS);
+  }
+
+  /**
+   * @param systemMonotonicClock Clock to get physical component of time. Should be monotonic
+   *                             clock.
+   */
+  @VisibleForTesting
+  public HybridLogicalClock(Clock systemMonotonicClock) {
+    assert(systemMonotonicClock.isMonotonic());
+    this.systemMonotonicClock = systemMonotonicClock;
+  }
+
+  @Override
+  public synchronized long now() throws ClockException {
+    final long newSystemTime = systemMonotonicClock.now();
+    if (newSystemTime <= currentPhysicalTime) {
+      currentLogicalTime++;
+    } else if (newSystemTime > currentPhysicalTime) {
+      currentLogicalTime = 0;
+      currentPhysicalTime = newSystemTime;
+    }
+    checkPhysicalTimeOverflow(newSystemTime, MAX_PHYSICAL_TIME);
+    checkLogicalTimeOverflow(currentLogicalTime, MAX_LOGICAL_TIME);
+
+    return toTimestamp();
+  }
+
+  /**
+   * Updates {@link org.apache.hadoop.hbase.HybridLogicalClock} with the given time received from elsewhere (possibly
+   * some other node).
+   *
+   * @param targetTime time from the external message.
+   * @return a hybrid timestamp of HLC that is strict greater than given {@code targetTime} and
+   * previously returned times.
+   * @throws ClockException If timestamp exceeds max clock skew allowed.
+   */
+  @Override
+  public synchronized long update(long targetTime)
+      throws ClockException {
+    final long targetPhysicalTime = TIMESTAMP_TYPE.getPhysicalTime(targetTime);
+    final long targetLogicalTime = TIMESTAMP_TYPE.getLogicalTime(targetTime);
+    final long oldPhysicalTime = currentPhysicalTime;
+    currentPhysicalTime = systemMonotonicClock.update(targetPhysicalTime);
+
+    checkPhysicalTimeOverflow(currentPhysicalTime, MAX_PHYSICAL_TIME);
+
+    if (currentPhysicalTime == targetPhysicalTime && currentPhysicalTime == oldPhysicalTime) {
+      currentLogicalTime = Math.max(currentLogicalTime, targetLogicalTime) + 1;
+    } else if (currentPhysicalTime == targetPhysicalTime) {
+      currentLogicalTime = targetLogicalTime + 1;
+    } else if (currentPhysicalTime == oldPhysicalTime) {
+      currentLogicalTime++;
+    } else {
+      currentLogicalTime = 0;
+    }
+
+    checkLogicalTimeOverflow(currentLogicalTime, MAX_LOGICAL_TIME);
+    return toTimestamp();
+  }
+
+  @Override
+  public boolean isMonotonic() {
+    return true;
+  }
+
+  public TimeUnit getTimeUnit() {
+    return systemMonotonicClock.getTimeUnit();
+  }
+
+  private long toTimestamp() {
+    return TIMESTAMP_TYPE.toTimestamp(TimeUnit.MILLISECONDS, currentPhysicalTime,
+        currentLogicalTime);
+  }
+
+  @VisibleForTesting
+  synchronized long getLogicalTime() { return currentLogicalTime; }
+
+  @VisibleForTesting
+  synchronized long getPhysicalTime() { return currentPhysicalTime; }
+
+  @Override
+  public TimestampType getTimestampType() {
+    return TIMESTAMP_TYPE;
+  }
+
+  @Override
+  public ClockType getClockType() {
+    return ClockType.HYBRID_LOGICAL;
+  }
+
+  static void checkLogicalTimeOverflow(long logicalTime, long maxLogicalTime) {
+    if (logicalTime >= maxLogicalTime) {
+      // highly unlikely to happen, when it happens, we throw exception for the above layer to
+      // handle it the way they wish to.
+      throw new ClockException(
+          "Logical Time Overflowed: " + logicalTime + "max " + "logical time: " + maxLogicalTime);
+    }
+  }
+
+  static void checkPhysicalTimeOverflow(long physicalTime, long maxPhysicalTime) {
+    if (physicalTime >= maxPhysicalTime) {
+      // Extremely unlikely to happen, if this happens upper layers may have to kill the server.
+      throw new ClockException(
+          "Physical Time overflowed: " + physicalTime + " and max physical time:" + maxPhysicalTime);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/82a9cec5/hbase-common/src/main/java/org/apache/hadoop/hbase/SettableTimestamp.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/SettableTimestamp.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/SettableTimestamp.java
index 41d3a0a..03733d0 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/SettableTimestamp.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/SettableTimestamp.java
@@ -26,7 +26,7 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience;
  * Note : Server side Cell implementations in write path must implement this.
  * @deprecated as of 2.0 and will be removed in 3.0. Use {@link ExtendedCell} instead
  */
-@Deprecated // Co Processors SHOULD NOT use this if the clock type of the tables is HLC
+@Deprecated // Co Processors SHOULD NOT use this if the clock type of the tables is hybrid logical
 @InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.COPROC)
 public interface SettableTimestamp {
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/82a9cec5/hbase-common/src/main/java/org/apache/hadoop/hbase/SystemClock.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/SystemClock.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/SystemClock.java
new file mode 100644
index 0000000..b44cabe
--- /dev/null
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/SystemClock.java
@@ -0,0 +1,44 @@
+package org.apache.hadoop.hbase;
+
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+
+import java.util.concurrent.TimeUnit;
+
+/**
+ * System clock is an implementation of clock which doesn't give any monotonic guarantees.
+ * Since it directly represents system's actual clock which cannot be changed, update() function
+ * is no-op.
+ */
+@InterfaceAudience.Private
+public class SystemClock implements Clock {
+  @Override
+  public long now() {
+    return EnvironmentEdgeManager.currentTime();
+  }
+
+  @Override
+  public long update(long timestamp) {
+    return EnvironmentEdgeManager.currentTime();
+  }
+
+  @Override
+  public boolean isMonotonic() {
+    return false;
+  }
+
+  @Override
+  public TimeUnit getTimeUnit() {
+    return TimeUnit.MILLISECONDS;
+  }
+
+  @Override
+  public TimestampType getTimestampType() {
+    return TimestampType.PHYSICAL;
+  }
+
+  @Override
+  public ClockType getClockType() {
+    return ClockType.SYSTEM;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/82a9cec5/hbase-common/src/main/java/org/apache/hadoop/hbase/SystemMonotonicClock.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/SystemMonotonicClock.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/SystemMonotonicClock.java
new file mode 100644
index 0000000..fb79f55
--- /dev/null
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/SystemMonotonicClock.java
@@ -0,0 +1,83 @@
+package org.apache.hadoop.hbase;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+
+import static org.apache.hadoop.hbase.util.AtomicUtils.updateMax;
+
+/**
+ * System monotonic clock is an implementation of clock which guarantees monotonically
+ * non-decreasing timestamps.
+ */
+@InterfaceAudience.Private
+public class SystemMonotonicClock implements Clock {
+  static final Clock SYSTEM_CLOCK = new SystemClock();
+  private final long maxClockSkewInMs;
+  private final Clock systemClock;
+  private final AtomicLong physicalTime = new AtomicLong();
+
+  public SystemMonotonicClock(long maxClockSkewInMs) {
+    this(SYSTEM_CLOCK, maxClockSkewInMs);
+  }
+
+  @VisibleForTesting
+  public SystemMonotonicClock() {
+    this(DEFAULT_MAX_CLOCK_SKEW_IN_MS);
+  }
+
+  @VisibleForTesting
+  public SystemMonotonicClock(Clock systemClock) {
+    this(systemClock, DEFAULT_MAX_CLOCK_SKEW_IN_MS);
+  }
+
+  @VisibleForTesting
+  public SystemMonotonicClock(Clock systemClock, long maxClockSkewInMs) {
+    this.systemClock = systemClock;
+    this.maxClockSkewInMs = maxClockSkewInMs > 0 ?
+        maxClockSkewInMs : DEFAULT_MAX_CLOCK_SKEW_IN_MS;
+  }
+
+  @Override
+  public long now() {
+    updateMax(physicalTime, systemClock.now());
+    return physicalTime.get();
+  }
+
+  /**
+   * @throws ClockException If timestamp exceeds max clock skew allowed.
+   */
+  public long update(long targetTimestamp) throws ClockException {
+    final long systemTime = systemClock.now();
+    if (maxClockSkewInMs > 0 && (targetTimestamp - systemTime) > maxClockSkewInMs) {
+      throw new ClockException(
+          "Received event with timestamp:" + getTimestampType().toString(targetTimestamp)
+              + " which is greater than allowed clock skew ");
+    }
+    final long oldPhysicalTime = systemTime > targetTimestamp ? systemTime : targetTimestamp;
+    updateMax(physicalTime, oldPhysicalTime);
+    return physicalTime.get();
+  }
+
+  @Override
+  public boolean isMonotonic() {
+    return true;
+  }
+
+  @Override
+  public TimeUnit getTimeUnit() {
+    return systemClock.getTimeUnit();
+  }
+
+  @Override
+  public TimestampType getTimestampType() {
+    return TimestampType.PHYSICAL;
+  }
+
+  @Override
+  public ClockType getClockType() {
+    return ClockType.SYSTEM_MONOTONIC;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/82a9cec5/hbase-common/src/main/java/org/apache/hadoop/hbase/TimestampType.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/TimestampType.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/TimestampType.java
index effbde2..26d8f6f 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/TimestampType.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/TimestampType.java
@@ -43,7 +43,6 @@ import java.util.concurrent.TimeUnit;
  * </p>
  */
 @InterfaceAudience.Private
-@InterfaceStability.Evolving
 public enum TimestampType {
   /**
    * Hybrid is a Timestamp type used to encode both physical time and logical time components

http://git-wip-us.apache.org/repos/asf/hbase/blob/82a9cec5/hbase-common/src/test/java/org/apache/hadoop/hbase/TestClock.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/TestClock.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/TestClock.java
index f9ad8e4..866b8f4 100644
--- a/hbase-common/src/test/java/org/apache/hadoop/hbase/TestClock.java
+++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/TestClock.java
@@ -92,7 +92,7 @@ public class TestClock {
   @Test
   public void testSystemMonotonicNow() {
     MonotonicityCheckerClock systemMonotonic =
-        new MonotonicityCheckerClock(new Clock.SystemMonotonic(MOCK_CLOCK), false);
+        new MonotonicityCheckerClock(new SystemMonotonicClock(MOCK_CLOCK), false);
 
     // case 1: Set time and assert
     when(MOCK_CLOCK.now()).thenReturn(100L);
@@ -111,13 +111,13 @@ public class TestClock {
 
   /**
    * Tests that
-   * - Progressing mock clock progresses SystemMonotonic clock.
+   * - Progressing mock clock progresses system monotonic clock.
    * - Skews in the clock are correctly updated/not changed on call to update(), depending on
    * target time and clock's own time ( = system time + current skew).
    */
   @Test
   public void testSystemMonotonicUpdate() {
-    Clock.SystemMonotonic systemMonotonicClock = new Clock.SystemMonotonic(MOCK_CLOCK);
+    SystemMonotonicClock systemMonotonicClock = new SystemMonotonicClock(MOCK_CLOCK);
     MonotonicityCheckerClock systemMonotonic =
         new MonotonicityCheckerClock(systemMonotonicClock, false);
 
@@ -155,20 +155,20 @@ public class TestClock {
   }
 
   @Test
-  public void testSystemMonotonicUpdateMaxClockSkew() throws Clock.ClockException {
+  public void testSystemMonotonicUpdateMaxClockSkew() throws ClockException {
     final long time = 100L;
-    Clock.SystemMonotonic systemMonotonic =
-        new Clock.SystemMonotonic(MOCK_CLOCK, MAX_CLOCK_SKEW_IN_MS);
+    SystemMonotonicClock systemMonotonicClock =
+        new SystemMonotonicClock(MOCK_CLOCK, MAX_CLOCK_SKEW_IN_MS);
 
     // Set Current Time.
     when(MOCK_CLOCK.now()).thenReturn(time);
-    systemMonotonic.now();
+    systemMonotonicClock.now();
 
     // Shouldn't throw ClockException
-    systemMonotonic.update(time + MAX_CLOCK_SKEW_IN_MS - 1);
+    systemMonotonicClock.update(time + MAX_CLOCK_SKEW_IN_MS - 1);
 
-    exception.expect(Clock.ClockException.class);
-    systemMonotonic.update(time + MAX_CLOCK_SKEW_IN_MS + 1);
+    exception.expect(ClockException.class);
+    systemMonotonicClock.update(time + MAX_CLOCK_SKEW_IN_MS + 1);
   }
 
   // All Hybrid Logical Clock Tests
@@ -179,9 +179,9 @@ public class TestClock {
   }
 
   @Test
-  public void testHLCNow() throws Clock.ClockException {
+  public void testHLCNow() throws ClockException {
     MonotonicityCheckerClock hybridLogicalClock = new MonotonicityCheckerClock(
-        new Clock.HLC(new Clock.SystemMonotonic(MOCK_CLOCK)),
+        new HybridLogicalClock(new SystemMonotonicClock(MOCK_CLOCK)),
         true);  // true for strict monotonicity
 
     // case 1: Test if it returns correct time based on current physical time.
@@ -210,23 +210,24 @@ public class TestClock {
 
   @Test
   public void testHLCNowLogicalTimeOverFlow() {
-    Clock.HLC hybridLogicalClock = new Clock.HLC(new Clock.SystemMonotonic(MOCK_CLOCK));
+    HybridLogicalClock hybridLogicalClock =
+        new HybridLogicalClock(new SystemMonotonicClock(MOCK_CLOCK));
 
     when(MOCK_CLOCK.now()).thenReturn(100L);
     hybridLogicalClock.now();  // current time (100, 0)
     for (int i = 0; i < TimestampType.HYBRID.getMaxLogicalTime() - 1; i++) {
       hybridLogicalClock.now();
     }
-    exception.expect(Clock.ClockException.class);
+    exception.expect(ClockException.class);
     hybridLogicalClock.now();
   }
 
-  // No need to check skews in this test, since they are member of SystemMonotonic and not HLC.
+  // No need to check skews in this test, since they are member of system monotonic and not HLC
   @Test
-  public void testHLCUpdate() throws Clock.ClockException {
+  public void testHLCUpdate() throws ClockException {
     long messageTimestamp;
     MonotonicityCheckerClock hybridLogicalClock = new MonotonicityCheckerClock(
-        new Clock.HLC(new Clock.SystemMonotonic(MOCK_CLOCK)),
+        new HybridLogicalClock(new SystemMonotonicClock(MOCK_CLOCK)),
         true);  // true for strictly increasing check
 
     // Set Current Time.
@@ -267,9 +268,10 @@ public class TestClock {
   }
 
   @Test
-  public void testHLCUpdateLogicalTimeOverFlow() throws Clock.ClockException {
+  public void testHLCUpdateLogicalTimeOverFlow() throws ClockException {
     long messageTimestamp;
-    Clock.HLC hybridLogicalClock = new Clock.HLC(new Clock.SystemMonotonic(MOCK_CLOCK));
+    HybridLogicalClock hybridLogicalClock =
+        new HybridLogicalClock(new SystemMonotonicClock(MOCK_CLOCK));
 
     // Set Current Time.
     when(MOCK_CLOCK.now()).thenReturn(100L);
@@ -277,16 +279,17 @@ public class TestClock {
 
     messageTimestamp = TimestampType.HYBRID.toTimestamp(TimeUnit.MILLISECONDS, 100,
         TimestampType.HYBRID.getMaxLogicalTime());
-    exception.expect(Clock.ClockException.class);
+    exception.expect(ClockException.class);
     hybridLogicalClock.update(messageTimestamp);
   }
 
   @Test
-  public void testHLCUpdateMaxClockSkew() throws Clock.ClockException {
+  public void testHLCUpdateMaxClockSkew() throws ClockException {
     final long time = 100;
     long messageTimestamp;
-    Clock.HLC hybridLogicalClock = new Clock.HLC(
-        new Clock.SystemMonotonic(MOCK_CLOCK, MAX_CLOCK_SKEW_IN_MS));
+    HybridLogicalClock
+        hybridLogicalClock = new HybridLogicalClock(
+        new SystemMonotonicClock(MOCK_CLOCK, MAX_CLOCK_SKEW_IN_MS));
 
     // Set Current Time.
     when(MOCK_CLOCK.now()).thenReturn(time);
@@ -298,7 +301,7 @@ public class TestClock {
 
     messageTimestamp = TimestampType.HYBRID.toTimestamp(TimeUnit.MILLISECONDS,
         time + MAX_CLOCK_SKEW_IN_MS + 1, 0);
-    exception.expect(Clock.ClockException.class);
+    exception.expect(ClockException.class);
     hybridLogicalClock.update(messageTimestamp);
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/82a9cec5/hbase-common/src/test/java/org/apache/hadoop/hbase/TestTimestampType.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/TestTimestampType.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/TestTimestampType.java
index 8b7d83f..2f83d9a 100644
--- a/hbase-common/src/test/java/org/apache/hadoop/hbase/TestTimestampType.java
+++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/TestTimestampType.java
@@ -58,7 +58,6 @@ public class TestTimestampType {
     }
   }
 
-  /* Tests for HLC Clock */
   @Test
   public void testHybridMaxValues() {
     // assert 44-bit Physical Time with signed comparison (actual 43 bits)

http://git-wip-us.apache.org/repos/asf/hbase/blob/82a9cec5/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/RSProcedureDispatcher.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/RSProcedureDispatcher.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/RSProcedureDispatcher.java
index 8d016b7..c40ecb6 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/RSProcedureDispatcher.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/RSProcedureDispatcher.java
@@ -33,10 +33,8 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.ClockType;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.ipc.RemoteException;
-import org.apache.hadoop.hbase.Clock;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.ServerName;
-import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.DoNotRetryIOException;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NodeTime;
 import org.apache.hadoop.hbase.ipc.ServerNotRunningYetException;
@@ -348,8 +346,8 @@ public class RSProcedureDispatcher
         .setClockType(ProtobufUtil.toClockType(ClockType.SYSTEM_MONOTONIC))
         .setTimestamp(env.getMasterServices().getClock(ClockType.SYSTEM_MONOTONIC).now());
     builder.addNodeTimesBuilder()
-        .setClockType(ProtobufUtil.toClockType(ClockType.HLC))
-        .setTimestamp(env.getMasterServices().getClock(ClockType.HLC).now());
+        .setClockType(ProtobufUtil.toClockType(ClockType.HYBRID_LOGICAL))
+        .setTimestamp(env.getMasterServices().getClock(ClockType.HYBRID_LOGICAL).now());
 
     for (RegionOpenOperation op: operations) {
       builder.addOpenInfo(op.buildRegionOpenInfoRequest(env));
@@ -578,8 +576,8 @@ public class RSProcedureDispatcher
       List<Pair<ClockType, Long>> nodeTimes = new ArrayList<>();
       nodeTimes.add(new Pair<>(ClockType.SYSTEM_MONOTONIC,
           env.getMasterServices().getClock(ClockType.SYSTEM_MONOTONIC).now()));
-      nodeTimes.add(new Pair<>(ClockType.HLC,
-          env.getMasterServices().getClock(ClockType.HLC).now()));
+      nodeTimes.add(new Pair<>(ClockType.HYBRID_LOGICAL,
+          env.getMasterServices().getClock(ClockType.HYBRID_LOGICAL).now()));
       return ProtobufUtil.buildCloseRegionRequest(serverName,
         getRegionInfo().getRegionName(), getDestinationServer(), nodeTimes);
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/82a9cec5/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
index 56f8bb6..bf48c63 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
@@ -98,6 +98,7 @@ import org.apache.hadoop.hbase.KeyValueUtil;
 import org.apache.hadoop.hbase.NamespaceDescriptor;
 import org.apache.hadoop.hbase.NotServingRegionException;
 import org.apache.hadoop.hbase.RegionTooBusyException;
+import org.apache.hadoop.hbase.SystemClock;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.Tag;
 import org.apache.hadoop.hbase.TagUtil;
@@ -812,7 +813,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
         recoveringRegions.put(encodedName, this);
       }
     } else {
-      this.clock = new Clock.System();
+      this.clock = new SystemClock();
       this.metricsRegionWrapper = null;
       this.metricsRegion = null;
     }
@@ -2846,7 +2847,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
 
   private RegionScanner getScanner(Scan scan, List<KeyValueScanner> additionalScanners,
       long nonceGroup, long nonce) throws IOException {
-    if (getClock().getClockType() == ClockType.HLC) {
+    if (getClock().getClockType() == ClockType.HYBRID_LOGICAL) {
       mapTimeRangesWithRespectToClock(scan);
     }
     startRegionOperation(Operation.SCAN);

http://git-wip-us.apache.org/repos/asf/hbase/blob/82a9cec5/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
index c46c634..bf6dc3a 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
@@ -78,15 +78,17 @@ import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.Clock;
 import org.apache.hadoop.hbase.ClockType;
 import org.apache.hadoop.hbase.HealthCheckChore;
+import org.apache.hadoop.hbase.HybridLogicalClock;
 import org.apache.hadoop.hbase.MetaTableAccessor;
 import org.apache.hadoop.hbase.NotServingRegionException;
 import org.apache.hadoop.hbase.PleaseHoldException;
 import org.apache.hadoop.hbase.ScheduledChore;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.Stoppable;
+import org.apache.hadoop.hbase.SystemClock;
+import org.apache.hadoop.hbase.SystemMonotonicClock;
 import org.apache.hadoop.hbase.TableDescriptors;
 import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.TimestampType;
 import org.apache.hadoop.hbase.YouAreDeadException;
 import org.apache.hadoop.hbase.ZNodeClearer;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
@@ -600,9 +602,9 @@ public class HRegionServer extends HasThread implements
 
     final long maxClockSkew =
         conf.getLong("hbase.max.clock.skew.in.ms", Clock.DEFAULT_MAX_CLOCK_SKEW_IN_MS);
-    this.hybridLogicalClock = new Clock.HLC(maxClockSkew);
-    this.systemMonotonicClock = new Clock.SystemMonotonic(maxClockSkew);
-    this.systemClock = new Clock.System();
+    this.hybridLogicalClock = new HybridLogicalClock(maxClockSkew);
+    this.systemMonotonicClock = new SystemMonotonicClock(maxClockSkew);
+    this.systemClock = new SystemClock();
 
     rpcServices = createRpcServices();
     this.startcode = System.currentTimeMillis();
@@ -2092,7 +2094,7 @@ public class HRegionServer extends HasThread implements
   @Override
   public Clock getClock(ClockType clockType) {
     switch (clockType) {
-      case HLC:
+      case HYBRID_LOGICAL:
         return this.hybridLogicalClock;
       case SYSTEM_MONOTONIC:
         return this.systemMonotonicClock;
@@ -2110,7 +2112,7 @@ public class HRegionServer extends HasThread implements
   @VisibleForTesting
   public void setClock(Clock clock) {
     switch (clock.getClockType()) {
-      case HLC:
+      case HYBRID_LOGICAL:
         this.hybridLogicalClock = clock;
         break;
       case SYSTEM_MONOTONIC:

http://git-wip-us.apache.org/repos/asf/hbase/blob/82a9cec5/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
index ab50ad3..87860f3 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
@@ -57,7 +57,6 @@ import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellScannable;
 import org.apache.hadoop.hbase.CellScanner;
 import org.apache.hadoop.hbase.CellUtil;
-import org.apache.hadoop.hbase.Clock;
 import org.apache.hadoop.hbase.ClockType;
 import org.apache.hadoop.hbase.DoNotRetryIOException;
 import org.apache.hadoop.hbase.DroppedSnapshotException;
@@ -202,7 +201,6 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanReques
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.RegionLoad;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameBytesPair;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameInt64Pair;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NodeTime;
@@ -1536,8 +1534,8 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
             .setClockType(ProtobufUtil.toClockType(ClockType.SYSTEM_MONOTONIC))
             .setTimestamp(regionServer.getClock(ClockType.SYSTEM_MONOTONIC).now()).build())
           .addNodeTimes(NodeTime.newBuilder()
-            .setClockType(ProtobufUtil.toClockType(ClockType.HLC))
-            .setTimestamp(regionServer.getClock(ClockType.HLC).now()).build());
+            .setClockType(ProtobufUtil.toClockType(ClockType.HYBRID_LOGICAL))
+            .setTimestamp(regionServer.getClock(ClockType.HYBRID_LOGICAL).now()).build());
       return builder.build();
     } catch (IOException ie) {
       throw new ServiceException(ie);
@@ -2026,8 +2024,8 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
         .setClockType(ProtobufUtil.toClockType(ClockType.SYSTEM_MONOTONIC))
         .setTimestamp(regionServer.getClock(ClockType.SYSTEM_MONOTONIC).now()).build())
       .addNodeTimes(NodeTime.newBuilder()
-        .setClockType(ProtobufUtil.toClockType(ClockType.HLC))
-        .setTimestamp(regionServer.getClock(ClockType.HLC).now()).build());
+        .setClockType(ProtobufUtil.toClockType(ClockType.HYBRID_LOGICAL))
+        .setTimestamp(regionServer.getClock(ClockType.HYBRID_LOGICAL).now()).build());
 
     return builder.build();
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/82a9cec5/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 bb0d1d5..3f453fd 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,7 +31,6 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.*;
 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;
@@ -44,7 +43,6 @@ import org.apache.hadoop.hbase.regionserver.querymatcher.LegacyScanQueryMatcher;
 import org.apache.hadoop.hbase.regionserver.querymatcher.ScanQueryMatcher;
 import org.apache.hadoop.hbase.regionserver.querymatcher.UserScanQueryMatcher;
 import org.apache.hadoop.hbase.util.CollectionUtils;
-import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 
 import com.google.common.annotations.VisibleForTesting;
 
@@ -176,7 +174,7 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
       now = store.getClock().now();
       timestampType = store.getClock().getTimestampType();
     } else {
-      now = new Clock.System().now();
+      now = new SystemClock().now();
       timestampType = TimestampType.PHYSICAL;
     }
     // Convert to milliseconds before subtracting time

http://git-wip-us.apache.org/repos/asf/hbase/blob/82a9cec5/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
index ea46090..0ad3735 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestClockWithCluster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestClockWithCluster.java
@@ -19,7 +19,6 @@
 package org.apache.hadoop.hbase;
 
 import static org.junit.Assert.*;
-import static org.mockito.Matchers.any;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
 
@@ -60,6 +59,7 @@ import org.junit.rules.TestName;
 @Category({MediumTests.class})
 public class TestClockWithCluster {
   private static final Log LOG = LogFactory.getLog(TestClockWithCluster.class);
+  private static final Clock SYSTEM_CLOCK = new SystemClock();
   @Rule
   public TestName name = new TestName();
   private static final HBaseTestingUtility HBTU = new HBaseTestingUtility();
@@ -132,7 +132,7 @@ public class TestClockWithCluster {
     // read , check if the it is same.
     HBTU.verifyNumericRows(table, TEST_FAMILY, 0, 1000, 0);
 
-    // This check will be useful if Clock type were to be system monotonic or HLC.
+    // This check will be useful if Clock type were to be system monotonic or hybrid logical.
     verifyTimestamps(table, TEST_FAMILY, 0, 1000, TimestampType.PHYSICAL);
   }
 
@@ -140,7 +140,7 @@ public class TestClockWithCluster {
   public void testMetaTableClockTypeIsHLC() throws IOException {
     ClockType clockType = admin
       .getTableDescriptor(TableName.META_TABLE_NAME).getClockType();
-    assertEquals(ClockType.HLC, clockType);
+    assertEquals(ClockType.HYBRID_LOGICAL, clockType);
   }
 
   @Test
@@ -161,7 +161,8 @@ public class TestClockWithCluster {
     return cell.getTimestamp();
   }
 
-  private void assertHLCTime(Clock.HLC clock, long expectedPhysicalTime, long expectedLogicalTime) {
+  private void assertHLCTime(HybridLogicalClock clock,
+      long expectedPhysicalTime, long expectedLogicalTime) {
     assertEquals(expectedPhysicalTime, clock.getPhysicalTime());
     assertEquals(expectedLogicalTime, clock.getLogicalTime());
   }
@@ -199,12 +200,13 @@ public class TestClockWithCluster {
     assertNotNull(regionMeta);
 
     // Inject physical clock that always returns same physical time into hybrid logical clock
-    long systemTime = Clock.SYSTEM_CLOCK.now();
+    long systemTime = SYSTEM_CLOCK.now();
     Clock mockSystemClock = mock(Clock.class);
     when(mockSystemClock.now()).thenReturn(systemTime);
     when(mockSystemClock.getTimeUnit()).thenReturn(TimeUnit.MILLISECONDS);
     when(mockSystemClock.isMonotonic()).thenReturn(true);
-    Clock.HLC masterHLC = new Clock.HLC(new Clock.SystemMonotonic(mockSystemClock));
+    HybridLogicalClock masterHLC =
+        new HybridLogicalClock(new SystemMonotonicClock(mockSystemClock));
 
     // The region clock is used for setting timestamps for table mutations and the region server
     // clock is used for updating the clock on region assign/unassign events.
@@ -307,11 +309,12 @@ public class TestClockWithCluster {
     assertNotNull(regionMeta);
 
     // Instantiate two hybrid logical clocks with mocked physical clocks
-    long expectedPhysicalTime = Clock.SYSTEM_CLOCK.now();
+    long expectedPhysicalTime = SYSTEM_CLOCK.now();
     Clock masterMockSystemClock = mock(Clock.class);
     when(masterMockSystemClock.now()).thenReturn(expectedPhysicalTime);
     when(masterMockSystemClock.getTimeUnit()).thenReturn(TimeUnit.MILLISECONDS);
-    Clock.HLC masterHLC= new Clock.HLC(new Clock.SystemMonotonic(masterMockSystemClock));
+    HybridLogicalClock masterHLC =
+        new HybridLogicalClock(new SystemMonotonicClock(masterMockSystemClock));
     master.setClock(masterHLC);
     regionMeta.setClock(masterHLC);
 
@@ -319,7 +322,8 @@ public class TestClockWithCluster {
     when(rsMockSystemClock.now()).thenReturn(expectedPhysicalTime);
     when(rsMockSystemClock.getTimeUnit()).thenReturn(TimeUnit.MILLISECONDS);
     when(rsMockSystemClock.isMonotonic()).thenReturn(true);
-    Clock.HLC rsHLC= new Clock.HLC(new Clock.SystemMonotonic(rsMockSystemClock));
+    HybridLogicalClock rsHLC =
+        new HybridLogicalClock(new SystemMonotonicClock(rsMockSystemClock));
     // We only mock the region server clock here because the region clock does not get used
     // during unassignment and assignment
     rs.setClock(rsHLC);

http://git-wip-us.apache.org/repos/asf/hbase/blob/82a9cec5/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 82e8510..ed30a2d 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
@@ -94,8 +94,8 @@ public class TestIncrementTimeRange {
 
   @Parameters(name = "{0}")
   public static Iterable<Object> data() {
-    return Arrays.asList(new Object[] {ClockType.HLC, ClockType.SYSTEM_MONOTONIC, ClockType
-        .SYSTEM});
+    return Arrays.asList(new Object[] {ClockType.HYBRID_LOGICAL, ClockType.SYSTEM_MONOTONIC,
+        ClockType.SYSTEM});
   }
 
   public TestIncrementTimeRange(ClockType clockType) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/82a9cec5/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 2630869..d66ffc0 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
@@ -195,7 +195,7 @@ public class TestCopyTable {
   public void testRenameFamily() throws Exception {
     testRenameFamily(ClockType.SYSTEM);
     testRenameFamily(ClockType.SYSTEM_MONOTONIC);
-    testRenameFamily(ClockType.HLC);
+    testRenameFamily(ClockType.HYBRID_LOGICAL);
   }
 
   public void testRenameFamily(ClockType clockType) throws Exception {

http://git-wip-us.apache.org/repos/asf/hbase/blob/82a9cec5/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 1f92bb8..0677b72 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
@@ -40,8 +40,8 @@ import org.apache.hadoop.hbase.ChoreService;
 import org.apache.hadoop.hbase.CoordinatedStateManager;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.SystemClock;
 import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.TimestampType;
 import org.apache.hadoop.hbase.ZooKeeperConnectionException;
 import org.apache.hadoop.hbase.client.ClusterConnection;
 import org.apache.hadoop.hbase.client.Result;
@@ -583,7 +583,7 @@ ClientProtos.ClientService.BlockingInterface, RegionServerServices {
 
   @Override
   public Clock getClock(ClockType clockType) {
-    return new Clock.System();
+    return new SystemClock();
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/82a9cec5/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 e5d4326..b7a5168 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
@@ -175,7 +175,7 @@ public class TestCompactingMemStore extends TestDefaultMemStore {
    */
   @Test
   public void testGetNextRowWithHybridLogicalClock() throws Exception {
-    testGetNextRow(new Clock.HLC());
+    testGetNextRow(new HybridLogicalClock());
   }
 
   /** Test getNextRow from memstore using timestamps from a system monotonic clock
@@ -183,7 +183,7 @@ public class TestCompactingMemStore extends TestDefaultMemStore {
    */
   @Test
   public void testGetNextRowWithSystemMonotonicClock() throws Exception {
-    testGetNextRow(new Clock.SystemMonotonic());
+    testGetNextRow(new SystemMonotonicClock());
   }
 
   /** Test getNextRow from memstore using timestamps from a system clock
@@ -191,7 +191,7 @@ public class TestCompactingMemStore extends TestDefaultMemStore {
    */
   @Test
   public void testGetNextRowWithSystemClock() throws Exception {
-    testGetNextRow(new Clock.System());
+    testGetNextRow(new SystemClock());
   }
 
   public void testGetNextRow(Clock clock) throws Exception {

http://git-wip-us.apache.org/repos/asf/hbase/blob/82a9cec5/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 06999f9..f9ad88f 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
@@ -37,10 +37,13 @@ import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.HybridLogicalClock;
 import org.apache.hadoop.hbase.KeepDeletedCells;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.KeyValueTestUtil;
 import org.apache.hadoop.hbase.KeyValueUtil;
+import org.apache.hadoop.hbase.SystemClock;
+import org.apache.hadoop.hbase.SystemMonotonicClock;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Scan;
@@ -583,7 +586,7 @@ public class TestDefaultMemStore {
    */
   @Test
   public void testGetNextRowWithHybridLogicalClock() throws Exception {
-    testGetNextRow(new Clock.HLC());
+    testGetNextRow(new HybridLogicalClock());
   }
 
   /** Test getNextRow from memstore with the system monotonic clock
@@ -591,7 +594,7 @@ public class TestDefaultMemStore {
    */
   @Test
   public void testGetNextRowWithSystemMonotonicClock() throws Exception {
-    testGetNextRow(new Clock.SystemMonotonic());
+    testGetNextRow(new SystemMonotonicClock());
   }
 
   /** Test getNextRow from memstore with the system clock
@@ -599,7 +602,7 @@ public class TestDefaultMemStore {
    */
   @Test
   public void testGetNextRowWithSystemClock() throws Exception {
-    testGetNextRow(new Clock.System());
+    testGetNextRow(new SystemClock());
   }
 
   public void testGetNextRow(Clock clock) throws Exception {

http://git-wip-us.apache.org/repos/asf/hbase/blob/82a9cec5/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 a7e590b..1c2fa5a 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
@@ -79,6 +79,9 @@ 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.HybridLogicalClock;
+import org.apache.hadoop.hbase.SystemClock;
+import org.apache.hadoop.hbase.SystemMonotonicClock;
 import org.apache.hadoop.hbase.TimestampType;
 import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.CompatibilitySingletonFactory;
@@ -768,12 +771,12 @@ public class TestHRegion {
 
   @Test
   public void testHybridLogicalClockUpdatesOnRecoveryEditReplay() throws Exception {
-    long systemTime = Clock.SYSTEM_CLOCK.now();
+    long systemTime = new SystemClock().now();
     Clock mockSystemClock = mock(Clock.class);
     when(mockSystemClock.now()).thenReturn(systemTime);
     when(mockSystemClock.getTimeUnit()).thenReturn(TimeUnit.MILLISECONDS);
     when(mockSystemClock.isMonotonic()).thenReturn(true);
-    Clock hlClock = new Clock.HLC(new Clock.SystemMonotonic(mockSystemClock));
+    Clock hlClock = new HybridLogicalClock(new SystemMonotonicClock(mockSystemClock));
 
     long maxSeqId = 1050;
     long minSeqId = 1000;
@@ -800,12 +803,12 @@ public class TestHRegion {
 
   @Test
   public void testSystemMonotonicClockUpdatesOnRecoveryEditReplay() throws Exception {
-    long systemTime = Clock.SYSTEM_CLOCK.now();
+    long systemTime = new SystemClock().now();
     Clock mockSystemClock = mock(Clock.class);
     when(mockSystemClock.now()).thenReturn(systemTime);
     when(mockSystemClock.getTimeUnit()).thenReturn(TimeUnit.MILLISECONDS);
     when(mockSystemClock.isMonotonic()).thenReturn(true);
-    Clock systemMonotonicClock = new Clock.SystemMonotonic(mockSystemClock);
+    Clock systemMonotonicClock = new SystemMonotonicClock(mockSystemClock);
 
     long maxSeqId = 1050;
     long minSeqId = 1000;
@@ -6120,7 +6123,7 @@ public class TestHRegion {
 
   @Test
   public void testCellTTLsWithHybridLogicalClock() throws IOException {
-    testCellTTLs(ClockType.HLC);
+    testCellTTLs(ClockType.HYBRID_LOGICAL);
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/hbase/blob/82a9cec5/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 aceff82..f544564 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
@@ -59,8 +59,8 @@ import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.SystemClock;
 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;
@@ -173,7 +173,7 @@ 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.getClock((ClockType)any())).thenReturn(new Clock.System());
+    when(rss.getClock((ClockType)any())).thenReturn(new SystemClock());
     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/82a9cec5/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 9c33a9b..7120ab9 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
@@ -27,10 +27,10 @@ import java.util.ArrayList;
 import java.util.List;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.SystemClock;
 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;
@@ -108,7 +108,7 @@ 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.getClock(ClockType.SYSTEM)).thenReturn(new Clock.System());
+    Mockito.when(rss.getClock(ClockType.SYSTEM)).thenReturn(new SystemClock());
     // Set max size for this 'table'.
     long maxSplitSize = 1024L;
     htd.setMaxFileSize(maxSplitSize);
@@ -170,7 +170,7 @@ public class TestRegionSplitPolicy {
     Mockito.when(mockRegion.getRegionServerServices()).thenReturn(rss);
     Mockito.when(mockRegion.getBlockedRequestsCount()).thenReturn(0L);
     Mockito.when(mockRegion.getWriteRequestsCount()).thenReturn(0L);
-    Mockito.when(rss.getClock(ClockType.SYSTEM)).thenReturn(new Clock.System());
+    Mockito.when(rss.getClock(ClockType.SYSTEM)).thenReturn(new SystemClock());
 
 
     BusyRegionSplitPolicy policy =

http://git-wip-us.apache.org/repos/asf/hbase/blob/82a9cec5/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 a832c70..c7ccc57 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
@@ -44,9 +44,12 @@ 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.HybridLogicalClock;
 import org.apache.hadoop.hbase.KeepDeletedCells;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.KeyValueTestUtil;
+import org.apache.hadoop.hbase.SystemClock;
+import org.apache.hadoop.hbase.SystemMonotonicClock;
 import org.apache.hadoop.hbase.TimestampType;
 import org.apache.hadoop.hbase.client.Get;
 import org.apache.hadoop.hbase.client.Scan;
@@ -823,7 +826,7 @@ public class TestStoreScanner {
   public void testWildCardTtlScan() throws IOException {
     //testWildCardTtlScan(ClockType.SYSTEM);
     //testWildCardTtlScan(ClockType.SYSTEM_MONOTONIC);
-    testWildCardTtlScan(ClockType.HLC);
+    testWildCardTtlScan(ClockType.HYBRID_LOGICAL);
   }
 
   public void testWildCardTtlScan(ClockType clockType) throws IOException {
@@ -911,17 +914,17 @@ public class TestStoreScanner {
 
   @Test
   public void testExpiredDeleteFamilyWithHybridLogicalClock() throws Exception {
-    testExpiredDeleteFamily(new Clock.HLC());
+    testExpiredDeleteFamily(new HybridLogicalClock());
   }
 
   @Test
   public void testExpiredDeleteFamilyWithSystemMonotonicClock() throws Exception {
-    testExpiredDeleteFamily(new Clock.SystemMonotonic());
+    testExpiredDeleteFamily(new SystemMonotonicClock());
   }
 
   @Test
   public void testExpiredDeleteFamilyWithSystemClock() throws Exception {
-    testExpiredDeleteFamily(new Clock.System());
+    testExpiredDeleteFamily(new SystemClock());
   }
 
   /**
@@ -959,17 +962,17 @@ public class TestStoreScanner {
 
   @Test
   public void testDeleteMarkerLongevityWithHybridLogicalClock() throws Exception {
-    testDeleteMarkerLongevity(new Clock.HLC());
+    testDeleteMarkerLongevity(new HybridLogicalClock());
   }
 
   @Test
   public void testDeleteMarkerLongevityWithSystemMonotonicClock() throws Exception {
-    testDeleteMarkerLongevity(new Clock.SystemMonotonic());
+    testDeleteMarkerLongevity(new SystemMonotonicClock());
   }
 
   @Test
   public void testDeleteMarkerLongevityWithSystemClock() throws Exception {
-    testDeleteMarkerLongevity(new Clock.System());
+    testDeleteMarkerLongevity(new SystemClock());
   }
 
   public void testDeleteMarkerLongevity(Clock clock) throws Exception {

http://git-wip-us.apache.org/repos/asf/hbase/blob/82a9cec5/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 c866833..4b448e6 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
@@ -32,7 +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;
@@ -41,6 +40,7 @@ import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.Server;
 import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.SystemClock;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.ClusterConnection;
 import org.apache.hadoop.hbase.client.Durability;
@@ -205,7 +205,7 @@ public class TestWALLockup {
     Mockito.when(server.isStopped()).thenReturn(false);
     Mockito.when(server.isAborted()).thenReturn(false);
     RegionServerServices services = Mockito.mock(RegionServerServices.class);
-    Mockito.when(services.getClock(ClockType.SYSTEM)).thenReturn(new Clock.System());
+    Mockito.when(services.getClock(ClockType.SYSTEM)).thenReturn(new SystemClock());
 
     // 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/82a9cec5/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 03f17cd..bed393f 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
@@ -653,17 +653,17 @@ public abstract class AbstractTestWALReplay {
 
   @Test
   public void testReplayEditsAfterAbortingFlushWithHybridLogicalClock() throws Exception {
-    testReplayEditsAfterAbortingFlush(new Clock.HLC());
+    testReplayEditsAfterAbortingFlush(new HybridLogicalClock());
   }
 
   @Test
   public void testReplayEditsAfterAbortingFlushWithSystemMonotonicClock() throws Exception {
-    testReplayEditsAfterAbortingFlush(new Clock.SystemMonotonic());
+    testReplayEditsAfterAbortingFlush(new SystemMonotonicClock());
   }
 
   @Test
   public void testReplayEditsAfterAbortingFlushWithSystemClock() throws Exception {
-    testReplayEditsAfterAbortingFlush(new Clock.System());
+    testReplayEditsAfterAbortingFlush(new SystemClock());
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/82a9cec5/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 c552384..7cabda1 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
@@ -61,7 +61,6 @@ 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;
@@ -82,7 +81,7 @@ public class TestCellACLWithMultipleVersions extends SecureTestUtil {
   @Parameters()
   public static Iterable<Object> data() {
     return Arrays.asList(new Object[] {ClockType
-        .SYSTEM, ClockType.SYSTEM_MONOTONIC, ClockType.HLC});
+        .SYSTEM, ClockType.SYSTEM_MONOTONIC, ClockType.HYBRID_LOGICAL });
   }
 
   public TestCellACLWithMultipleVersions(ClockType clockType) {
@@ -418,9 +417,9 @@ 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.
+    // Setting of future timestamps is not allowed with System Monotonic and Hybrid Logical.
     // So need not run this test against these two clocks.
-    if (clockType == ClockType.HLC || clockType == ClockType.SYSTEM_MONOTONIC) {
+    if (clockType == ClockType.HYBRID_LOGICAL || clockType == ClockType.SYSTEM_MONOTONIC) {
       assertTrue(true);
       return;
     }
@@ -809,8 +808,8 @@ public class TestCellACLWithMultipleVersions extends SecureTestUtil {
   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) {
+    // Hybrid Logical and System Monotonic Clocks.
+    if (clockType == ClockType.HYBRID_LOGICAL || clockType == ClockType.SYSTEM_MONOTONIC) {
       assertTrue(true);
       return;
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/82a9cec5/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 dfd34a4..d1f0282 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
@@ -151,7 +151,7 @@ public class TestCoprocessorScanPolicy {
 
   @Test
   public void testTTl() throws Exception {
-    testTTL(ClockType.HLC);
+    testTTL(ClockType.HYBRID_LOGICAL);
     testTTL(ClockType.SYSTEM_MONOTONIC);
     testTTL(ClockType.SYSTEM);
   }