You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@geode.apache.org by kl...@apache.org on 2019/08/08 16:33:47 UTC

[geode] branch develop updated: GEODE-7010: Replace static globals in CachePerfStats with StatisticsClock (#3859)

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

klund pushed a commit to branch develop
in repository https://gitbox.apache.org/repos/asf/geode.git


The following commit(s) were added to refs/heads/develop by this push:
     new b492794  GEODE-7010: Replace static globals in CachePerfStats with StatisticsClock (#3859)
b492794 is described below

commit b49279477a4c79b90220fb34239d12c2d9e50782
Author: Kirk Lund <kl...@apache.org>
AuthorDate: Thu Aug 8 09:33:26 2019 -0700

    GEODE-7010: Replace static globals in CachePerfStats with StatisticsClock (#3859)
    
    * Replace CachePerfStats.getStatTime with Cache owned StatisticsClock
    * Inject StatisticsClock dependency via constructor
---
 ...ingGiiOplogWithMissingCreateRegressionTest.java |   9 +-
 .../GiiDiskAccessExceptionRegressionTest.java      |   9 +-
 .../internal/cache/ha/HAConflationDUnitTest.java   | 113 ++++++------
 .../internal/cache/ha/HADuplicateDUnitTest.java    |  47 +++--
 .../geode/internal/cache/ha/HAExpiryDUnitTest.java |  25 +--
 .../geode/internal/cache/ha/HAGIIBugDUnitTest.java |  61 +++----
 .../geode/internal/cache/ha/HARegionDUnitTest.java |   6 +-
 .../internal/cache/ha/HARegionQueueDUnitTest.java  |  21 ++-
 .../cache/tier/sockets/HABug36738DUnitTest.java    |   3 +-
 .../SerialAsyncEventQueueImplJUnitTest.java        |   4 +-
 .../LRUClearWithDiskRegionOpRegressionTest.java    |   3 +-
 .../apache/geode/internal/cache/PRTXJUnitTest.java |   4 +-
 .../cache/PartitionedRegionDataStoreJUnitTest.java |   3 +-
 .../cache/ServerBuilderIntegrationTest.java        |  25 ++-
 .../ha/BlockingHARQAddOperationJUnitTest.java      |   6 +-
 .../cache/ha/BlockingHARQStatsJUnitTest.java       |   7 +-
 .../cache/ha/BlockingHARegionJUnitTest.java        |  15 +-
 .../cache/ha/HARQAddOperationJUnitTest.java        |   7 +-
 .../geode/internal/cache/ha/HARegionJUnitTest.java |   4 +-
 .../cache/ha/HARegionQueueIntegrationTest.java     |  45 +----
 .../internal/cache/ha/HARegionQueueJUnitTest.java  |  27 +--
 .../cache/ha/HARegionQueueStartStopJUnitTest.java  |   3 +-
 .../cache/ha/HARegionQueueStatsJUnitTest.java      |   5 +-
 .../cache/tier/sockets/CacheClientProxyTest.java   |   5 +-
 .../bean/stats/AsyncEventQueueStatsJUnitTest.java  |   3 +-
 .../bean/stats/GatewayMBeanBridgeJUnitTest.java    |   3 +-
 .../bean/stats/RegionStatsJUnitTest.java           |  21 +--
 .../apache/geode/cache/DynamicRegionFactory.java   |   5 +-
 .../internal/AsyncEventQueueFactoryImpl.java       |   8 +-
 .../asyncqueue/internal/AsyncEventQueueStats.java  |   6 +-
 .../internal/ParallelAsyncEventQueueImpl.java      |  11 +-
 .../internal/SerialAsyncEventQueueImpl.java        |  11 +-
 .../geode/cache/client/internal/PoolImpl.java      |   5 +-
 .../geode/cache/client/internal/ProxyCache.java    |   4 +-
 .../geode/cache/client/internal/ProxyRegion.java   |   7 +-
 .../geode/cache/query/internal/DefaultQuery.java   |  12 +-
 .../apache/geode/cache/query/internal/QRegion.java |   3 +-
 .../cache/query/internal/index/HashIndexSet.java   |   2 +-
 .../cache/query/internal/index/IndexStats.java     |  20 +-
 .../internal/admin/remote/RemoteBridgeServer.java  |   6 +
 .../internal/cache/AbstractBucketRegionQueue.java  |   8 +-
 .../geode/internal/cache/AbstractRegion.java       |  20 +-
 .../internal/cache/AbstractUpdateOperation.java    |   4 +-
 .../apache/geode/internal/cache/BucketRegion.java  |   6 +-
 .../geode/internal/cache/BucketRegionQueue.java    |   6 +-
 .../geode/internal/cache/CachePerfStats.java       |  91 +++-------
 .../geode/internal/cache/CacheServerImpl.java      |  15 +-
 .../geode/internal/cache/ColocationHelper.java     |   6 +-
 .../apache/geode/internal/cache/DistTXState.java   |   6 +-
 .../internal/cache/DistTXStateOnCoordinator.java   |   6 +-
 .../geode/internal/cache/DistTXStateProxyImpl.java |   9 +-
 .../cache/DistTXStateProxyImplOnCoordinator.java   |  15 +-
 .../cache/DistTXStateProxyImplOnDatanode.java      |  13 +-
 .../geode/internal/cache/DistributedRegion.java    |   8 +-
 .../geode/internal/cache/DummyCachePerfStats.java  |   4 +-
 .../geode/internal/cache/EntryEventImpl.java       |   2 +-
 .../geode/internal/cache/GemFireCacheImpl.java     |  47 +++--
 .../org/apache/geode/internal/cache/HARegion.java  |  10 +-
 .../apache/geode/internal/cache/InternalCache.java |   4 +-
 .../cache/InternalCacheForClientAccess.java        |   6 +
 .../geode/internal/cache/InternalCacheServer.java  |   3 +
 .../apache/geode/internal/cache/LocalDataSet.java  |   1 -
 .../apache/geode/internal/cache/LocalRegion.java   |  66 ++++---
 .../geode/internal/cache/PartitionedRegion.java    |  34 ++--
 .../internal/cache/PartitionedRegionDataStore.java |  19 +-
 .../internal/cache/PartitionedRegionHelper.java    |   3 +-
 .../internal/cache/PartitionedRegionStats.java     | 128 ++++++-------
 .../geode/internal/cache/PoolFactoryImpl.java      |   2 +-
 .../geode/internal/cache/RegionPerfStats.java      | 109 +++--------
 .../apache/geode/internal/cache/RegionStats.java   | 152 ++++++++++++++++
 .../apache/geode/internal/cache/ServerBuilder.java |  10 +-
 .../apache/geode/internal/cache/TXManagerImpl.java |  39 ++--
 .../org/apache/geode/internal/cache/TXState.java   |  31 ++--
 .../geode/internal/cache/TXStateProxyImpl.java     |  19 +-
 .../geode/internal/cache/eviction/HeapEvictor.java |  19 +-
 .../internal/cache/eviction/OffHeapEvictor.java    |   5 +-
 .../internal/cache/eviction/RegionEvictorTask.java |  10 +-
 .../geode/internal/cache/ha/HARegionQueue.java     | 202 +++++++--------------
 .../partitioned/CreateMissingBucketsTask.java      |   5 +-
 .../cache/snapshot/RegionSnapshotServiceImpl.java  |   5 +-
 .../cache/tier/sockets/AcceptorBuilder.java        |  20 +-
 .../internal/cache/tier/sockets/AcceptorImpl.java  |  22 ++-
 .../cache/tier/sockets/CacheClientNotifier.java    |  32 ++--
 .../cache/tier/sockets/CacheClientProxy.java       |  15 +-
 .../internal/cache/tx/PartitionedTXRegionStub.java |   5 +-
 .../internal/cache/wan/AbstractGatewaySender.java  |  23 ++-
 .../internal/cache/wan/GatewaySenderStats.java     |  25 ++-
 .../internal/cache/wan/InternalGatewaySender.java  |   3 +
 .../wan/parallel/ParallelGatewaySenderQueue.java   |  13 +-
 .../cache/wan/serial/SerialGatewaySenderQueue.java |  10 +-
 .../internal/cache/xmlcache/CacheCreation.java     |   7 +
 .../cache/xmlcache/CacheServerCreation.java        |   6 +
 .../xmlcache/ParallelAsyncEventQueueCreation.java  |   4 +-
 .../xmlcache/ParallelGatewaySenderCreation.java    |   4 +-
 .../xmlcache/SerialAsyncEventQueueCreation.java    |   4 +-
 .../xmlcache/SerialGatewaySenderCreation.java      |   4 +-
 .../DisabledStatisticsClock.java}                  |  26 +--
 .../EnabledStatisticsClock.java}                   |  24 +--
 .../StatisticsClock.java}                          |  38 ++--
 .../statistics/StatisticsClockFactory.java         |  85 +++++++++
 .../StatisticsClockSupplier.java}                  |  26 +--
 .../management/internal/FederatingManager.java     |  15 +-
 .../geode/management/internal/LocalManager.java    |  30 ++-
 .../apache/geode/management/internal/Manager.java  |  16 +-
 .../internal/SystemManagementService.java          |  18 +-
 .../internal/beans/QueryDataFunction.java          |   3 +-
 .../cache/AbstractDistributedRegionJUnitTest.java  |   8 +-
 .../internal/cache/AbstractRegionJUnitTest.java    |   3 +-
 .../internal/cache/BucketRegionJUnitTest.java      |   6 +-
 .../geode/internal/cache/BucketRegionTest.java     |  72 ++++----
 .../geode/internal/cache/CachePerfStatsTest.java   |   9 +-
 .../geode/internal/cache/CacheServerImplTest.java  |  41 +++--
 .../internal/cache/DistributedRegionJUnitTest.java |  12 +-
 .../DistributedRegionSearchLoadJUnitTest.java      |   4 +-
 .../internal/cache/LocalRegionPartialMockTest.java |   5 +-
 .../geode/internal/cache/LocalRegionTest.java      |  10 +-
 .../internal/cache/PartitionedRegionTest.java      |   3 +-
 .../geode/internal/cache/RegionPerfStatsTest.java  |   9 +-
 .../geode/internal/cache/ServerBuilderTest.java    |  21 ++-
 .../geode/internal/cache/TXManagerImplTest.java    |   5 +-
 .../geode/internal/cache/TXStateProxyImplTest.java |  26 +--
 .../apache/geode/internal/cache/TXStateTest.java   |  35 ++--
 .../internal/cache/UpdateOperationJUnitTest.java   |  11 +-
 .../geode/internal/cache/ha/HARegionQueueTest.java |   4 +-
 .../cache/partitioned/FetchKeysMessageTest.java    |   3 +-
 .../cache/partitioned/PartitionMessageTest.java    |   3 +-
 .../cache/tier/sockets/AcceptorBuilderTest.java    |  27 +++
 .../cache/tier/sockets/AcceptorImplTest.java       |  11 +-
 .../tier/sockets/CacheClientNotifierTest.java      |  19 +-
 .../sockets/command/TXFailoverCommandTest.java     |   3 +-
 .../wan/parallel/ParallelGatewaySenderHelper.java  |   3 +-
 .../ParallelQueueRemovalMessageJUnitTest.java      |   4 +-
 .../statistics/StatisticsClockFactoryTest.java     | 127 +++++++++++++
 .../bean/stats/MemberLevelStatsTest.java           |   5 +-
 .../cache/ha/TestBlockingHARegionQueue.java        |   8 +-
 .../internal/cache/ha/TestOnlyHARegionQueue.java   |  65 +++++++
 .../java/org/apache/geode/test/fake/Fakes.java     |   3 +
 .../internal/locator/wan/WANFactoryImpl.java       |   2 +-
 .../cache/wan/AbstractRemoteGatewaySender.java     |   6 +-
 .../cache/wan/GatewaySenderFactoryImpl.java        |  24 ++-
 .../wan/parallel/ParallelGatewaySenderImpl.java    |   6 +-
 .../cache/wan/serial/SerialGatewaySenderImpl.java  |   6 +-
 142 files changed, 1691 insertions(+), 1128 deletions(-)

diff --git a/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/ClearDuringGiiOplogWithMissingCreateRegressionTest.java b/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/ClearDuringGiiOplogWithMissingCreateRegressionTest.java
index 09109c2..1d70bfd 100644
--- a/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/ClearDuringGiiOplogWithMissingCreateRegressionTest.java
+++ b/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/ClearDuringGiiOplogWithMissingCreateRegressionTest.java
@@ -14,6 +14,7 @@
  */
 package org.apache.geode.internal.cache;
 
+import static org.apache.geode.internal.statistics.StatisticsClockFactory.disabledClock;
 import static org.apache.geode.test.dunit.Host.getHost;
 import static org.assertj.core.api.Assertions.assertThat;
 
@@ -38,15 +39,14 @@ import org.apache.geode.test.junit.rules.serializable.SerializableTemporaryFolde
 import org.apache.geode.test.junit.rules.serializable.SerializableTestName;
 
 /**
- * Bug37377 DUNIT Test: The Clear operation during a GII in progress can leave a Entry in the Oplog
+ * Regression test: The Clear operation during a GII in progress can leave a Entry in the Oplog
  * due to a race condition wherein the clearFlag getting set after the entry gets written to the
  * disk, The Test verifies the existence of the scenario.
  *
  * <p>
- * TRAC #37377: Clear operation with GII in progress may result in a deleted entry to be logged in
+ * Bug: Clear operation with GII in progress may result in a deleted entry to be logged in
  * the oplog without accompanying create
  */
-
 public class ClearDuringGiiOplogWithMissingCreateRegressionTest extends CacheTestCase {
 
   private static final int PUT_COUNT = 10000;
@@ -148,7 +148,8 @@ public class ClearDuringGiiOplogWithMissingCreateRegressionTest extends CacheTes
 
     DistributedRegion distRegion = new DistributedRegion(regionName, factory.create(), null,
         getCache(), new InternalRegionArguments().setDestroyLockFlag(true).setRecreateFlag(false)
-            .setSnapshotInputStream(null).setImageTarget(null));
+            .setSnapshotInputStream(null).setImageTarget(null),
+        disabledClock());
 
     distRegion.entries.setEntryFactory(new TestableDiskRegionEntryFactory());
 
diff --git a/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/GiiDiskAccessExceptionRegressionTest.java b/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/GiiDiskAccessExceptionRegressionTest.java
index 2e56d84..0a011e7 100644
--- a/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/GiiDiskAccessExceptionRegressionTest.java
+++ b/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/GiiDiskAccessExceptionRegressionTest.java
@@ -14,6 +14,7 @@
  */
 package org.apache.geode.internal.cache;
 
+import static org.apache.geode.internal.statistics.StatisticsClockFactory.disabledClock;
 import static org.apache.geode.test.dunit.Host.getHost;
 import static org.apache.geode.test.dunit.IgnoredException.addIgnoredException;
 import static org.apache.geode.test.dunit.Invoke.invokeInEveryVM;
@@ -43,12 +44,9 @@ import org.apache.geode.test.junit.rules.serializable.SerializableTestName;
  * Tests that if a node doing GII experiences DiskAccessException, it should also not try to recover
  * from the disk
  *
- * GiiDiskAccessExceptionRegressionTest
- *
  * <p>
- * TRAC #39079: Regions with persistence remain in use after IOException have occurred
+ * Bug: Regions with persistence remain in use after IOException have occurred
  */
-
 public class GiiDiskAccessExceptionRegressionTest extends CacheTestCase {
 
   private String uniqueName;
@@ -133,7 +131,8 @@ public class GiiDiskAccessExceptionRegressionTest extends CacheTestCase {
 
     DistributedRegion distributedRegion = new DistributedRegion(uniqueName, factory.create(), null,
         getCache(), new InternalRegionArguments().setDestroyLockFlag(true).setRecreateFlag(false)
-            .setSnapshotInputStream(null).setImageTarget(null));
+            .setSnapshotInputStream(null).setImageTarget(null),
+        disabledClock());
 
     distributedRegion.entries.setEntryFactory(new DiskRegionEntryThrowsFactory());
 
diff --git a/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/ha/HAConflationDUnitTest.java b/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/ha/HAConflationDUnitTest.java
index fd94599..8f1d2e5 100755
--- a/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/ha/HAConflationDUnitTest.java
+++ b/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/ha/HAConflationDUnitTest.java
@@ -130,7 +130,6 @@ public class HAConflationDUnitTest extends JUnit4CacheTestCase {
 
   /**
    * In this test do a create & then update on same key , the client should receive 2 calabcks.
-   *
    */
 
   @Test
@@ -146,7 +145,6 @@ public class HAConflationDUnitTest extends JUnit4CacheTestCase {
   /**
    * In this test do create , then update & update. The client should receive 2 callbacks , one for
    * create & one for the last update.
-   *
    */
   @Test
   public void testConflationUpdate() throws Exception {
@@ -169,7 +167,6 @@ public class HAConflationDUnitTest extends JUnit4CacheTestCase {
   /**
    * In this test do create , then update, update, invalidate. The client should receive 3
    * callbacks, one for create one for the last update and one for the invalidate.
-   *
    */
   @Test
   public void testConflationCreateUpdateInvalidate() throws Exception {
@@ -187,7 +184,6 @@ public class HAConflationDUnitTest extends JUnit4CacheTestCase {
   /**
    * In this test do a create , update , update & destroy. The client should receive 3 callbacks (
    * craete , conflated update & destroy).
-   *
    */
   @Test
   public void testConflationCreateUpdateDestroy() throws Exception {
@@ -348,86 +344,83 @@ public class HAConflationDUnitTest extends JUnit4CacheTestCase {
     return new Integer(server.getPort());
   }
 
-}
-
-
-class HAClientCountEventListener implements CacheListener, Declarable {
-
-  @Override
-  public void afterCreate(EntryEvent event) {
-    String key = (String) event.getKey();
-    if (key.equals(HAConflationDUnitTest.LAST_KEY)) {
-      synchronized (HAConflationDUnitTest.LOCK) {
-        HAConflationDUnitTest.lastKeyArrived = true;
-        HAConflationDUnitTest.LOCK.notifyAll();
+  private static class HAClientCountEventListener implements CacheListener, Declarable {
+
+    @Override
+    public void afterCreate(EntryEvent event) {
+      String key = (String) event.getKey();
+      if (key.equals(LAST_KEY)) {
+        synchronized (LOCK) {
+          lastKeyArrived = true;
+          LOCK.notifyAll();
+        }
+      } else {
+        actualNoEvents++;
       }
-    } else {
-      HAConflationDUnitTest.actualNoEvents++;
+
     }
 
-  }
+    @Override
+    public void afterUpdate(EntryEvent event) {
 
-  @Override
-  public void afterUpdate(EntryEvent event) {
+      actualNoEvents++;
 
-    HAConflationDUnitTest.actualNoEvents++;
-
-  }
+    }
 
-  @Override
-  public void afterInvalidate(EntryEvent event) {
+    @Override
+    public void afterInvalidate(EntryEvent event) {
 
-    HAConflationDUnitTest.actualNoEvents++;
+      actualNoEvents++;
 
-  }
-
-  @Override
-  public void afterDestroy(EntryEvent event) {
-    HAConflationDUnitTest.actualNoEvents++;
+    }
 
-  }
+    @Override
+    public void afterDestroy(EntryEvent event) {
+      actualNoEvents++;
 
-  @Override
-  public void afterRegionInvalidate(RegionEvent event) {
-    // TODO Auto-generated method stub
+    }
 
-  }
+    @Override
+    public void afterRegionInvalidate(RegionEvent event) {
+      // TODO Auto-generated method stub
 
-  @Override
-  public void afterRegionDestroy(RegionEvent event) {
-    // TODO Auto-generated method stub
+    }
 
-  }
+    @Override
+    public void afterRegionDestroy(RegionEvent event) {
+      // TODO Auto-generated method stub
 
-  @Override
-  public void afterRegionClear(RegionEvent event) {
-    // TODO Auto-generated method stub
+    }
 
-  }
+    @Override
+    public void afterRegionClear(RegionEvent event) {
+      // TODO Auto-generated method stub
 
-  @Override
-  public void afterRegionCreate(RegionEvent event) {
-    // TODO Auto-generated method stub
+    }
 
-  }
+    @Override
+    public void afterRegionCreate(RegionEvent event) {
+      // TODO Auto-generated method stub
 
-  @Override
-  public void afterRegionLive(RegionEvent event) {
-    // TODO NOT Auto-generated method stub, added by vrao
+    }
 
-  }
+    @Override
+    public void afterRegionLive(RegionEvent event) {
+      // TODO NOT Auto-generated method stub, added by vrao
 
+    }
 
 
-  @Override
-  public void close() {
-    // TODO Auto-generated method stub
+    @Override
+    public void close() {
+      // TODO Auto-generated method stub
 
-  }
+    }
 
-  @Override
-  public void init(Properties props) {
-    // TODO Auto-generated method stub
+    @Override
+    public void init(Properties props) {
+      // TODO Auto-generated method stub
 
+    }
   }
 }
diff --git a/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/ha/HADuplicateDUnitTest.java b/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/ha/HADuplicateDUnitTest.java
index 8e276e6..60137e2 100755
--- a/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/ha/HADuplicateDUnitTest.java
+++ b/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/ha/HADuplicateDUnitTest.java
@@ -275,35 +275,32 @@ public class HADuplicateDUnitTest extends JUnit4DistributedTestCase {
       cache.getDistributedSystem().disconnect();
     }
   }
-}
-
-// TODO: move these classes to be inner static classes
-
 
-// Listener class for the validation purpose
-class HAValidateDuplicateListener extends CacheListenerAdapter {
-  @Override
-  public void afterCreate(EntryEvent event) {
-    System.out.println("After Create");
-    HADuplicateDUnitTest.storeEvents.put(event.getKey(), event.getNewValue());
-  }
+  // Listener class for the validation purpose
+  private static class HAValidateDuplicateListener extends CacheListenerAdapter {
+    @Override
+    public void afterCreate(EntryEvent event) {
+      System.out.println("After Create");
+      storeEvents.put(event.getKey(), event.getNewValue());
+    }
 
-  @Override
-  public void afterUpdate(EntryEvent event) {
-    Object value = HADuplicateDUnitTest.storeEvents.get(event.getKey());
-    if (value == null)
-      HADuplicateDUnitTest.isEventDuplicate = false;
-    synchronized (HADuplicateDUnitTest.dummyObj) {
-      try {
-        HADuplicateDUnitTest.put_counter++;
-        if (HADuplicateDUnitTest.put_counter == HADuplicateDUnitTest.NO_OF_PUTS) {
-          HADuplicateDUnitTest.waitFlag = false;
-          HADuplicateDUnitTest.dummyObj.notifyAll();
+    @Override
+    public void afterUpdate(EntryEvent event) {
+      Object value = storeEvents.get(event.getKey());
+      if (value == null)
+        isEventDuplicate = false;
+      synchronized (dummyObj) {
+        try {
+          put_counter++;
+          if (put_counter == NO_OF_PUTS) {
+            waitFlag = false;
+            dummyObj.notifyAll();
+          }
+        } catch (Exception e) {
+          e.printStackTrace();
         }
-      } catch (Exception e) {
-        e.printStackTrace();
       }
-    }
 
+    }
   }
 }
diff --git a/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/ha/HAExpiryDUnitTest.java b/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/ha/HAExpiryDUnitTest.java
index cb5edf0..d90e384 100755
--- a/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/ha/HAExpiryDUnitTest.java
+++ b/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/ha/HAExpiryDUnitTest.java
@@ -16,6 +16,7 @@ package org.apache.geode.internal.cache.ha;
 
 import static org.apache.geode.cache.Region.SEPARATOR;
 import static org.apache.geode.internal.cache.ha.HARegionQueue.createRegionName;
+import static org.apache.geode.internal.statistics.StatisticsClockFactory.disabledClock;
 import static org.junit.Assert.assertNotNull;
 
 import java.util.Properties;
@@ -60,13 +61,13 @@ import org.apache.geode.test.junit.categories.ClientSubscriptionTest;
 @Category({ClientSubscriptionTest.class})
 public class HAExpiryDUnitTest extends JUnit4DistributedTestCase {
 
-  VM vm0 = null;
+  private VM vm0 = null;
 
-  VM vm1 = null;
+  private VM vm1 = null;
 
-  VM vm2 = null;
+  private VM vm2 = null;
 
-  VM vm3 = null;
+  private VM vm3 = null;
 
   protected static InternalCache cache = null;
 
@@ -74,13 +75,7 @@ public class HAExpiryDUnitTest extends JUnit4DistributedTestCase {
 
   private static final String REGION_NAME = "HAExpiryDUnitTest_region";
 
-  static RegionQueue regionqueue = null;
-
-  protected static int regionQueueSize = -1;
-
-  public HAExpiryDUnitTest() {
-    super();
-  }
+  private static int regionQueueSize = -1;
 
   /**
    * This function creates regionqueue on 4 VMs
@@ -178,7 +173,7 @@ public class HAExpiryDUnitTest extends JUnit4DistributedTestCase {
    * This function checks the regionqueue size before expiration. size should be > 0.
    *
    */
-  public static void checkSizeBeforeExpiration() throws Exception {
+  private static void checkSizeBeforeExpiration() throws Exception {
     HARegion regionForQueue = (HARegion) cache
         .getRegion(SEPARATOR + createRegionName(regionQueueName));
     final HARegionQueue regionqueue = regionForQueue.getOwner();
@@ -205,7 +200,7 @@ public class HAExpiryDUnitTest extends JUnit4DistributedTestCase {
    * This function checks the regionqueue size After expiration. size should be = 0.
    *
    */
-  public static void checkSizeAfterExpiration() throws Exception {
+  private static void checkSizeAfterExpiration() throws Exception {
 
     HARegion regionForQueue = (HARegion) cache
         .getRegion(SEPARATOR + createRegionName(regionQueueName));
@@ -239,13 +234,13 @@ public class HAExpiryDUnitTest extends JUnit4DistributedTestCase {
     assertNotNull(cache);
   }
 
-  public static void createRegionQueue(Boolean isDurable) throws Exception {
+  private static void createRegionQueue(Boolean isDurable) throws Exception {
     new HAExpiryDUnitTest().createCache(new Properties());
     HARegionQueueAttributes hattr = new HARegionQueueAttributes();
     // setting expiry time for the regionqueue.
     hattr.setExpiryTime(4);
     RegionQueue regionqueue = HARegionQueue.getHARegionQueueInstance(regionQueueName, cache, hattr,
-        HARegionQueue.NON_BLOCKING_HA_QUEUE, isDurable.booleanValue());
+        HARegionQueue.NON_BLOCKING_HA_QUEUE, isDurable.booleanValue(), disabledClock());
     assertNotNull(regionqueue);
     AttributesFactory factory = new AttributesFactory();
     factory.setScope(Scope.DISTRIBUTED_ACK);
diff --git a/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/ha/HAGIIBugDUnitTest.java b/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/ha/HAGIIBugDUnitTest.java
index 60e2ba1..d6b6001 100755
--- a/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/ha/HAGIIBugDUnitTest.java
+++ b/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/ha/HAGIIBugDUnitTest.java
@@ -14,6 +14,7 @@
  */
 package org.apache.geode.internal.cache.ha;
 
+import static org.apache.geode.internal.statistics.StatisticsClockFactory.disabledClock;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.fail;
 
@@ -167,7 +168,7 @@ public class HAGIIBugDUnitTest extends JUnit4DistributedTestCase {
           hattr.setExpiryTime(12000000);
           RegionQueue regionqueue = null;
           regionqueue = HARegionQueue.getHARegionQueueInstance(regionQueueName, cache, hattr,
-              HARegionQueue.NON_BLOCKING_HA_QUEUE, false);
+              HARegionQueue.NON_BLOCKING_HA_QUEUE, false, disabledClock());
           isHARegionQueueUp = true;
           vm0.invoke(setStopFlag());
           assertNotNull(regionqueue);
@@ -325,7 +326,7 @@ public class HAGIIBugDUnitTest extends JUnit4DistributedTestCase {
     // setting expiry time for the regionqueue.
     hattr.setExpiryTime(12000000);
     RegionQueue regionqueue = HARegionQueue.getHARegionQueueInstance(regionQueueName, cache, hattr,
-        HARegionQueue.NON_BLOCKING_HA_QUEUE, false);
+        HARegionQueue.NON_BLOCKING_HA_QUEUE, false, disabledClock());
     assertNotNull(regionqueue);
     AttributesFactory factory = new AttributesFactory();
     factory.setScope(Scope.DISTRIBUTED_ACK);
@@ -344,40 +345,18 @@ public class HAGIIBugDUnitTest extends JUnit4DistributedTestCase {
     }
   }
 
-}
-
-
-/**
- * This listener performs the put of Conflatable object in the regionqueue.
- */
-
-class vmListenerToPutInHARegionQueue extends CacheListenerAdapter {
-  @Override
-  public void afterCreate(EntryEvent event) {
-
-    Cache cache = event.getRegion().getCache();
-    HARegion regionForQueue = (HARegion) cache.getRegion(
-        Region.SEPARATOR + HARegionQueue.createRegionName(HAGIIBugDUnitTest.regionQueueName));
-    HARegionQueue regionqueue = regionForQueue.getOwner();
-
-    try {
-      regionqueue.put(new ConflatableObject(event.getKey(), event.getNewValue(),
-          ((EntryEventImpl) event).getEventId(), false, "region1"));
-    } catch (Exception e) {
-      e.printStackTrace();
-    }
-  }
-}
-
+  /**
+   * This listener performs the put of Conflatable object in the regionqueue.
+   */
+  private static class vmListenerToPutInHARegionQueue extends CacheListenerAdapter {
+    @Override
+    public void afterCreate(EntryEvent event) {
 
-class vmListenerToCheckHARegionQueue extends CacheListenerAdapter {
-  @Override
-  public void afterCreate(EntryEvent event) {
-    if (HAGIIBugDUnitTest.isHARegionQueueUp) {
       Cache cache = event.getRegion().getCache();
       HARegion regionForQueue = (HARegion) cache.getRegion(
-          Region.SEPARATOR + HARegionQueue.createRegionName(HAGIIBugDUnitTest.regionQueueName));
+          Region.SEPARATOR + HARegionQueue.createRegionName(regionQueueName));
       HARegionQueue regionqueue = regionForQueue.getOwner();
+
       try {
         regionqueue.put(new ConflatableObject(event.getKey(), event.getNewValue(),
             ((EntryEventImpl) event).getEventId(), false, "region1"));
@@ -386,4 +365,22 @@ class vmListenerToCheckHARegionQueue extends CacheListenerAdapter {
       }
     }
   }
+
+  private static class vmListenerToCheckHARegionQueue extends CacheListenerAdapter {
+    @Override
+    public void afterCreate(EntryEvent event) {
+      if (isHARegionQueueUp) {
+        Cache cache = event.getRegion().getCache();
+        HARegion regionForQueue = (HARegion) cache.getRegion(
+            Region.SEPARATOR + HARegionQueue.createRegionName(regionQueueName));
+        HARegionQueue regionqueue = regionForQueue.getOwner();
+        try {
+          regionqueue.put(new ConflatableObject(event.getKey(), event.getNewValue(),
+              ((EntryEventImpl) event).getEventId(), false, "region1"));
+        } catch (Exception e) {
+          e.printStackTrace();
+        }
+      }
+    }
+  }
 }
diff --git a/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/ha/HARegionDUnitTest.java b/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/ha/HARegionDUnitTest.java
index 4865786..b631e65 100755
--- a/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/ha/HARegionDUnitTest.java
+++ b/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/ha/HARegionDUnitTest.java
@@ -14,6 +14,7 @@
  */
 package org.apache.geode.internal.cache.ha;
 
+import static org.apache.geode.internal.statistics.StatisticsClockFactory.disabledClock;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 import static org.mockito.ArgumentMatchers.any;
@@ -224,7 +225,8 @@ public class HARegionDUnitTest extends JUnit4DistributedTestCase {
     when(harq.updateHAEventWrapper(any(), any(), any()))
         .thenAnswer(AdditionalAnswers.returnsSecondArg());
 
-    HARegion.getInstance(REGION_NAME, (GemFireCacheImpl) cache, harq, factory.create());
+    HARegion.getInstance(REGION_NAME, (GemFireCacheImpl) cache, harq, factory.create(),
+        disabledClock());
   }
 
   private static HARegionQueue hrq = null;
@@ -239,7 +241,7 @@ public class HARegionDUnitTest extends JUnit4DistributedTestCase {
      * factory.setDataPolicy(DataPolicy.REPLICATE);
      */
     hrq = HARegionQueue.getHARegionQueueInstance(REGION_NAME, cache,
-        HARegionQueue.NON_BLOCKING_HA_QUEUE, false);
+        HARegionQueue.NON_BLOCKING_HA_QUEUE, false, disabledClock());
     EventID id1 = new EventID(new byte[] {1}, 1, 1);
     EventID id2 = new EventID(new byte[] {1}, 1, 2);
     ConflatableObject c1 = new ConflatableObject("1", "1", id1, false, REGION_NAME);
diff --git a/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/ha/HARegionQueueDUnitTest.java b/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/ha/HARegionQueueDUnitTest.java
index d706a49..c552952 100755
--- a/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/ha/HARegionQueueDUnitTest.java
+++ b/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/ha/HARegionQueueDUnitTest.java
@@ -17,6 +17,7 @@ package org.apache.geode.internal.cache.ha;
 import static java.lang.Thread.yield;
 import static org.apache.geode.internal.cache.ha.HARegionQueue.NON_BLOCKING_HA_QUEUE;
 import static org.apache.geode.internal.cache.ha.HARegionQueue.getHARegionQueueInstance;
+import static org.apache.geode.internal.statistics.StatisticsClockFactory.disabledClock;
 import static org.apache.geode.test.awaitility.GeodeAwaitility.await;
 import static org.apache.geode.test.dunit.Assert.assertEquals;
 import static org.apache.geode.test.dunit.Assert.assertNotNull;
@@ -219,7 +220,7 @@ public class HARegionQueueDUnitTest extends JUnit4DistributedTestCase {
             hrqa.setExpiryTime(300);
             try {
               hrq = HARegionQueue.getHARegionQueueInstance("testregion1", cache, hrqa,
-                  HARegionQueue.NON_BLOCKING_HA_QUEUE, false);
+                  HARegionQueue.NON_BLOCKING_HA_QUEUE, false, disabledClock());
               // Do 1000 putand 100 take in a separate thread
               hrq.put(new ConflatableObject(new Long(1), new Long(1),
                   new EventID(new byte[] {0}, 1, 1), false, "dummy"));
@@ -281,7 +282,7 @@ public class HARegionQueueDUnitTest extends JUnit4DistributedTestCase {
         .thenAnswer(AdditionalAnswers.returnsSecondArg());
 
     HARegion.getInstance("HARegionQueueDUnitTest_region", (GemFireCacheImpl) cache, harq,
-        factory.create());
+        factory.create(), disabledClock());
   }
 
   private static void createRegionQueue() throws Exception {
@@ -292,7 +293,7 @@ public class HARegionQueueDUnitTest extends JUnit4DistributedTestCase {
      * factory.setDataPolicy(DataPolicy.REPLICATE);
      */
     hrq = HARegionQueue.getHARegionQueueInstance("HARegionQueueDUnitTest_region", cache,
-        HARegionQueue.NON_BLOCKING_HA_QUEUE, false);
+        HARegionQueue.NON_BLOCKING_HA_QUEUE, false, disabledClock());
     EventID id1 = new EventID(new byte[] {1}, 1, 1);
     EventID id2 = new EventID(new byte[] {1}, 1, 2);
     ConflatableObject c1 =
@@ -313,7 +314,7 @@ public class HARegionQueueDUnitTest extends JUnit4DistributedTestCase {
     HARegionQueueAttributes harqAttr = new HARegionQueueAttributes();
     harqAttr.setExpiryTime(3);
     hrq = HARegionQueue.getHARegionQueueInstance("HARegionQueueDUnitTest_region", cache, harqAttr,
-        HARegionQueue.NON_BLOCKING_HA_QUEUE, false);
+        HARegionQueue.NON_BLOCKING_HA_QUEUE, false, disabledClock());
   }
 
   private static void clearRegion() {
@@ -592,10 +593,10 @@ public class HARegionQueueDUnitTest extends JUnit4DistributedTestCase {
         try {
           if (createBlockingQueue) {
             hrq = HARegionQueue.getHARegionQueueInstance("testregion1", cache, hrqa,
-                HARegionQueue.BLOCKING_HA_QUEUE, false);
+                HARegionQueue.BLOCKING_HA_QUEUE, false, disabledClock());
           } else {
             hrq = HARegionQueue.getHARegionQueueInstance("testregion1", cache, hrqa,
-                HARegionQueue.NON_BLOCKING_HA_QUEUE, false);
+                HARegionQueue.NON_BLOCKING_HA_QUEUE, false, disabledClock());
           }
         } catch (Exception e) {
           throw new AssertionError(e);
@@ -777,7 +778,7 @@ public class HARegionQueueDUnitTest extends JUnit4DistributedTestCase {
             try {
               hrq = HARegionQueue.getHARegionQueueInstance(
                   "testNPEDueToHARegionQueueEscapeInConstructor", cache, hrqa,
-                  HARegionQueue.NON_BLOCKING_HA_QUEUE, false);
+                  HARegionQueue.NON_BLOCKING_HA_QUEUE, false, disabledClock());
               // changing OP_COUNT to 20 makes no difference in test time
               final int OP_COUNT = 200;
               // Do 1000 putand 100 take in a separate thread
@@ -824,7 +825,7 @@ public class HARegionQueueDUnitTest extends JUnit4DistributedTestCase {
             try {
               hrq = HARegionQueue.getHARegionQueueInstance(
                   "testNPEDueToHARegionQueueEscapeInConstructor", cache, hrqa,
-                  HARegionQueue.NON_BLOCKING_HA_QUEUE, false);
+                  HARegionQueue.NON_BLOCKING_HA_QUEUE, false, disabledClock());
             } catch (Exception e) {
               throw new AssertionError(e);
             }
@@ -955,7 +956,7 @@ public class HARegionQueueDUnitTest extends JUnit4DistributedTestCase {
     HARegionQueueAttributes attrs = new HARegionQueueAttributes();
     attrs.setExpiryTime(1);
     hrq = getHARegionQueueInstance("HARegionQueueDUnitTest_region", cache, attrs,
-        NON_BLOCKING_HA_QUEUE, false);
+        NON_BLOCKING_HA_QUEUE, false, disabledClock());
     // wait until we have a dead
     // server
     WaitCriterion ev = new WaitCriterion() {
@@ -987,7 +988,7 @@ public class HARegionQueueDUnitTest extends JUnit4DistributedTestCase {
     cache = test.createCache();
 
     hrq = HARegionQueue.getHARegionQueueInstance("HARegionQueueDUnitTest_region", cache,
-        HARegionQueue.NON_BLOCKING_HA_QUEUE, false);
+        HARegionQueue.NON_BLOCKING_HA_QUEUE, false, disabledClock());
 
     assertEquals(2, hrq.size());
 
diff --git a/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/tier/sockets/HABug36738DUnitTest.java b/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/tier/sockets/HABug36738DUnitTest.java
index 12cb5e9..0d1746a 100755
--- a/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/tier/sockets/HABug36738DUnitTest.java
+++ b/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/tier/sockets/HABug36738DUnitTest.java
@@ -14,6 +14,7 @@
  */
 package org.apache.geode.internal.cache.tier.sockets;
 
+import static org.apache.geode.internal.statistics.StatisticsClockFactory.disabledClock;
 import static org.apache.geode.test.awaitility.GeodeAwaitility.await;
 import static org.apache.geode.test.dunit.Invoke.invokeInEveryVM;
 import static org.junit.Assert.assertEquals;
@@ -124,7 +125,7 @@ public class HABug36738DUnitTest extends JUnit4DistributedTestCase {
         .thenAnswer(AdditionalAnswers.returnsSecondArg());
 
     haRegion = HARegion.getInstance(HAREGION_NAME, (GemFireCacheImpl) cache, harq,
-        factory.createRegionAttributes());
+        factory.createRegionAttributes(), disabledClock());
   }
 
   private void checkRegionQueueSize() {
diff --git a/geode-core/src/integrationTest/java/org/apache/geode/cache/asyncqueue/internal/SerialAsyncEventQueueImplJUnitTest.java b/geode-core/src/integrationTest/java/org/apache/geode/cache/asyncqueue/internal/SerialAsyncEventQueueImplJUnitTest.java
index d701dc2..0c23b9f 100644
--- a/geode-core/src/integrationTest/java/org/apache/geode/cache/asyncqueue/internal/SerialAsyncEventQueueImplJUnitTest.java
+++ b/geode-core/src/integrationTest/java/org/apache/geode/cache/asyncqueue/internal/SerialAsyncEventQueueImplJUnitTest.java
@@ -47,7 +47,9 @@ public class SerialAsyncEventQueueImplJUnitTest {
   public void testStopClearsStats() {
     GatewaySenderAttributes attrs = new GatewaySenderAttributes();
     attrs.id = AsyncEventQueueImpl.ASYNC_EVENT_QUEUE_PREFIX + "id";
-    SerialAsyncEventQueueImpl queue = new SerialAsyncEventQueueImpl(cache, attrs);
+    SerialAsyncEventQueueImpl queue = new SerialAsyncEventQueueImpl(cache,
+        cache.getInternalDistributedSystem().getStatisticsManager(), cache.getStatisticsClock(),
+        attrs);
     queue.getStatistics().incQueueSize(5);
     queue.getStatistics().incSecondaryQueueSize(6);
     queue.getStatistics().incTempQueueSize(10);
diff --git a/geode-core/src/integrationTest/java/org/apache/geode/internal/cache/LRUClearWithDiskRegionOpRegressionTest.java b/geode-core/src/integrationTest/java/org/apache/geode/internal/cache/LRUClearWithDiskRegionOpRegressionTest.java
index 1f3cad1..70812c4 100644
--- a/geode-core/src/integrationTest/java/org/apache/geode/internal/cache/LRUClearWithDiskRegionOpRegressionTest.java
+++ b/geode-core/src/integrationTest/java/org/apache/geode/internal/cache/LRUClearWithDiskRegionOpRegressionTest.java
@@ -14,6 +14,7 @@
  */
 package org.apache.geode.internal.cache;
 
+import static org.apache.geode.internal.statistics.StatisticsClockFactory.disabledClock;
 import static org.assertj.core.api.Assertions.assertThat;
 
 import java.io.File;
@@ -83,7 +84,7 @@ public class LRUClearWithDiskRegionOpRegressionTest {
         .setRecreateFlag(false).setSnapshotInputStream(null).setImageTarget(null);
 
     DistributedRegion distributedRegion =
-        new DistributedRegion(regionName, regionAttributes, null, cache, args);
+        new DistributedRegion(regionName, regionAttributes, null, cache, args, disabledClock());
 
     region = cache.createVMRegion(regionName, regionAttributes,
         new InternalRegionArguments().setInternalMetaRegion(distributedRegion)
diff --git a/geode-core/src/integrationTest/java/org/apache/geode/internal/cache/PRTXJUnitTest.java b/geode-core/src/integrationTest/java/org/apache/geode/internal/cache/PRTXJUnitTest.java
index 9699ffd..7ca7b1e 100644
--- a/geode-core/src/integrationTest/java/org/apache/geode/internal/cache/PRTXJUnitTest.java
+++ b/geode-core/src/integrationTest/java/org/apache/geode/internal/cache/PRTXJUnitTest.java
@@ -14,6 +14,8 @@
  */
 package org.apache.geode.internal.cache;
 
+import static org.apache.geode.internal.statistics.StatisticsClockFactory.disabledClock;
+
 import org.junit.Ignore;
 import org.junit.Test;
 
@@ -88,7 +90,7 @@ public class PRTXJUnitTest extends TXJUnitTest {
 
     PRWithLocalOps(String regionName, RegionAttributes ra, LocalRegion parentRegion,
         GemFireCacheImpl cache, InternalRegionArguments internalRegionArgs) {
-      super(regionName, ra, parentRegion, cache, internalRegionArgs);
+      super(regionName, ra, parentRegion, cache, internalRegionArgs, disabledClock());
     }
 
     @Override
diff --git a/geode-core/src/integrationTest/java/org/apache/geode/internal/cache/PartitionedRegionDataStoreJUnitTest.java b/geode-core/src/integrationTest/java/org/apache/geode/internal/cache/PartitionedRegionDataStoreJUnitTest.java
index 330004f..0b4c381 100644
--- a/geode-core/src/integrationTest/java/org/apache/geode/internal/cache/PartitionedRegionDataStoreJUnitTest.java
+++ b/geode-core/src/integrationTest/java/org/apache/geode/internal/cache/PartitionedRegionDataStoreJUnitTest.java
@@ -15,6 +15,7 @@
 package org.apache.geode.internal.cache;
 
 import static org.apache.geode.distributed.ConfigurationProperties.MCAST_PORT;
+import static org.apache.geode.internal.statistics.StatisticsClockFactory.disabledClock;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
@@ -79,7 +80,7 @@ public class PartitionedRegionDataStoreJUnitTest {
     PartitionedRegion pr = null;
     pr = (PartitionedRegion) cache.createRegion("PR2", ra);
     paf.setLocalProperties(null).create();
-    /* PartitionedRegionDataStore prDS = */ new PartitionedRegionDataStore(pr);
+    /* PartitionedRegionDataStore prDS = */ new PartitionedRegionDataStore(pr, disabledClock());
     /*
      * PartitionedRegionHelper.removeGlobalMetadataForFailedNode(PartitionedRegion.node,
      * prDS.partitionedRegion.getRegionIdentifier(), prDS.partitionedRegion.cache);
diff --git a/geode-core/src/integrationTest/java/org/apache/geode/internal/cache/ServerBuilderIntegrationTest.java b/geode-core/src/integrationTest/java/org/apache/geode/internal/cache/ServerBuilderIntegrationTest.java
index deec2ff..b898175 100644
--- a/geode-core/src/integrationTest/java/org/apache/geode/internal/cache/ServerBuilderIntegrationTest.java
+++ b/geode-core/src/integrationTest/java/org/apache/geode/internal/cache/ServerBuilderIntegrationTest.java
@@ -35,6 +35,7 @@ import org.apache.geode.cache.CacheFactory;
 import org.apache.geode.cache.wan.GatewayReceiver;
 import org.apache.geode.cache.wan.GatewayTransportFilter;
 import org.apache.geode.internal.cache.tier.Acceptor;
+import org.apache.geode.internal.statistics.StatisticsClockFactory;
 import org.apache.geode.test.junit.categories.ClientServerTest;
 
 @Category(ClientServerTest.class)
@@ -64,8 +65,9 @@ public class ServerBuilderIntegrationTest {
 
   @Test
   public void byDefaultCreatesServerWithCacheServerAcceptor() throws IOException {
-    server = new ServerBuilder(cache, cache.getSecurityService())
-        .createServer();
+    server = new ServerBuilder(cache, cache.getSecurityService(),
+        StatisticsClockFactory.disabledClock())
+            .createServer();
     server.setPort(0);
 
     server.start();
@@ -78,9 +80,10 @@ public class ServerBuilderIntegrationTest {
   public void forGatewayReceiverCreatesServerWithGatewayReceiverAcceptor() throws IOException {
     when(gatewayReceiver.getGatewayTransportFilters())
         .thenReturn(singletonList(mock(GatewayTransportFilter.class)));
-    server = new ServerBuilder(cache, cache.getSecurityService())
-        .forGatewayReceiver(gatewayReceiver)
-        .createServer();
+    server = new ServerBuilder(cache, cache.getSecurityService(),
+        StatisticsClockFactory.disabledClock())
+            .forGatewayReceiver(gatewayReceiver)
+            .createServer();
     server.setPort(0);
 
     server.start();
@@ -94,8 +97,9 @@ public class ServerBuilderIntegrationTest {
     cache.close();
     String membershipGroup = "group-m0";
     cache = (InternalCache) new CacheFactory().set(GROUPS, membershipGroup).create();
-    server = new ServerBuilder(cache, cache.getSecurityService())
-        .createServer();
+    server = new ServerBuilder(cache, cache.getSecurityService(),
+        StatisticsClockFactory.disabledClock())
+            .createServer();
 
     assertThat(server.getCombinedGroups()).containsExactly(membershipGroup);
   }
@@ -107,9 +111,10 @@ public class ServerBuilderIntegrationTest {
     cache.close();
     String membershipGroup = "group-m0";
     cache = (InternalCache) new CacheFactory().set(GROUPS, membershipGroup).create();
-    server = new ServerBuilder(cache, cache.getSecurityService())
-        .forGatewayReceiver(gatewayReceiver)
-        .createServer();
+    server = new ServerBuilder(cache, cache.getSecurityService(),
+        StatisticsClockFactory.disabledClock())
+            .forGatewayReceiver(gatewayReceiver)
+            .createServer();
 
     assertThat(server.getCombinedGroups()).doesNotContain(membershipGroup);
   }
diff --git a/geode-core/src/integrationTest/java/org/apache/geode/internal/cache/ha/BlockingHARQAddOperationJUnitTest.java b/geode-core/src/integrationTest/java/org/apache/geode/internal/cache/ha/BlockingHARQAddOperationJUnitTest.java
index 6a769af..c215767 100755
--- a/geode-core/src/integrationTest/java/org/apache/geode/internal/cache/ha/BlockingHARQAddOperationJUnitTest.java
+++ b/geode-core/src/integrationTest/java/org/apache/geode/internal/cache/ha/BlockingHARQAddOperationJUnitTest.java
@@ -14,6 +14,7 @@
  */
 package org.apache.geode.internal.cache.ha;
 
+import static org.apache.geode.internal.statistics.StatisticsClockFactory.disabledClock;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.fail;
@@ -51,7 +52,8 @@ public class BlockingHARQAddOperationJUnitTest extends HARQAddOperationJUnitTest
   protected HARegionQueue createHARegionQueue(String name)
       throws IOException, ClassNotFoundException, CacheException, InterruptedException {
     HARegionQueue regionqueue =
-        HARegionQueue.getHARegionQueueInstance(name, cache, HARegionQueue.BLOCKING_HA_QUEUE, false);
+        HARegionQueue.getHARegionQueueInstance(name, cache, HARegionQueue.BLOCKING_HA_QUEUE, false,
+            disabledClock());
     return regionqueue;
   }
 
@@ -64,7 +66,7 @@ public class BlockingHARQAddOperationJUnitTest extends HARQAddOperationJUnitTest
   protected HARegionQueue createHARegionQueue(String name, HARegionQueueAttributes attrs)
       throws IOException, ClassNotFoundException, CacheException, InterruptedException {
     HARegionQueue regionqueue = HARegionQueue.getHARegionQueueInstance(name, cache, attrs,
-        HARegionQueue.BLOCKING_HA_QUEUE, false);
+        HARegionQueue.BLOCKING_HA_QUEUE, false, disabledClock());
     return regionqueue;
   }
 
diff --git a/geode-core/src/integrationTest/java/org/apache/geode/internal/cache/ha/BlockingHARQStatsJUnitTest.java b/geode-core/src/integrationTest/java/org/apache/geode/internal/cache/ha/BlockingHARQStatsJUnitTest.java
index 75c4c50..32d2137 100755
--- a/geode-core/src/integrationTest/java/org/apache/geode/internal/cache/ha/BlockingHARQStatsJUnitTest.java
+++ b/geode-core/src/integrationTest/java/org/apache/geode/internal/cache/ha/BlockingHARQStatsJUnitTest.java
@@ -14,6 +14,8 @@
  */
 package org.apache.geode.internal.cache.ha;
 
+import static org.apache.geode.internal.statistics.StatisticsClockFactory.disabledClock;
+
 import java.io.IOException;
 
 import org.junit.experimental.categories.Category;
@@ -40,7 +42,8 @@ public class BlockingHARQStatsJUnitTest extends HARegionQueueStatsJUnitTest {
   protected HARegionQueue createHARegionQueue(String name)
       throws IOException, ClassNotFoundException, CacheException, InterruptedException {
     HARegionQueue regionqueue =
-        HARegionQueue.getHARegionQueueInstance(name, cache, HARegionQueue.BLOCKING_HA_QUEUE, false);
+        HARegionQueue.getHARegionQueueInstance(name, cache, HARegionQueue.BLOCKING_HA_QUEUE, false,
+            disabledClock());
     return regionqueue;
   }
 
@@ -55,7 +58,7 @@ public class BlockingHARQStatsJUnitTest extends HARegionQueueStatsJUnitTest {
   protected HARegionQueue createHARegionQueue(String name, HARegionQueueAttributes attrs)
       throws IOException, ClassNotFoundException, CacheException, InterruptedException {
     HARegionQueue regionqueue = HARegionQueue.getHARegionQueueInstance(name, cache, attrs,
-        HARegionQueue.BLOCKING_HA_QUEUE, false);
+        HARegionQueue.BLOCKING_HA_QUEUE, false, disabledClock());
     return regionqueue;
   }
 
diff --git a/geode-core/src/integrationTest/java/org/apache/geode/internal/cache/ha/BlockingHARegionJUnitTest.java b/geode-core/src/integrationTest/java/org/apache/geode/internal/cache/ha/BlockingHARegionJUnitTest.java
index fd36faf..c392bc2 100755
--- a/geode-core/src/integrationTest/java/org/apache/geode/internal/cache/ha/BlockingHARegionJUnitTest.java
+++ b/geode-core/src/integrationTest/java/org/apache/geode/internal/cache/ha/BlockingHARegionJUnitTest.java
@@ -18,6 +18,7 @@ import static java.lang.Thread.sleep;
 import static org.apache.geode.distributed.ConfigurationProperties.MCAST_PORT;
 import static org.apache.geode.internal.cache.ha.HARegionQueue.BLOCKING_HA_QUEUE;
 import static org.apache.geode.internal.cache.ha.HARegionQueue.getHARegionQueueInstance;
+import static org.apache.geode.internal.statistics.StatisticsClockFactory.disabledClock;
 import static org.apache.geode.test.dunit.ThreadUtils.join;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
@@ -72,7 +73,7 @@ public class BlockingHARegionJUnitTest {
     HARegionQueueAttributes harqa = new HARegionQueueAttributes();
     harqa.setBlockingQueueCapacity(1);
     HARegionQueue hrq = HARegionQueue.getHARegionQueueInstance("BlockingHARegionJUnitTest_Region",
-        cache, harqa, HARegionQueue.BLOCKING_HA_QUEUE, false);
+        cache, harqa, HARegionQueue.BLOCKING_HA_QUEUE, false, disabledClock());
     hrq.setPrimary(true);// fix for 40314 - capacity constraint is checked for primary only.
     Thread thread1 = new DoPuts(hrq, 1000);
     Thread thread2 = new DoTake(hrq, 1000);
@@ -105,7 +106,8 @@ public class BlockingHARegionJUnitTest {
     HARegionQueueAttributes harqa = new HARegionQueueAttributes();
     harqa.setBlockingQueueCapacity(1);
     final HARegionQueue hrq = getHARegionQueueInstance(
-        "BlockingHARegionJUnitTest_Region", cache, harqa, BLOCKING_HA_QUEUE, false);
+        "BlockingHARegionJUnitTest_Region", cache, harqa, BLOCKING_HA_QUEUE, false,
+        disabledClock());
     hrq.setPrimary(true);// fix for 40314 - capacity constraint is checked for primary only.
     final Thread thread1 = new DoPuts(hrq, 2);
     thread1.start();
@@ -176,7 +178,8 @@ public class BlockingHARegionJUnitTest {
     HARegionQueueAttributes harqa = new HARegionQueueAttributes();
     harqa.setBlockingQueueCapacity(10000);
     final HARegionQueue hrq = getHARegionQueueInstance(
-        "BlockingHARegionJUnitTest_Region", cache, harqa, BLOCKING_HA_QUEUE, false);
+        "BlockingHARegionJUnitTest_Region", cache, harqa, BLOCKING_HA_QUEUE, false,
+        disabledClock());
     hrq.setPrimary(true);// fix for 40314 - capacity constraint is checked for primary only.
     Thread thread1 = new DoPuts(hrq, 20000, 1);
     Thread thread2 = new DoPuts(hrq, 20000, 2);
@@ -246,7 +249,8 @@ public class BlockingHARegionJUnitTest {
     HARegionQueueAttributes harqa = new HARegionQueueAttributes();
     harqa.setBlockingQueueCapacity(10000);
     final HARegionQueue hrq = getHARegionQueueInstance(
-        "BlockingHARegionJUnitTest_Region", cache, harqa, BLOCKING_HA_QUEUE, false);
+        "BlockingHARegionJUnitTest_Region", cache, harqa, BLOCKING_HA_QUEUE, false,
+        disabledClock());
     Thread thread1 = new DoPuts(hrq, 40000, 1);
     Thread thread2 = new DoPuts(hrq, 40000, 2);
     Thread thread3 = new DoPuts(hrq, 40000, 3);
@@ -336,7 +340,8 @@ public class BlockingHARegionJUnitTest {
       harqa.setBlockingQueueCapacity(1);
       harqa.setExpiryTime(180);
       final HARegionQueue hrq = HARegionQueue.getHARegionQueueInstance(
-          "BlockingHARegionJUnitTest_Region", cache, harqa, HARegionQueue.BLOCKING_HA_QUEUE, false);
+          "BlockingHARegionJUnitTest_Region", cache, harqa, HARegionQueue.BLOCKING_HA_QUEUE, false,
+          disabledClock());
       hrq.setPrimary(true);// fix for 40314 - capacity constraint is checked for primary only.
       final EventID id1 = new EventID(new byte[] {1}, 1, 2); // violation
       final EventID ignore = new EventID(new byte[] {1}, 1, 1); //
diff --git a/geode-core/src/integrationTest/java/org/apache/geode/internal/cache/ha/HARQAddOperationJUnitTest.java b/geode-core/src/integrationTest/java/org/apache/geode/internal/cache/ha/HARQAddOperationJUnitTest.java
index 6342614..40a9289 100755
--- a/geode-core/src/integrationTest/java/org/apache/geode/internal/cache/ha/HARQAddOperationJUnitTest.java
+++ b/geode-core/src/integrationTest/java/org/apache/geode/internal/cache/ha/HARQAddOperationJUnitTest.java
@@ -15,6 +15,7 @@
 package org.apache.geode.internal.cache.ha;
 
 import static org.apache.geode.distributed.ConfigurationProperties.MCAST_PORT;
+import static org.apache.geode.internal.statistics.StatisticsClockFactory.disabledClock;
 import static org.apache.geode.test.awaitility.GeodeAwaitility.await;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
@@ -111,7 +112,7 @@ public class HARQAddOperationJUnitTest {
     factory.setDataPolicy(DataPolicy.REPLICATE);
     factory.setScope(Scope.DISTRIBUTED_ACK);
     HARegionQueue regionqueue = HARegionQueue.getHARegionQueueInstance(name, cache,
-        HARegionQueue.NON_BLOCKING_HA_QUEUE, false);
+        HARegionQueue.NON_BLOCKING_HA_QUEUE, false, disabledClock());
     return regionqueue;
   }
 
@@ -121,7 +122,7 @@ public class HARQAddOperationJUnitTest {
   protected HARegionQueue createHARegionQueue(String name, HARegionQueueAttributes attrs)
       throws IOException, ClassNotFoundException, CacheException, InterruptedException {
     HARegionQueue regionqueue = HARegionQueue.getHARegionQueueInstance(name, cache, attrs,
-        HARegionQueue.NON_BLOCKING_HA_QUEUE, false);
+        HARegionQueue.NON_BLOCKING_HA_QUEUE, false, disabledClock());
     return regionqueue;
   }
 
@@ -891,7 +892,7 @@ public class HARQAddOperationJUnitTest {
       HARegionQueueAttributes attrs = new HARegionQueueAttributes();
       attrs.setExpiryTime(10);
       final HARegionQueue regionqueue =
-          new HARegionQueue.TestOnlyHARegionQueue("testing", cache, attrs) {
+          new TestOnlyHARegionQueue("testing", cache, attrs, disabledClock()) {
             @Override
             CacheListener createCacheListenerForHARegion() {
               return new CacheListenerAdapter() {
diff --git a/geode-core/src/integrationTest/java/org/apache/geode/internal/cache/ha/HARegionJUnitTest.java b/geode-core/src/integrationTest/java/org/apache/geode/internal/cache/ha/HARegionJUnitTest.java
index d5e8431..3d6bb95 100755
--- a/geode-core/src/integrationTest/java/org/apache/geode/internal/cache/ha/HARegionJUnitTest.java
+++ b/geode-core/src/integrationTest/java/org/apache/geode/internal/cache/ha/HARegionJUnitTest.java
@@ -15,6 +15,7 @@
 package org.apache.geode.internal.cache.ha;
 
 import static org.apache.geode.distributed.ConfigurationProperties.MCAST_PORT;
+import static org.apache.geode.internal.statistics.StatisticsClockFactory.disabledClock;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 
@@ -100,7 +101,8 @@ public class HARegionJUnitTest {
     });
     RegionAttributes ra = factory.create();
     Region region =
-        HARegion.getInstance("HARegionJUnitTest_region", (GemFireCacheImpl) cache, null, ra);
+        HARegion.getInstance("HARegionJUnitTest_region", (GemFireCacheImpl) cache, null, ra,
+            disabledClock());
     region.getAttributesMutator().setEntryTimeToLive(ea);
     return region;
   }
diff --git a/geode-core/src/integrationTest/java/org/apache/geode/internal/cache/ha/HARegionQueueIntegrationTest.java b/geode-core/src/integrationTest/java/org/apache/geode/internal/cache/ha/HARegionQueueIntegrationTest.java
index 6592dce..4352448 100644
--- a/geode-core/src/integrationTest/java/org/apache/geode/internal/cache/ha/HARegionQueueIntegrationTest.java
+++ b/geode-core/src/integrationTest/java/org/apache/geode/internal/cache/ha/HARegionQueueIntegrationTest.java
@@ -20,7 +20,6 @@ import static org.assertj.core.api.Assertions.assertThat;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
-import static org.mockito.ArgumentMatchers.any;
 import static org.mockito.Mockito.doAnswer;
 import static org.mockito.Mockito.doReturn;
 import static org.mockito.Mockito.mock;
@@ -50,8 +49,6 @@ import org.mockito.MockitoAnnotations;
 import util.TestException;
 
 import org.apache.geode.CancelCriterion;
-import org.apache.geode.Statistics;
-import org.apache.geode.StatisticsType;
 import org.apache.geode.cache.AttributesFactory;
 import org.apache.geode.cache.Cache;
 import org.apache.geode.cache.CacheFactory;
@@ -61,12 +58,7 @@ import org.apache.geode.cache.EvictionAttributes;
 import org.apache.geode.cache.Region;
 import org.apache.geode.cache.RegionShortcut;
 import org.apache.geode.cache.Scope;
-import org.apache.geode.distributed.internal.DSClock;
-import org.apache.geode.distributed.internal.DistributionConfig;
-import org.apache.geode.distributed.internal.DistributionManager;
-import org.apache.geode.distributed.internal.InternalDistributedSystem;
 import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
-import org.apache.geode.internal.SystemTimer;
 import org.apache.geode.internal.Version;
 import org.apache.geode.internal.cache.CacheServerImpl;
 import org.apache.geode.internal.cache.CachedDeserializable;
@@ -86,6 +78,7 @@ import org.apache.geode.internal.cache.tier.sockets.ClientUpdateMessageImpl;
 import org.apache.geode.internal.cache.tier.sockets.ConnectionListener;
 import org.apache.geode.internal.cache.tier.sockets.HAEventWrapper;
 import org.apache.geode.internal.concurrent.ConcurrentHashSet;
+import org.apache.geode.internal.statistics.StatisticsClock;
 import org.apache.geode.internal.util.BlobHelper;
 import org.apache.geode.internal.util.concurrent.StoppableReentrantReadWriteLock;
 
@@ -108,7 +101,6 @@ public class HARegionQueueIntegrationTest {
     dataRegion = createDataRegion();
     ccn = createCacheClientNotifier();
     member = createMember();
-    HAContainerWrapper haContainerWrapper = (HAContainerWrapper) ccn.getHaContainer();
   }
 
   @After
@@ -125,38 +117,11 @@ public class HARegionQueueIntegrationTest {
     return cache.createRegionFactory(RegionShortcut.REPLICATE).create("data");
   }
 
-  private InternalCache createMockInternalCache() {
-    InternalCache mockInternalCache = mock(InternalCache.class);
-    doReturn(mock(SystemTimer.class)).when(mockInternalCache).getCCPTimer();
-    doReturn(mock(CancelCriterion.class)).when(mockInternalCache).getCancelCriterion();
-
-    InternalDistributedSystem mockInteralDistributedSystem = createMockInternalDistributedSystem();
-    doReturn(mockInteralDistributedSystem).when(mockInternalCache).getInternalDistributedSystem();
-    doReturn(mockInteralDistributedSystem).when(mockInternalCache).getDistributedSystem();
-
-    return mockInternalCache;
-  }
-
-  private InternalDistributedSystem createMockInternalDistributedSystem() {
-    InternalDistributedSystem mockInternalDistributedSystem =
-        mock(InternalDistributedSystem.class);
-    DistributionManager mockDistributionManager = mock(DistributionManager.class);
-
-    doReturn(mock(InternalDistributedMember.class)).when(mockInternalDistributedSystem)
-        .getDistributedMember();
-    doReturn(mock(Statistics.class)).when(mockInternalDistributedSystem)
-        .createAtomicStatistics(any(StatisticsType.class), any(String.class));
-    doReturn(mock(DistributionConfig.class)).when(mockDistributionManager).getConfig();
-    doReturn(mockDistributionManager).when(mockInternalDistributedSystem).getDistributionManager();
-    doReturn(mock(DSClock.class)).when(mockInternalDistributedSystem).getClock();
-
-    return mockInternalDistributedSystem;
-  }
-
   private CacheClientNotifier createCacheClientNotifier() {
     CacheClientNotifier ccn =
-        CacheClientNotifier.getInstance((InternalCache) cache, mock(CacheServerStats.class),
-            100000, 100000, mock(ConnectionListener.class), null, false);
+        CacheClientNotifier.getInstance((InternalCache) cache, mock(StatisticsClock.class),
+            mock(CacheServerStats.class), 100000, 100000, mock(ConnectionListener.class), null,
+            false);
     return ccn;
   }
 
@@ -653,7 +618,7 @@ public class HARegionQueueIntegrationTest {
 
     return new HARegionQueue("haRegion+" + index, haRegion, (InternalCache) cache, haContainer,
         null, (byte) 1, true, mock(HARegionQueueStats.class), giiLock, rwLock,
-        mock(CancelCriterion.class), puttingGIIDataInQueue);
+        mock(CancelCriterion.class), puttingGIIDataInQueue, mock(StatisticsClock.class));
   }
 
   private HARegionQueue createHARegionQueue(Map haContainer, int index) throws Exception {
diff --git a/geode-core/src/integrationTest/java/org/apache/geode/internal/cache/ha/HARegionQueueJUnitTest.java b/geode-core/src/integrationTest/java/org/apache/geode/internal/cache/ha/HARegionQueueJUnitTest.java
index 324846b..67c39a8 100755
--- a/geode-core/src/integrationTest/java/org/apache/geode/internal/cache/ha/HARegionQueueJUnitTest.java
+++ b/geode-core/src/integrationTest/java/org/apache/geode/internal/cache/ha/HARegionQueueJUnitTest.java
@@ -17,6 +17,7 @@ package org.apache.geode.internal.cache.ha;
 import static org.apache.geode.distributed.ConfigurationProperties.MCAST_PORT;
 import static org.apache.geode.internal.lang.SystemPropertyHelper.GEODE_PREFIX;
 import static org.apache.geode.internal.lang.SystemPropertyHelper.HA_REGION_QUEUE_EXPIRY_TIME_PROPERTY;
+import static org.apache.geode.internal.statistics.StatisticsClockFactory.disabledClock;
 import static org.apache.geode.test.awaitility.GeodeAwaitility.await;
 import static org.hamcrest.CoreMatchers.is;
 import static org.hamcrest.CoreMatchers.not;
@@ -188,7 +189,7 @@ public class HARegionQueueJUnitTest {
   @Test
   public void testBlockQueue() throws Exception {
     HARegionQueue regionQueue = HARegionQueue.getHARegionQueueInstance(testName.getMethodName(),
-        cache, HARegionQueue.BLOCKING_HA_QUEUE, false);
+        cache, HARegionQueue.BLOCKING_HA_QUEUE, false, disabledClock());
     Thread[] threads = new Thread[10];
     int threadsLength = threads.length;
     CyclicBarrier barrier = new CyclicBarrier(threadsLength + 1);
@@ -1075,7 +1076,8 @@ public class HARegionQueueJUnitTest {
   @Test
   public void testBlockingQueueForConcurrentPeekAndTake() throws Exception {
     TestBlockingHARegionQueue regionQueue =
-        new TestBlockingHARegionQueue("testBlockQueueForConcurrentPeekAndTake", cache);
+        new TestBlockingHARegionQueue("testBlockQueueForConcurrentPeekAndTake", cache,
+            disabledClock());
     Thread[] threads = new Thread[3];
 
     for (int i = 0; i < 3; i++) {
@@ -1136,7 +1138,8 @@ public class HARegionQueueJUnitTest {
   @Test
   public void testBlockingQueueForTakeWhenPeekInProgress() throws Exception {
     TestBlockingHARegionQueue regionQueue =
-        new TestBlockingHARegionQueue("testBlockQueueForTakeWhenPeekInProgress", cache);
+        new TestBlockingHARegionQueue("testBlockQueueForTakeWhenPeekInProgress", cache,
+            disabledClock());
     Thread[] threads = new Thread[3];
 
     for (int i = 0; i < 3; i++) {
@@ -1209,7 +1212,8 @@ public class HARegionQueueJUnitTest {
     haa.setExpiryTime(3);
 
     RegionQueue regionqueue =
-        new HARegionQueue.TestOnlyHARegionQueue(testName.getMethodName(), cache, haa) {
+        new TestOnlyHARegionQueue(testName.getMethodName(), cache, haa,
+            disabledClock()) {
           @Override
           CacheListener createCacheListenerForHARegion() {
 
@@ -1333,10 +1337,10 @@ public class HARegionQueueJUnitTest {
 
     if (createBlockingQueue) {
       return HARegionQueue.getHARegionQueueInstance(testName.getMethodName(), cache, haa,
-          HARegionQueue.BLOCKING_HA_QUEUE, false);
+          HARegionQueue.BLOCKING_HA_QUEUE, false, disabledClock());
     } else {
       return HARegionQueue.getHARegionQueueInstance(testName.getMethodName(), cache, haa,
-          HARegionQueue.NON_BLOCKING_HA_QUEUE, false);
+          HARegionQueue.NON_BLOCKING_HA_QUEUE, false, disabledClock());
     }
   }
 
@@ -1852,7 +1856,7 @@ public class HARegionQueueJUnitTest {
    */
   private HARegionQueue createHARegionQueue(String name)
       throws IOException, ClassNotFoundException, CacheException, InterruptedException {
-    return HARegionQueue.getHARegionQueueInstance(name, cache, queueType(), false);
+    return HARegionQueue.getHARegionQueueInstance(name, cache, queueType(), false, disabledClock());
   }
 
   /**
@@ -1860,7 +1864,7 @@ public class HARegionQueueJUnitTest {
    */
   private HARegionQueue createHARegionQueue(String name, int queueType)
       throws IOException, ClassNotFoundException, CacheException, InterruptedException {
-    return HARegionQueue.getHARegionQueueInstance(name, cache, queueType, false);
+    return HARegionQueue.getHARegionQueueInstance(name, cache, queueType, false, disabledClock());
   }
 
   /**
@@ -1868,7 +1872,8 @@ public class HARegionQueueJUnitTest {
    */
   HARegionQueue createHARegionQueue(String name, HARegionQueueAttributes attrs)
       throws IOException, ClassNotFoundException, CacheException, InterruptedException {
-    return HARegionQueue.getHARegionQueueInstance(name, cache, attrs, queueType(), false);
+    return HARegionQueue.getHARegionQueueInstance(name, cache, attrs, queueType(), false,
+        disabledClock());
   }
 
   /**
@@ -1892,11 +1897,11 @@ public class HARegionQueueJUnitTest {
   /**
    * Extends HARegionQueue for testing purposes. used by testSafeConflationRemoval
    */
-  static class HARQTestClass extends HARegionQueue.TestOnlyHARegionQueue {
+  static class HARQTestClass extends TestOnlyHARegionQueue {
 
     HARQTestClass(String regionName, InternalCache cache)
         throws IOException, ClassNotFoundException, CacheException, InterruptedException {
-      super(regionName, cache);
+      super(regionName, cache, disabledClock());
     }
 
     @Override
diff --git a/geode-core/src/integrationTest/java/org/apache/geode/internal/cache/ha/HARegionQueueStartStopJUnitTest.java b/geode-core/src/integrationTest/java/org/apache/geode/internal/cache/ha/HARegionQueueStartStopJUnitTest.java
index 7a5990f..e17287e 100755
--- a/geode-core/src/integrationTest/java/org/apache/geode/internal/cache/ha/HARegionQueueStartStopJUnitTest.java
+++ b/geode-core/src/integrationTest/java/org/apache/geode/internal/cache/ha/HARegionQueueStartStopJUnitTest.java
@@ -16,6 +16,7 @@ package org.apache.geode.internal.cache.ha;
 
 import static org.apache.geode.distributed.ConfigurationProperties.LOCATORS;
 import static org.apache.geode.distributed.ConfigurationProperties.MCAST_PORT;
+import static org.apache.geode.internal.statistics.StatisticsClockFactory.disabledClock;
 import static org.junit.Assert.fail;
 
 import java.io.IOException;
@@ -47,7 +48,7 @@ public class HARegionQueueStartStopJUnitTest {
   private RegionQueue createHARegionQueue(String name, InternalCache cache)
       throws IOException, ClassNotFoundException, CacheException, InterruptedException {
     RegionQueue regionqueue = HARegionQueue.getHARegionQueueInstance(name, cache,
-        HARegionQueue.NON_BLOCKING_HA_QUEUE, false);
+        HARegionQueue.NON_BLOCKING_HA_QUEUE, false, disabledClock());
     return regionqueue;
   }
 
diff --git a/geode-core/src/integrationTest/java/org/apache/geode/internal/cache/ha/HARegionQueueStatsJUnitTest.java b/geode-core/src/integrationTest/java/org/apache/geode/internal/cache/ha/HARegionQueueStatsJUnitTest.java
index 7196afe..5f35195 100755
--- a/geode-core/src/integrationTest/java/org/apache/geode/internal/cache/ha/HARegionQueueStatsJUnitTest.java
+++ b/geode-core/src/integrationTest/java/org/apache/geode/internal/cache/ha/HARegionQueueStatsJUnitTest.java
@@ -15,6 +15,7 @@
 package org.apache.geode.internal.cache.ha;
 
 import static org.apache.geode.distributed.ConfigurationProperties.MCAST_PORT;
+import static org.apache.geode.internal.statistics.StatisticsClockFactory.disabledClock;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotNull;
 
@@ -85,7 +86,7 @@ public class HARegionQueueStatsJUnitTest {
   protected HARegionQueue createHARegionQueue(String name)
       throws IOException, ClassNotFoundException, CacheException, InterruptedException {
     HARegionQueue regionqueue = HARegionQueue.getHARegionQueueInstance(name, cache,
-        HARegionQueue.NON_BLOCKING_HA_QUEUE, false);
+        HARegionQueue.NON_BLOCKING_HA_QUEUE, false, disabledClock());
     return regionqueue;
   }
 
@@ -102,7 +103,7 @@ public class HARegionQueueStatsJUnitTest {
     factory.setDataPolicy(DataPolicy.REPLICATE);
     factory.setScope(Scope.DISTRIBUTED_ACK);
     HARegionQueue regionqueue = HARegionQueue.getHARegionQueueInstance(name, cache, attrs,
-        HARegionQueue.NON_BLOCKING_HA_QUEUE, false);
+        HARegionQueue.NON_BLOCKING_HA_QUEUE, false, disabledClock());
     return regionqueue;
   }
 
diff --git a/geode-core/src/integrationTest/java/org/apache/geode/internal/cache/tier/sockets/CacheClientProxyTest.java b/geode-core/src/integrationTest/java/org/apache/geode/internal/cache/tier/sockets/CacheClientProxyTest.java
index 0d21cd3..f0b7f89 100644
--- a/geode-core/src/integrationTest/java/org/apache/geode/internal/cache/tier/sockets/CacheClientProxyTest.java
+++ b/geode-core/src/integrationTest/java/org/apache/geode/internal/cache/tier/sockets/CacheClientProxyTest.java
@@ -36,6 +36,7 @@ import org.apache.geode.distributed.DistributedMember;
 import org.apache.geode.internal.Version;
 import org.apache.geode.internal.cache.InternalCache;
 import org.apache.geode.internal.net.SocketCloser;
+import org.apache.geode.internal.statistics.StatisticsClock;
 import org.apache.geode.test.junit.rules.ExecutorServiceRule;
 import org.apache.geode.test.junit.rules.ServerStarterRule;
 
@@ -49,7 +50,6 @@ public class CacheClientProxyTest {
 
   @Test
   public void closeSocketShouldBeAtomic() {
-
     final CacheServerStats stats = mock(CacheServerStats.class);
     doNothing().when(stats).incCurrentQueueConnections();
 
@@ -73,7 +73,7 @@ public class CacheClientProxyTest {
 
     CacheClientProxy proxy = new CacheClientProxy(ccn, socket, proxyID, true,
         Handshake.CONFLATION_DEFAULT, Version.CURRENT, 1L, true,
-        null, null);
+        null, null, mock(StatisticsClock.class));
 
     CompletableFuture<Void> result1 = executorServiceRule.runAsync(() -> proxy.close());
     CompletableFuture<Void> result2 = executorServiceRule.runAsync(() -> proxy.close());
@@ -93,5 +93,4 @@ public class CacheClientProxyTest {
       closeSocketShouldBeAtomic();
     }
   }
-
 }
diff --git a/geode-core/src/integrationTest/java/org/apache/geode/management/bean/stats/AsyncEventQueueStatsJUnitTest.java b/geode-core/src/integrationTest/java/org/apache/geode/management/bean/stats/AsyncEventQueueStatsJUnitTest.java
index cc03446..8a65a04 100644
--- a/geode-core/src/integrationTest/java/org/apache/geode/management/bean/stats/AsyncEventQueueStatsJUnitTest.java
+++ b/geode-core/src/integrationTest/java/org/apache/geode/management/bean/stats/AsyncEventQueueStatsJUnitTest.java
@@ -14,6 +14,7 @@
  */
 package org.apache.geode.management.bean.stats;
 
+import static org.apache.geode.internal.statistics.StatisticsClockFactory.disabledClock;
 import static org.junit.Assert.assertEquals;
 
 import org.junit.Test;
@@ -33,7 +34,7 @@ public class AsyncEventQueueStatsJUnitTest extends MBeanStatsTestCase {
 
   @Override
   public void init() {
-    asyncEventQueueStats = new AsyncEventQueueStats(system, "test");
+    asyncEventQueueStats = new AsyncEventQueueStats(system, "test", disabledClock());
 
     bridge = new AsyncEventQueueMBeanBridge();
     bridge.addAsyncEventQueueStats(asyncEventQueueStats);
diff --git a/geode-core/src/integrationTest/java/org/apache/geode/management/bean/stats/GatewayMBeanBridgeJUnitTest.java b/geode-core/src/integrationTest/java/org/apache/geode/management/bean/stats/GatewayMBeanBridgeJUnitTest.java
index 971c703..cd1a55d 100644
--- a/geode-core/src/integrationTest/java/org/apache/geode/management/bean/stats/GatewayMBeanBridgeJUnitTest.java
+++ b/geode-core/src/integrationTest/java/org/apache/geode/management/bean/stats/GatewayMBeanBridgeJUnitTest.java
@@ -14,6 +14,7 @@
  */
 package org.apache.geode.management.bean.stats;
 
+import static org.apache.geode.internal.statistics.StatisticsClockFactory.disabledClock;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 
@@ -40,7 +41,7 @@ public class GatewayMBeanBridgeJUnitTest extends MBeanStatsTestCase {
 
   @Override
   public void init() {
-    senderStats = new GatewaySenderStats(system, "test");
+    senderStats = new GatewaySenderStats(system, "gatewaySenderStats-", "test", disabledClock());
 
     sender = Mockito.mock(AbstractGatewaySender.class);
     Mockito.when(sender.getStatistics()).thenReturn(senderStats);
diff --git a/geode-core/src/integrationTest/java/org/apache/geode/management/bean/stats/RegionStatsJUnitTest.java b/geode-core/src/integrationTest/java/org/apache/geode/management/bean/stats/RegionStatsJUnitTest.java
index 6b294ff..039de12 100644
--- a/geode-core/src/integrationTest/java/org/apache/geode/management/bean/stats/RegionStatsJUnitTest.java
+++ b/geode-core/src/integrationTest/java/org/apache/geode/management/bean/stats/RegionStatsJUnitTest.java
@@ -14,6 +14,7 @@
  */
 package org.apache.geode.management.bean.stats;
 
+import static org.apache.geode.internal.statistics.StatisticsClockFactory.enabledClock;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 
@@ -23,6 +24,7 @@ import org.junit.experimental.categories.Category;
 import org.apache.geode.internal.cache.CachePerfStats;
 import org.apache.geode.internal.cache.DiskRegionStats;
 import org.apache.geode.internal.cache.PartitionedRegionStats;
+import org.apache.geode.internal.statistics.StatisticsClock;
 import org.apache.geode.management.internal.beans.DiskRegionBridge;
 import org.apache.geode.management.internal.beans.PartitionedRegionBridge;
 import org.apache.geode.management.internal.beans.RegionMBeanBridge;
@@ -36,21 +38,18 @@ import org.apache.geode.test.junit.categories.JMXTest;
 public class RegionStatsJUnitTest extends MBeanStatsTestCase {
 
   private RegionMBeanBridge bridge;
-
   private PartitionedRegionBridge parBridge;
-
   private DiskRegionBridge diskBridge;
-
   private CachePerfStats cachePerfStats;
-
   private PartitionedRegionStats partitionedRegionStats;
-
   private DiskRegionStats diskRegionStats;
+  private StatisticsClock statisticsClock;
 
   @Override
   protected void init() {
-    cachePerfStats = new CachePerfStats(system);
-    partitionedRegionStats = new PartitionedRegionStats(system, "/tests");
+    statisticsClock = enabledClock();
+    cachePerfStats = new CachePerfStats(system, statisticsClock);
+    partitionedRegionStats = new PartitionedRegionStats(system, "/tests", statisticsClock);
     diskRegionStats = new DiskRegionStats(system, "test-disk");
 
     bridge = new RegionMBeanBridge(cachePerfStats);
@@ -84,7 +83,7 @@ public class RegionStatsJUnitTest extends MBeanStatsTestCase {
 
   @Test
   public void testDiskCounters() throws InterruptedException {
-    final long startTime = CachePerfStats.getStatTime();
+    final long startTime = statisticsClock.getTime();
 
     diskRegionStats.incNumEntriesInVM(10);
     diskRegionStats.incNumOverflowOnDisk(15);
@@ -97,8 +96,8 @@ public class RegionStatsJUnitTest extends MBeanStatsTestCase {
     assertEquals(10, getTotalDiskEntriesInVM());
     assertEquals(15, getTotalEntriesOnlyOnDisk());
 
-    diskRegionStats.endWrite(startTime, CachePerfStats.getStatTime());
-    diskRegionStats.endRead(startTime, CachePerfStats.getStatTime(), 1000);
+    diskRegionStats.endWrite(startTime, statisticsClock.getTime());
+    diskRegionStats.endRead(startTime, statisticsClock.getTime(), 1000);
 
     sample();
 
@@ -111,7 +110,7 @@ public class RegionStatsJUnitTest extends MBeanStatsTestCase {
 
   @Test
   public void testTimeBasedCounters() throws InterruptedException {
-    final long startTime = CachePerfStats.getStatTime();
+    final long startTime = statisticsClock.getTime();
 
     cachePerfStats.startCacheListenerCall();
     cachePerfStats.startCacheWriterCall();
diff --git a/geode-core/src/main/java/org/apache/geode/cache/DynamicRegionFactory.java b/geode-core/src/main/java/org/apache/geode/cache/DynamicRegionFactory.java
index 042e461..c145a37 100644
--- a/geode-core/src/main/java/org/apache/geode/cache/DynamicRegionFactory.java
+++ b/geode-core/src/main/java/org/apache/geode/cache/DynamicRegionFactory.java
@@ -879,7 +879,8 @@ public abstract class DynamicRegionFactory {
   // the meta data
   private class LocalMetaRegion extends LocalRegion {
     protected LocalMetaRegion(RegionAttributes attrs, InternalRegionArguments ira) {
-      super(DYNAMIC_REGION_LIST_NAME, attrs, null, DynamicRegionFactory.this.cache, ira);
+      super(DYNAMIC_REGION_LIST_NAME, attrs, null, DynamicRegionFactory.this.cache, ira,
+          DynamicRegionFactory.this.cache.getStatisticsClock());
       Assert.assertTrue(attrs.getScope().isLocal());
     }
 
@@ -989,7 +990,7 @@ public abstract class DynamicRegionFactory {
   private class DistributedMetaRegion extends DistributedRegion {
     protected DistributedMetaRegion(RegionAttributes attrs) {
       super(DYNAMIC_REGION_LIST_NAME, attrs, null, DynamicRegionFactory.this.cache,
-          new InternalRegionArguments());
+          new InternalRegionArguments(), DynamicRegionFactory.this.cache.getStatisticsClock());
     }
 
     // This is an internal uses only region
diff --git a/geode-core/src/main/java/org/apache/geode/cache/asyncqueue/internal/AsyncEventQueueFactoryImpl.java b/geode-core/src/main/java/org/apache/geode/cache/asyncqueue/internal/AsyncEventQueueFactoryImpl.java
index 70f0c2c..dc4ff03 100644
--- a/geode-core/src/main/java/org/apache/geode/cache/asyncqueue/internal/AsyncEventQueueFactoryImpl.java
+++ b/geode-core/src/main/java/org/apache/geode/cache/asyncqueue/internal/AsyncEventQueueFactoryImpl.java
@@ -204,7 +204,9 @@ public class AsyncEventQueueFactoryImpl implements AsyncEventQueueFactory {
       if (cache instanceof CacheCreation) {
         sender = new ParallelAsyncEventQueueCreation(cache, gatewaySenderAttributes);
       } else {
-        sender = new ParallelAsyncEventQueueImpl(cache, gatewaySenderAttributes);
+        sender = new ParallelAsyncEventQueueImpl(cache,
+            cache.getInternalDistributedSystem().getStatisticsManager(), cache.getStatisticsClock(),
+            gatewaySenderAttributes);
       }
       cache.addGatewaySender(sender);
 
@@ -217,7 +219,9 @@ public class AsyncEventQueueFactoryImpl implements AsyncEventQueueFactory {
       if (cache instanceof CacheCreation) {
         sender = new SerialAsyncEventQueueCreation(cache, gatewaySenderAttributes);
       } else {
-        sender = new SerialAsyncEventQueueImpl(cache, gatewaySenderAttributes);
+        sender = new SerialAsyncEventQueueImpl(cache,
+            cache.getInternalDistributedSystem().getStatisticsManager(), cache.getStatisticsClock(),
+            gatewaySenderAttributes);
       }
       cache.addGatewaySender(sender);
     }
diff --git a/geode-core/src/main/java/org/apache/geode/cache/asyncqueue/internal/AsyncEventQueueStats.java b/geode-core/src/main/java/org/apache/geode/cache/asyncqueue/internal/AsyncEventQueueStats.java
index baab769..ffffef4 100644
--- a/geode-core/src/main/java/org/apache/geode/cache/asyncqueue/internal/AsyncEventQueueStats.java
+++ b/geode-core/src/main/java/org/apache/geode/cache/asyncqueue/internal/AsyncEventQueueStats.java
@@ -19,6 +19,7 @@ import org.apache.geode.StatisticsType;
 import org.apache.geode.StatisticsTypeFactory;
 import org.apache.geode.annotations.Immutable;
 import org.apache.geode.internal.cache.wan.GatewaySenderStats;
+import org.apache.geode.internal.statistics.StatisticsClock;
 import org.apache.geode.internal.statistics.StatisticsTypeFactoryImpl;
 
 public class AsyncEventQueueStats extends GatewaySenderStats {
@@ -42,7 +43,8 @@ public class AsyncEventQueueStats extends GatewaySenderStats {
    * @param asyncQueueId The id of the <code>AsyncEventQueue</code> used to generate the name of the
    *        <code>Statistics</code>
    */
-  public AsyncEventQueueStats(StatisticsFactory f, String asyncQueueId) {
-    super(f, asyncQueueId, type);
+  public AsyncEventQueueStats(StatisticsFactory f, String asyncQueueId,
+      StatisticsClock statisticsClock) {
+    super(f, "asyncEventQueueStats-", asyncQueueId, type, statisticsClock);
   }
 }
diff --git a/geode-core/src/main/java/org/apache/geode/cache/asyncqueue/internal/ParallelAsyncEventQueueImpl.java b/geode-core/src/main/java/org/apache/geode/cache/asyncqueue/internal/ParallelAsyncEventQueueImpl.java
index 222f00b..acf2547 100644
--- a/geode-core/src/main/java/org/apache/geode/cache/asyncqueue/internal/ParallelAsyncEventQueueImpl.java
+++ b/geode-core/src/main/java/org/apache/geode/cache/asyncqueue/internal/ParallelAsyncEventQueueImpl.java
@@ -16,6 +16,7 @@ package org.apache.geode.cache.asyncqueue.internal;
 
 import org.apache.logging.log4j.Logger;
 
+import org.apache.geode.StatisticsFactory;
 import org.apache.geode.cache.EntryOperation;
 import org.apache.geode.cache.asyncqueue.AsyncEventListener;
 import org.apache.geode.cache.wan.GatewayTransportFilter;
@@ -38,18 +39,20 @@ import org.apache.geode.internal.cache.wan.parallel.ConcurrentParallelGatewaySen
 import org.apache.geode.internal.cache.xmlcache.CacheCreation;
 import org.apache.geode.internal.logging.LogService;
 import org.apache.geode.internal.monitoring.ThreadsMonitoring;
+import org.apache.geode.internal.statistics.StatisticsClock;
 
 public class ParallelAsyncEventQueueImpl extends AbstractGatewaySender {
 
   private static final Logger logger = LogService.getLogger();
 
-  public ParallelAsyncEventQueueImpl(InternalCache cache, GatewaySenderAttributes attrs) {
-    super(cache, attrs);
+  public ParallelAsyncEventQueueImpl(InternalCache cache, StatisticsFactory statisticsFactory,
+      StatisticsClock statisticsClock, GatewaySenderAttributes attrs) {
+    super(cache, statisticsClock, attrs);
     if (!(this.cache instanceof CacheCreation)) {
       // this sender lies underneath the AsyncEventQueue. Need to have
       // AsyncEventQueueStats
-      this.statistics = new AsyncEventQueueStats(cache.getDistributedSystem(),
-          AsyncEventQueueImpl.getAsyncEventQueueIdFromSenderId(id));
+      this.statistics = new AsyncEventQueueStats(statisticsFactory,
+          AsyncEventQueueImpl.getAsyncEventQueueIdFromSenderId(id), statisticsClock);
     }
     this.isForInternalUse = true;
   }
diff --git a/geode-core/src/main/java/org/apache/geode/cache/asyncqueue/internal/SerialAsyncEventQueueImpl.java b/geode-core/src/main/java/org/apache/geode/cache/asyncqueue/internal/SerialAsyncEventQueueImpl.java
index 259ae81..207c382 100644
--- a/geode-core/src/main/java/org/apache/geode/cache/asyncqueue/internal/SerialAsyncEventQueueImpl.java
+++ b/geode-core/src/main/java/org/apache/geode/cache/asyncqueue/internal/SerialAsyncEventQueueImpl.java
@@ -17,6 +17,7 @@ package org.apache.geode.cache.asyncqueue.internal;
 import org.apache.logging.log4j.Logger;
 
 import org.apache.geode.CancelException;
+import org.apache.geode.StatisticsFactory;
 import org.apache.geode.cache.asyncqueue.AsyncEventListener;
 import org.apache.geode.cache.wan.GatewayTransportFilter;
 import org.apache.geode.distributed.DistributedLockService;
@@ -40,18 +41,20 @@ import org.apache.geode.internal.cache.wan.serial.SerialGatewaySenderQueue;
 import org.apache.geode.internal.cache.xmlcache.CacheCreation;
 import org.apache.geode.internal.logging.LogService;
 import org.apache.geode.internal.monitoring.ThreadsMonitoring;
+import org.apache.geode.internal.statistics.StatisticsClock;
 
 public class SerialAsyncEventQueueImpl extends AbstractGatewaySender {
 
   private static final Logger logger = LogService.getLogger();
 
-  public SerialAsyncEventQueueImpl(InternalCache cache, GatewaySenderAttributes attrs) {
-    super(cache, attrs);
+  public SerialAsyncEventQueueImpl(InternalCache cache, StatisticsFactory statisticsFactory,
+      StatisticsClock statisticsClock, GatewaySenderAttributes attrs) {
+    super(cache, statisticsClock, attrs);
     if (!(this.cache instanceof CacheCreation)) {
       // this sender lies underneath the AsyncEventQueue. Need to have
       // AsyncEventQueueStats
-      this.statistics = new AsyncEventQueueStats(cache.getDistributedSystem(),
-          AsyncEventQueueImpl.getAsyncEventQueueIdFromSenderId(id));
+      this.statistics = new AsyncEventQueueStats(statisticsFactory,
+          AsyncEventQueueImpl.getAsyncEventQueueIdFromSenderId(id), statisticsClock);
     }
   }
 
diff --git a/geode-core/src/main/java/org/apache/geode/cache/client/internal/PoolImpl.java b/geode-core/src/main/java/org/apache/geode/cache/client/internal/PoolImpl.java
index d7777f2..8c51619 100644
--- a/geode-core/src/main/java/org/apache/geode/cache/client/internal/PoolImpl.java
+++ b/geode-core/src/main/java/org/apache/geode/cache/client/internal/PoolImpl.java
@@ -149,7 +149,7 @@ public class PoolImpl implements InternalPool {
 
   public static final int PRIMARY_QUEUE_NOT_AVAILABLE = -2;
   public static final int PRIMARY_QUEUE_TIMED_OUT = -1;
-  private AtomicInteger primaryQueueSize = new AtomicInteger(PRIMARY_QUEUE_NOT_AVAILABLE);
+  private final AtomicInteger primaryQueueSize = new AtomicInteger(PRIMARY_QUEUE_NOT_AVAILABLE);
 
   private final ThreadsMonitoring threadMonitoring;
 
@@ -216,7 +216,8 @@ public class PoolImpl implements InternalPool {
     subscriptionAckInterval = attributes.getSubscriptionAckInterval();
     subscriptionTimeoutMultiplier = attributes.getSubscriptionTimeoutMultiplier();
     if (subscriptionTimeoutMultiplier < 0) {
-      throw new IllegalArgumentException("The subscription timeout multipler must not be negative");
+      throw new IllegalArgumentException(
+          "The subscription timeout multiplier must not be negative");
     }
     serverGroup = attributes.getServerGroup();
     multiuserSecureModeEnabled = attributes.getMultiuserAuthentication();
diff --git a/geode-core/src/main/java/org/apache/geode/cache/client/internal/ProxyCache.java b/geode-core/src/main/java/org/apache/geode/cache/client/internal/ProxyCache.java
index 324176d..ebd7163 100755
--- a/geode-core/src/main/java/org/apache/geode/cache/client/internal/ProxyCache.java
+++ b/geode-core/src/main/java/org/apache/geode/cache/client/internal/ProxyCache.java
@@ -127,7 +127,7 @@ public class ProxyCache implements RegionService {
         throw new IllegalStateException(
             "Region's data-policy must be EMPTY when multiuser-authentication is true");
       }
-      return new ProxyRegion(this, this.cache.getRegion(path));
+      return new ProxyRegion(this, this.cache.getRegion(path), cache.getStatisticsClock());
     }
   }
 
@@ -209,7 +209,7 @@ public class ProxyCache implements RegionService {
     Set<Region<?, ?>> rootRegions = new HashSet<>();
     for (Region<?, ?> region : this.cache.rootRegions()) {
       if (!region.getAttributes().getDataPolicy().withStorage()) {
-        rootRegions.add(new ProxyRegion(this, region));
+        rootRegions.add(new ProxyRegion(this, region, cache.getStatisticsClock()));
       }
     }
     return Collections.unmodifiableSet(rootRegions);
diff --git a/geode-core/src/main/java/org/apache/geode/cache/client/internal/ProxyRegion.java b/geode-core/src/main/java/org/apache/geode/cache/client/internal/ProxyRegion.java
index e534e6b..c322531 100755
--- a/geode-core/src/main/java/org/apache/geode/cache/client/internal/ProxyRegion.java
+++ b/geode-core/src/main/java/org/apache/geode/cache/client/internal/ProxyRegion.java
@@ -44,6 +44,7 @@ import org.apache.geode.cache.query.SelectResults;
 import org.apache.geode.cache.query.TypeMismatchException;
 import org.apache.geode.cache.snapshot.RegionSnapshotService;
 import org.apache.geode.internal.cache.snapshot.RegionSnapshotServiceImpl;
+import org.apache.geode.internal.statistics.StatisticsClock;
 
 /**
  * A wrapper class over an actual Region instance. This is used when the multiuser-authentication
@@ -56,10 +57,12 @@ public class ProxyRegion implements Region {
 
   private final ProxyCache proxyCache;
   private final Region realRegion;
+  private final StatisticsClock statisticsClock;
 
-  public ProxyRegion(ProxyCache proxyCache, Region realRegion) {
+  public ProxyRegion(ProxyCache proxyCache, Region realRegion, StatisticsClock statisticsClock) {
     this.proxyCache = proxyCache;
     this.realRegion = realRegion;
+    this.statisticsClock = statisticsClock;
   }
 
   @Override
@@ -366,7 +369,7 @@ public class ProxyRegion implements Region {
   @Override
   public Region getSubregion(String path) {
     Region region = this.realRegion.getSubregion(path);
-    return region != null ? new ProxyRegion(this.proxyCache, region) : null;
+    return region != null ? new ProxyRegion(this.proxyCache, region, statisticsClock) : null;
   }
 
   @Override
diff --git a/geode-core/src/main/java/org/apache/geode/cache/query/internal/DefaultQuery.java b/geode-core/src/main/java/org/apache/geode/cache/query/internal/DefaultQuery.java
index 92b3be2..688337c 100644
--- a/geode-core/src/main/java/org/apache/geode/cache/query/internal/DefaultQuery.java
+++ b/geode-core/src/main/java/org/apache/geode/cache/query/internal/DefaultQuery.java
@@ -51,13 +51,13 @@ import org.apache.geode.cache.query.internal.cq.InternalCqQuery;
 import org.apache.geode.distributed.internal.DistributionConfig;
 import org.apache.geode.internal.NanoTimer;
 import org.apache.geode.internal.cache.BucketRegion;
-import org.apache.geode.internal.cache.CachePerfStats;
 import org.apache.geode.internal.cache.InternalCache;
 import org.apache.geode.internal.cache.LocalDataSet;
 import org.apache.geode.internal.cache.PRQueryProcessor;
 import org.apache.geode.internal.cache.PartitionedRegion;
 import org.apache.geode.internal.cache.TXManagerImpl;
 import org.apache.geode.internal.cache.TXStateProxy;
+import org.apache.geode.internal.statistics.StatisticsClock;
 
 /**
  * Thread-safe implementation of org.apache.persistence.query.Query
@@ -127,6 +127,7 @@ public class DefaultQuery implements Query {
   // to prevent objects from getting deserialized
   private boolean keepSerialized = false;
 
+  private final StatisticsClock statisticsClock;
 
   /**
    * Caches the fields not found in any Pdx version. This threadlocal will be cleaned up after query
@@ -175,6 +176,7 @@ public class DefaultQuery implements Query {
     }
     this.traceOn = compiler.isTraceRequested() || QUERY_VERBOSE;
     this.cache = cache;
+    statisticsClock = cache.getStatisticsClock();
     this.stats = new DefaultQueryStatistics();
   }
 
@@ -316,7 +318,7 @@ public class DefaultQuery implements Query {
   }
 
   private Object executeOnServer(Object[] parameters) {
-    long startTime = CachePerfStats.getStatTime();
+    long startTime = statisticsClock.getTime();
     Object result = null;
     try {
       if (this.proxyCache != null) {
@@ -328,7 +330,7 @@ public class DefaultQuery implements Query {
       result = this.serverProxy.query(this.queryString, parameters);
     } finally {
       UserAttributes.userAttributes.set(null);
-      long endTime = CachePerfStats.getStatTime();
+      long endTime = statisticsClock.getTime();
       updateStatistics(endTime - startTime);
     }
     return result;
@@ -409,7 +411,7 @@ public class DefaultQuery implements Query {
       TypeMismatchException, NameResolutionException, QueryInvocationTargetException {
     QueryObserver observer = QueryObserverHolder.getInstance();
 
-    long startTime = CachePerfStats.getStatTime();
+    long startTime = statisticsClock.getTime();
     TXStateProxy tx = ((TXManagerImpl) this.cache.getCacheTransactionManager()).pauseTransaction();
     try {
       observer.startQuery(this);
@@ -436,7 +438,7 @@ public class DefaultQuery implements Query {
       return results;
     } finally {
       observer.endQuery();
-      long endTime = CachePerfStats.getStatTime();
+      long endTime = statisticsClock.getTime();
       updateStatistics(endTime - startTime);
       pdxClassToFieldsMap.remove();
       pdxClassToMethodsMap.remove();
diff --git a/geode-core/src/main/java/org/apache/geode/cache/query/internal/QRegion.java b/geode-core/src/main/java/org/apache/geode/cache/query/internal/QRegion.java
index be87f41..2a30963 100644
--- a/geode-core/src/main/java/org/apache/geode/cache/query/internal/QRegion.java
+++ b/geode-core/src/main/java/org/apache/geode/cache/query/internal/QRegion.java
@@ -97,8 +97,9 @@ public class QRegion implements SelectResults {
 
     ResultsCollectionWrapper res = null;
     if (context.getBucketList() != null && region instanceof PartitionedRegion) {
+      PartitionedRegion partitionedRegion = (PartitionedRegion) region;
       LocalDataSet localData =
-          new LocalDataSet(((PartitionedRegion) region), new HashSet(context.getBucketList()));
+          new LocalDataSet(partitionedRegion, new HashSet(context.getBucketList()));
       this.region = localData;
       if (includeKeys) {
         res = new ResultsCollectionWrapper(TypeUtils.getObjectType(constraint),
diff --git a/geode-core/src/main/java/org/apache/geode/cache/query/internal/index/HashIndexSet.java b/geode-core/src/main/java/org/apache/geode/cache/query/internal/index/HashIndexSet.java
index 0f74a14..bbc0960 100755
--- a/geode-core/src/main/java/org/apache/geode/cache/query/internal/index/HashIndexSet.java
+++ b/geode-core/src/main/java/org/apache/geode/cache/query/internal/index/HashIndexSet.java
@@ -314,7 +314,7 @@ public class HashIndexSet implements Set {
 
     long start = -1L;
     if (this.cacheStats != null) {
-      start = this.cacheStats.getStatTime();
+      start = this.cacheStats.getTime();
       this.cacheStats.incQueryResultsHashCollisions();
     }
     try {
diff --git a/geode-core/src/main/java/org/apache/geode/cache/query/internal/index/IndexStats.java b/geode-core/src/main/java/org/apache/geode/cache/query/internal/index/IndexStats.java
index 3899587..8c8a586 100644
--- a/geode-core/src/main/java/org/apache/geode/cache/query/internal/index/IndexStats.java
+++ b/geode-core/src/main/java/org/apache/geode/cache/query/internal/index/IndexStats.java
@@ -20,7 +20,8 @@ import org.apache.geode.StatisticsFactory;
 import org.apache.geode.StatisticsType;
 import org.apache.geode.StatisticsTypeFactory;
 import org.apache.geode.annotations.Immutable;
-import org.apache.geode.internal.cache.CachePerfStats;
+import org.apache.geode.internal.statistics.StatisticsClock;
+import org.apache.geode.internal.statistics.StatisticsClockFactory;
 import org.apache.geode.internal.statistics.StatisticsTypeFactoryImpl;
 
 /**
@@ -46,6 +47,8 @@ public class IndexStats {
   /** The Statistics object that we delegate most behavior to */
   private final Statistics stats;
 
+  private final StatisticsClock clock;
+
   static {
     StatisticsTypeFactory f = StatisticsTypeFactoryImpl.singleton();
 
@@ -85,11 +88,16 @@ public class IndexStats {
   }
 
   /**
-   * Creates a new <code>CachePerfStats</code> and registers itself with the given statistics
+   * Creates a new <code>IndexStats</code> and registers itself with the given statistics
    * factory.
    */
   public IndexStats(StatisticsFactory factory, String indexName) {
+    this(factory, indexName, StatisticsClockFactory.clock());
+  }
+
+  private IndexStats(StatisticsFactory factory, String indexName, StatisticsClock clock) {
     stats = factory.createAtomicStatistics(type, indexName);
+    this.clock = clock;
   }
 
   public long getNumberOfKeys() {
@@ -109,11 +117,11 @@ public class IndexStats {
   }
 
   public long getTotalUpdateTime() {
-    return CachePerfStats.enableClockStats ? stats.getLong(updateTimeId) : 0;
+    return clock.isEnabled() ? stats.getLong(updateTimeId) : 0;
   }
 
   public long getUseTime() {
-    return CachePerfStats.enableClockStats ? stats.getLong(useTimeId) : 0;
+    return clock.isEnabled() ? stats.getLong(useTimeId) : 0;
   }
 
   public int getReadLockCount() {
@@ -149,7 +157,7 @@ public class IndexStats {
   }
 
   public void incUpdateTime(long delta) {
-    if (CachePerfStats.enableClockStats) {
+    if (clock.isEnabled()) {
       this.stats.incLong(updateTimeId, delta);
     }
   }
@@ -167,7 +175,7 @@ public class IndexStats {
   }
 
   public void incUseTime(long delta) {
-    if (CachePerfStats.enableClockStats) {
+    if (clock.isEnabled()) {
       this.stats.incLong(useTimeId, delta);
     }
   }
diff --git a/geode-core/src/main/java/org/apache/geode/internal/admin/remote/RemoteBridgeServer.java b/geode-core/src/main/java/org/apache/geode/internal/admin/remote/RemoteBridgeServer.java
index 382004b..55f1afa 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/admin/remote/RemoteBridgeServer.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/admin/remote/RemoteBridgeServer.java
@@ -43,6 +43,7 @@ import org.apache.geode.internal.cache.tier.sockets.ConnectionListener;
 import org.apache.geode.internal.cache.tier.sockets.ServerConnectionFactory;
 import org.apache.geode.internal.net.SocketCreator;
 import org.apache.geode.internal.security.SecurityService;
+import org.apache.geode.internal.statistics.StatisticsClock;
 
 /**
  * A remote (serializable) implementation of <code>BridgeServer</code> that is passed between
@@ -193,6 +194,11 @@ public class RemoteBridgeServer extends AbstractCacheServer
   }
 
   @Override
+  public StatisticsClock getStatisticsClock() {
+    throw new UnsupportedOperationException("Unsupported in RemoteBridgeServer");
+  }
+
+  @Override
   public ClientSession getClientSession(String durableClientId) {
     String s = "Cannot get a client session for a remote BridgeServer";
     throw new UnsupportedOperationException(s);
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/AbstractBucketRegionQueue.java b/geode-core/src/main/java/org/apache/geode/internal/cache/AbstractBucketRegionQueue.java
index 15fae18..5b237c0 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/AbstractBucketRegionQueue.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/AbstractBucketRegionQueue.java
@@ -42,6 +42,7 @@ import org.apache.geode.internal.cache.wan.parallel.ConcurrentParallelGatewaySen
 import org.apache.geode.internal.concurrent.ConcurrentHashSet;
 import org.apache.geode.internal.logging.LogService;
 import org.apache.geode.internal.offheap.OffHeapRegionEntryHelper;
+import org.apache.geode.internal.statistics.StatisticsClock;
 
 public abstract class AbstractBucketRegionQueue extends BucketRegion {
   protected static final Logger logger = LogService.getLogger();
@@ -68,8 +69,9 @@ public abstract class AbstractBucketRegionQueue extends BucketRegion {
   private final ConcurrentHashSet<Object> failedBatchRemovalMessageKeys = new ConcurrentHashSet<>();
 
   AbstractBucketRegionQueue(String regionName, RegionAttributes attrs, LocalRegion parentRegion,
-      InternalCache cache, InternalRegionArguments internalRegionArgs) {
-    super(regionName, attrs, parentRegion, cache, internalRegionArgs);
+      InternalCache cache, InternalRegionArguments internalRegionArgs,
+      StatisticsClock statisticsClock) {
+    super(regionName, attrs, parentRegion, cache, internalRegionArgs, statisticsClock);
     this.gatewaySenderStats =
         this.getPartitionedRegion().getParallelGatewaySender().getStatistics();
   }
@@ -348,7 +350,7 @@ public abstract class AbstractBucketRegionQueue extends BucketRegion {
     }
 
     boolean didPut = false;
-    long startPut = CachePerfStats.getStatTime();
+    long startPut = getStatisticsClock().getTime();
     // Value will always be an instanceof GatewaySenderEventImpl which
     // is never stored offheap so this EntryEventImpl values will never be off-heap.
     // So the value that ends up being stored in this region is a GatewaySenderEventImpl
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/AbstractRegion.java b/geode-core/src/main/java/org/apache/geode/internal/cache/AbstractRegion.java
index e1ebc1d..9e3141a 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/AbstractRegion.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/AbstractRegion.java
@@ -15,6 +15,7 @@
 package org.apache.geode.internal.cache;
 
 import static org.apache.geode.internal.cache.LocalRegion.InitializationLevel.ANY_INIT;
+import static org.apache.geode.internal.statistics.StatisticsClockFactory.disabledClock;
 
 import java.io.DataInput;
 import java.io.DataOutput;
@@ -94,6 +95,7 @@ import org.apache.geode.internal.cache.extension.ExtensionPoint;
 import org.apache.geode.internal.cache.extension.SimpleExtensionPoint;
 import org.apache.geode.internal.cache.snapshot.RegionSnapshotServiceImpl;
 import org.apache.geode.internal.logging.LogService;
+import org.apache.geode.internal.statistics.StatisticsClock;
 import org.apache.geode.internal.util.ArrayUtils;
 import org.apache.geode.pdx.internal.PeerTypeRegistration;
 
@@ -274,10 +276,14 @@ public abstract class AbstractRegion implements InternalRegion, AttributesMutato
 
   private final PoolFinder poolFinder;
 
+  private final StatisticsClock statisticsClock;
+
   /** Creates a new instance of AbstractRegion */
   protected AbstractRegion(InternalCache cache, RegionAttributes<?, ?> attrs, String regionName,
-      InternalRegionArguments internalRegionArgs, PoolFinder poolFinder) {
+      InternalRegionArguments internalRegionArgs, PoolFinder poolFinder,
+      StatisticsClock statisticsClock) {
     this.poolFinder = poolFinder;
+    this.statisticsClock = statisticsClock;
 
     this.cache = cache;
     serialNumber = DistributionAdvisor.createSerialNumber();
@@ -387,6 +393,7 @@ public abstract class AbstractRegion implements InternalRegion, AttributesMutato
 
   @VisibleForTesting
   AbstractRegion() {
+    statisticsClock = disabledClock();
     cache = null;
     serialNumber = 0;
     isPdxTypesRegion = false;
@@ -1867,6 +1874,17 @@ public abstract class AbstractRegion implements InternalRegion, AttributesMutato
     // nothing
   }
 
+  /**
+   * Only subclasses of {@code AbstractRegion} should use this supplier to acquire the
+   * {@code StatisticsClock}.
+   *
+   * <p>
+   * Please do not use this accessor from any class other than a Region.
+   */
+  protected StatisticsClock getStatisticsClock() {
+    return statisticsClock;
+  }
+
   protected interface PoolFinder {
     PoolImpl find(String poolName);
   }
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/AbstractUpdateOperation.java b/geode-core/src/main/java/org/apache/geode/internal/cache/AbstractUpdateOperation.java
index 96b03b0..fa1b83f 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/AbstractUpdateOperation.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/AbstractUpdateOperation.java
@@ -137,7 +137,7 @@ public abstract class AbstractUpdateOperation extends DistributedCacheOperation
         if (logger.isDebugEnabled()) {
           logger.debug("doPutOrCreate: attempting to update or create entry");
         }
-        final long startPut = CachePerfStats.getStatTime();
+        final long startPut = rgn.getCachePerfStats().getTime();
         final boolean isBucket = rgn.isUsedForPartitionedRegionBucket();
         if (isBucket) {
           BucketRegion br = (BucketRegion) rgn;
@@ -169,7 +169,7 @@ public abstract class AbstractUpdateOperation extends DistributedCacheOperation
       // from this message.
       if (doUpdate) {
         if (!ev.isLocalInvalid()) {
-          final long startPut = CachePerfStats.getStatTime();
+          final long startPut = rgn.getCachePerfStats().getTime();
           boolean overwriteDestroyed = ev.getOperation().isCreate();
           final boolean isBucket = rgn.isUsedForPartitionedRegionBucket();
           if (isBucket) {
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/BucketRegion.java b/geode-core/src/main/java/org/apache/geode/internal/cache/BucketRegion.java
index aa6b360..f4ced74 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/BucketRegion.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/BucketRegion.java
@@ -93,6 +93,7 @@ import org.apache.geode.internal.logging.log4j.LogMarker;
 import org.apache.geode.internal.offheap.annotations.Released;
 import org.apache.geode.internal.offheap.annotations.Retained;
 import org.apache.geode.internal.offheap.annotations.Unretained;
+import org.apache.geode.internal.statistics.StatisticsClock;
 
 
 /**
@@ -223,8 +224,9 @@ public class BucketRegion extends DistributedRegion implements Bucket {
   }
 
   public BucketRegion(String regionName, RegionAttributes attrs, LocalRegion parentRegion,
-      InternalCache cache, InternalRegionArguments internalRegionArgs) {
-    super(regionName, attrs, parentRegion, cache, internalRegionArgs);
+      InternalCache cache, InternalRegionArguments internalRegionArgs,
+      StatisticsClock statisticsClock) {
+    super(regionName, attrs, parentRegion, cache, internalRegionArgs, statisticsClock);
     if (PartitionedRegion.DISABLE_SECONDARY_BUCKET_ACK) {
       Assert.assertTrue(attrs.getScope().isDistributedNoAck());
     } else {
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/BucketRegionQueue.java b/geode-core/src/main/java/org/apache/geode/internal/cache/BucketRegionQueue.java
index cc85246..c25a6c9 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/BucketRegionQueue.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/BucketRegionQueue.java
@@ -48,6 +48,7 @@ import org.apache.geode.internal.concurrent.Atomics;
 import org.apache.geode.internal.logging.LogService;
 import org.apache.geode.internal.offheap.OffHeapRegionEntryHelper;
 import org.apache.geode.internal.offheap.annotations.Released;
+import org.apache.geode.internal.statistics.StatisticsClock;
 
 public class BucketRegionQueue extends AbstractBucketRegionQueue {
 
@@ -73,8 +74,9 @@ public class BucketRegionQueue extends AbstractBucketRegionQueue {
   private final AtomicLong latestAcknowledgedKey = new AtomicLong();
 
   public BucketRegionQueue(String regionName, RegionAttributes attrs, LocalRegion parentRegion,
-      InternalCache cache, InternalRegionArguments internalRegionArgs) {
-    super(regionName, attrs, parentRegion, cache, internalRegionArgs);
+      InternalCache cache, InternalRegionArguments internalRegionArgs,
+      StatisticsClock statisticsClock) {
+    super(regionName, attrs, parentRegion, cache, internalRegionArgs, statisticsClock);
     this.keySet();
     this.indexes = new ConcurrentHashMap<Object, Long>();
   }
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/CachePerfStats.java b/geode-core/src/main/java/org/apache/geode/internal/cache/CachePerfStats.java
index 83a55ce..b01eed8 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/CachePerfStats.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/CachePerfStats.java
@@ -14,28 +14,22 @@
  */
 package org.apache.geode.internal.cache;
 
-import java.util.function.LongSupplier;
-
 import org.apache.geode.StatisticDescriptor;
 import org.apache.geode.Statistics;
 import org.apache.geode.StatisticsFactory;
 import org.apache.geode.StatisticsType;
 import org.apache.geode.StatisticsTypeFactory;
 import org.apache.geode.annotations.Immutable;
-import org.apache.geode.annotations.VisibleForTesting;
-import org.apache.geode.annotations.internal.MakeNotStatic;
 import org.apache.geode.distributed.internal.PoolStatHelper;
 import org.apache.geode.distributed.internal.QueueStatHelper;
 import org.apache.geode.internal.NanoTimer;
+import org.apache.geode.internal.statistics.StatisticsClock;
 import org.apache.geode.internal.statistics.StatisticsTypeFactoryImpl;
 
 /**
  * CachePerfStats tracks statistics about Geode cache performance and usage.
  */
 public class CachePerfStats {
-  @MakeNotStatic
-  public static boolean enableClockStats;
-
   @Immutable
   private static final StatisticsType type;
 
@@ -622,52 +616,17 @@ public class CachePerfStats {
   /** The Statistics object that we delegate most behavior to */
   protected final Statistics stats;
 
-  private final LongSupplier clock;
+  private final StatisticsClock clock;
 
-  public CachePerfStats(StatisticsFactory factory) {
-    this(factory, "cachePerfStats");
-  }
-
-  @VisibleForTesting
-  public CachePerfStats(StatisticsFactory factory, LongSupplier clock) {
+  public CachePerfStats(StatisticsFactory factory, StatisticsClock clock) {
     this(factory, "cachePerfStats", clock);
   }
 
-  public CachePerfStats(StatisticsFactory factory, String textId) {
-    this(factory, textId, createClock());
-  }
-
-  CachePerfStats(StatisticsFactory factory, String textId, LongSupplier clock) {
-    this(createStatistics(factory, textId), clock);
-  }
-
-  private CachePerfStats(Statistics stats, LongSupplier clock) {
-    this.stats = stats;
+  public CachePerfStats(StatisticsFactory factory, String textId, StatisticsClock clock) {
+    stats = factory == null ? null : factory.createAtomicStatistics(type, textId);
     this.clock = clock;
   }
 
-  private static Statistics createStatistics(StatisticsFactory factory, String textId) {
-    if (factory == null) {
-      return null;
-    }
-    return factory.createAtomicStatistics(type, textId);
-  }
-
-  private static LongSupplier createClock() {
-    return enableClockStats ? NanoTimer::getTime : () -> 0;
-  }
-
-  /**
-   * Returns the current NanoTime or, if clock stats are disabled, zero.
-   *
-   * @since GemFire 5.0
-   * @deprecated Please use instance method {@link #getTime()} instead.
-   */
-  @Deprecated
-  public static long getStatTime() {
-    return enableClockStats ? NanoTimer.getTime() : 0;
-  }
-
   public static StatisticsType getStatisticsType() {
     return type;
   }
@@ -681,8 +640,8 @@ public class CachePerfStats {
     return stats;
   }
 
-  protected long getTime() {
-    return clock.getAsLong();
+  public long getTime() {
+    return clock.getTime();
   }
 
   public int getLoadsCompleted() {
@@ -887,7 +846,7 @@ public class CachePerfStats {
   }
 
   public void endCompression(long startTime, long startSize, long endSize) {
-    if (enableClockStats) {
+    if (clock.isEnabled()) {
       stats.incLong(compressionCompressTimeId, getTime() - startTime);
     }
     stats.incLong(compressionPreCompressedBytesId, startSize);
@@ -900,7 +859,7 @@ public class CachePerfStats {
   }
 
   public void endDecompression(long startTime) {
-    if (enableClockStats) {
+    if (clock.isEnabled()) {
       stats.incLong(compressionDecompressTimeId, getTime() - startTime);
     }
   }
@@ -937,7 +896,7 @@ public class CachePerfStats {
    * @param start the timestamp taken when the operation started
    */
   public void endNetload(long start) {
-    if (enableClockStats) {
+    if (clock.isEnabled()) {
       stats.incLong(netloadTimeId, getTime() - start);
     }
     stats.incInt(netloadsInProgressId, -1);
@@ -976,7 +935,7 @@ public class CachePerfStats {
    * @param start the timestamp taken when the operation started
    */
   public void endCacheWriterCall(long start) {
-    if (enableClockStats) {
+    if (clock.isEnabled()) {
       stats.incLong(cacheWriterCallTimeId, getTime() - start);
     }
     stats.incInt(cacheWriterCallsInProgressId, -1);
@@ -1001,7 +960,7 @@ public class CachePerfStats {
    * @since GemFire 3.5
    */
   public void endCacheListenerCall(long start) {
-    if (enableClockStats) {
+    if (clock.isEnabled()) {
       stats.incLong(cacheListenerCallTimeId, getTime() - start);
     }
     stats.incInt(cacheListenerCallsInProgressId, -1);
@@ -1024,7 +983,7 @@ public class CachePerfStats {
    * @param start the timestamp taken when the operation started
    */
   public void endGetInitialImage(long start) {
-    if (enableClockStats) {
+    if (clock.isEnabled()) {
       stats.incLong(getInitialImageTimeId, getTime() - start);
     }
     stats.incInt(getInitialImagesInProgressId, -1);
@@ -1035,7 +994,7 @@ public class CachePerfStats {
    * @param start the timestamp taken when the operation started
    */
   public void endNoGIIDone(long start) {
-    if (enableClockStats) {
+    if (clock.isEnabled()) {
       stats.incLong(getInitialImageTimeId, getTime() - start);
     }
     stats.incInt(getInitialImagesInProgressId, -1);
@@ -1112,7 +1071,7 @@ public class CachePerfStats {
    * @param start the timestamp taken when the operation started
    */
   public void endGet(long start, boolean miss) {
-    if (enableClockStats) {
+    if (clock.isEnabled()) {
       long delta = getTime() - start;
       stats.incLong(getTimeId, delta);
     }
@@ -1130,13 +1089,13 @@ public class CachePerfStats {
     long total = 0;
     if (isUpdate) {
       stats.incLong(updatesId, 1L);
-      if (enableClockStats) {
+      if (clock.isEnabled()) {
         total = getTime() - start;
         stats.incLong(updateTimeId, total);
       }
     } else {
       stats.incLong(putsId, 1L);
-      if (enableClockStats) {
+      if (clock.isEnabled()) {
         total = getTime() - start;
         stats.incLong(putTimeId, total);
       }
@@ -1146,19 +1105,19 @@ public class CachePerfStats {
 
   public void endPutAll(long start) {
     stats.incInt(putAllsId, 1);
-    if (enableClockStats)
+    if (clock.isEnabled())
       stats.incLong(putAllTimeId, getTime() - start);
   }
 
   public void endRemoveAll(long start) {
     stats.incInt(removeAllsId, 1);
-    if (enableClockStats)
+    if (clock.isEnabled())
       stats.incLong(removeAllTimeId, getTime() - start);
   }
 
   public void endQueryExecution(long executionTime) {
     stats.incInt(queryExecutionsId, 1);
-    if (enableClockStats) {
+    if (clock.isEnabled()) {
       stats.incLong(queryExecutionTimeId, executionTime);
     }
   }
@@ -1168,7 +1127,7 @@ public class CachePerfStats {
   }
 
   public void endQueryResultsHashCollisionProbe(long start) {
-    if (enableClockStats) {
+    if (clock.isEnabled()) {
       stats.incLong(queryResultsHashCollisionProbeTimeId, getTime() - start);
     }
   }
@@ -1252,7 +1211,7 @@ public class CachePerfStats {
 
   void endDeltaUpdate(long start) {
     stats.incInt(deltaUpdatesId, 1);
-    if (enableClockStats) {
+    if (clock.isEnabled()) {
       stats.incLong(deltaUpdatesTimeId, getTime() - start);
     }
   }
@@ -1263,7 +1222,7 @@ public class CachePerfStats {
 
   public void endDeltaPrepared(long start) {
     stats.incInt(deltasPreparedId, 1);
-    if (enableClockStats) {
+    if (clock.isEnabled()) {
       stats.incLong(deltasPreparedTimeId, getTime() - start);
     }
   }
@@ -1476,14 +1435,14 @@ public class CachePerfStats {
 
   public void endImport(long entryCount, long start) {
     stats.incLong(importedEntriesCountId, entryCount);
-    if (enableClockStats) {
+    if (clock.isEnabled()) {
       stats.incLong(importTimeId, getTime() - start);
     }
   }
 
   public void endExport(long entryCount, long start) {
     stats.incLong(exportedEntriesCountId, entryCount);
-    if (enableClockStats) {
+    if (clock.isEnabled()) {
       stats.incLong(exportTimeId, getTime() - start);
     }
   }
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/CacheServerImpl.java b/geode-core/src/main/java/org/apache/geode/internal/cache/CacheServerImpl.java
index 8531e01..be617c4 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/CacheServerImpl.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/CacheServerImpl.java
@@ -77,6 +77,7 @@ import org.apache.geode.internal.cache.tier.sockets.ServerConnectionFactory;
 import org.apache.geode.internal.logging.LogService;
 import org.apache.geode.internal.net.SocketCreator;
 import org.apache.geode.internal.security.SecurityService;
+import org.apache.geode.internal.statistics.StatisticsClock;
 import org.apache.geode.management.membership.ClientMembership;
 import org.apache.geode.management.membership.ClientMembershipListener;
 
@@ -93,10 +94,12 @@ public class CacheServerImpl extends AbstractCacheServer implements Distribution
   private static final int FORCE_LOAD_UPDATE_FREQUENCY = getInteger(
       DistributionConfig.GEMFIRE_PREFIX + "BridgeServer.FORCE_LOAD_UPDATE_FREQUENCY", 10);
 
+  static final String CACHE_SERVER_BIND_ADDRESS_NOT_AVAILABLE_EXCEPTION_MESSAGE =
+      "A cache server's bind address is only available if it has been started";
+
   private final SecurityService securityService;
 
-  public static final String CACHE_SERVER_BIND_ADDRESS_NOT_AVAILABLE_EXCEPTION_MESSAGE =
-      "A cache server's bind address is only available if it has been started";
+  private final StatisticsClock statisticsClock;
 
   private final AcceptorBuilder acceptorBuilder;
 
@@ -146,6 +149,7 @@ public class CacheServerImpl extends AbstractCacheServer implements Distribution
 
   CacheServerImpl(final InternalCache cache,
       final SecurityService securityService,
+      final StatisticsClock statisticsClock,
       final AcceptorBuilder acceptorBuilder,
       final boolean sendResourceEvents,
       final boolean includeMembershipGroups,
@@ -155,6 +159,7 @@ public class CacheServerImpl extends AbstractCacheServer implements Distribution
       final Function<DistributionAdvisee, CacheServerAdvisor> cacheServerAdvisorProvider) {
     super(cache);
     this.securityService = securityService;
+    this.statisticsClock = statisticsClock;
     this.acceptorBuilder = acceptorBuilder;
     this.sendResourceEvents = sendResourceEvents;
     this.includeMembershipGroups = includeMembershipGroups;
@@ -169,6 +174,11 @@ public class CacheServerImpl extends AbstractCacheServer implements Distribution
     return cache.getCancelCriterion();
   }
 
+  @Override
+  public StatisticsClock getStatisticsClock() {
+    return statisticsClock;
+  }
+
   /**
    * Checks to see whether or not this cache server is running. If so, an
    * {@link IllegalStateException} is thrown.
@@ -276,7 +286,6 @@ public class CacheServerImpl extends AbstractCacheServer implements Distribution
     return this.messageTimeToLive;
   }
 
-
   @Override
   public ClientSubscriptionConfig getClientSubscriptionConfig() {
     return this.clientSubscriptionConfig;
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/ColocationHelper.java b/geode-core/src/main/java/org/apache/geode/internal/cache/ColocationHelper.java
index ad580e0..fac58c1 100755
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/ColocationHelper.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/ColocationHelper.java
@@ -337,7 +337,8 @@ public class ColocationHelper {
       PartitionedRegion region, Set<Integer> bucketSet) {
     Map<String, LocalDataSet> colocatedLocalDataSets = new HashMap<String, LocalDataSet>();
     if (region.getColocatedWith() == null && (!region.isColocatedBy())) {
-      colocatedLocalDataSets.put(region.getFullPath(), new LocalDataSet(region, bucketSet));
+      colocatedLocalDataSets.put(region.getFullPath(),
+          new LocalDataSet(region, bucketSet));
       return colocatedLocalDataSets;
     }
     Map<String, PartitionedRegion> colocatedRegions =
@@ -346,7 +347,8 @@ public class ColocationHelper {
       colocatedLocalDataSets.put(colocatedRegion.getFullPath(),
           new LocalDataSet((PartitionedRegion) colocatedRegion, bucketSet));
     }
-    colocatedLocalDataSets.put(region.getFullPath(), new LocalDataSet(region, bucketSet));
+    colocatedLocalDataSets.put(region.getFullPath(),
+        new LocalDataSet(region, bucketSet));
     return colocatedLocalDataSets;
   }
 
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/DistTXState.java b/geode-core/src/main/java/org/apache/geode/internal/cache/DistTXState.java
index 944fcf7..15e0a90 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/DistTXState.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/DistTXState.java
@@ -44,6 +44,7 @@ import org.apache.geode.internal.cache.tx.DistTxEntryEvent;
 import org.apache.geode.internal.cache.tx.DistTxKeyInfo;
 import org.apache.geode.internal.cache.versions.RegionVersionVector;
 import org.apache.geode.internal.offheap.annotations.Released;
+import org.apache.geode.internal.statistics.StatisticsClock;
 
 /**
  * TxState on a data node VM
@@ -59,8 +60,9 @@ public class DistTXState extends TXState {
 
   private boolean updatingTxStateDuringPreCommit = false;
 
-  public DistTXState(TXStateProxy proxy, boolean onBehalfOfRemoteStub) {
-    super(proxy, onBehalfOfRemoteStub);
+  public DistTXState(TXStateProxy proxy, boolean onBehalfOfRemoteStub,
+      StatisticsClock statisticsClock) {
+    super(proxy, onBehalfOfRemoteStub, statisticsClock);
   }
 
   @Override
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/DistTXStateOnCoordinator.java b/geode-core/src/main/java/org/apache/geode/internal/cache/DistTXStateOnCoordinator.java
index a2cec45..2df073e 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/DistTXStateOnCoordinator.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/DistTXStateOnCoordinator.java
@@ -24,6 +24,7 @@ import org.apache.geode.cache.UnsupportedOperationInTransactionException;
 import org.apache.geode.distributed.internal.DistributionManager;
 import org.apache.geode.internal.cache.tier.sockets.VersionedObjectList;
 import org.apache.geode.internal.cache.tx.DistTxEntryEvent;
+import org.apache.geode.internal.statistics.StatisticsClock;
 
 /**
  * TxState on TX coordinator, created when coordinator is also a data node
@@ -38,8 +39,9 @@ public class DistTXStateOnCoordinator extends DistTXState implements DistTXCoord
   private boolean preCommitResponse = false;
   private boolean rollbackResponse = false;
 
-  public DistTXStateOnCoordinator(TXStateProxy proxy, boolean onBehalfOfRemoteStub) {
-    super(proxy, onBehalfOfRemoteStub);
+  public DistTXStateOnCoordinator(TXStateProxy proxy, boolean onBehalfOfRemoteStub,
+      StatisticsClock statisticsClock) {
+    super(proxy, onBehalfOfRemoteStub, statisticsClock);
     primaryTransactionalOperations = new ArrayList<DistTxEntryEvent>();
     secondaryTransactionalOperations = new ArrayList<DistTxEntryEvent>();
   }
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/DistTXStateProxyImpl.java b/geode-core/src/main/java/org/apache/geode/internal/cache/DistTXStateProxyImpl.java
index 59aeaa2..60016a9 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/DistTXStateProxyImpl.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/DistTXStateProxyImpl.java
@@ -18,19 +18,20 @@ import org.apache.logging.log4j.Logger;
 
 import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
 import org.apache.geode.internal.logging.LogService;
+import org.apache.geode.internal.statistics.StatisticsClock;
 
 public abstract class DistTXStateProxyImpl extends TXStateProxyImpl {
 
   protected static final Logger logger = LogService.getLogger();
 
   public DistTXStateProxyImpl(InternalCache cache, TXManagerImpl managerImpl, TXId id,
-      InternalDistributedMember clientMember) {
-    super(cache, managerImpl, id, clientMember);
+      InternalDistributedMember clientMember, StatisticsClock statisticsClock) {
+    super(cache, managerImpl, id, clientMember, statisticsClock);
   }
 
   public DistTXStateProxyImpl(InternalCache cache, TXManagerImpl managerImpl, TXId id,
-      boolean isjta) {
-    super(cache, managerImpl, id, isjta);
+      boolean isjta, StatisticsClock statisticsClock) {
+    super(cache, managerImpl, id, isjta, statisticsClock);
   }
 
   @Override
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/DistTXStateProxyImplOnCoordinator.java b/geode-core/src/main/java/org/apache/geode/internal/cache/DistTXStateProxyImplOnCoordinator.java
index a03d1e5..25f047e 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/DistTXStateProxyImplOnCoordinator.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/DistTXStateProxyImplOnCoordinator.java
@@ -35,6 +35,7 @@ import org.apache.geode.internal.cache.TXEntryState.DistTxThinEntryState;
 import org.apache.geode.internal.cache.tier.sockets.VersionedObjectList;
 import org.apache.geode.internal.cache.tx.DistClientTXStateStub;
 import org.apache.geode.internal.cache.tx.DistTxEntryEvent;
+import org.apache.geode.internal.statistics.StatisticsClock;
 
 public class DistTXStateProxyImplOnCoordinator extends DistTXStateProxyImpl {
 
@@ -52,13 +53,13 @@ public class DistTXStateProxyImplOnCoordinator extends DistTXStateProxyImpl {
   private HashMap<String, ArrayList<DistTxThinEntryState>> txEntryEventMap = null;
 
   public DistTXStateProxyImplOnCoordinator(InternalCache cache, TXManagerImpl managerImpl, TXId id,
-      InternalDistributedMember clientMember) {
-    super(cache, managerImpl, id, clientMember);
+      InternalDistributedMember clientMember, StatisticsClock statisticsClock) {
+    super(cache, managerImpl, id, clientMember, statisticsClock);
   }
 
   public DistTXStateProxyImplOnCoordinator(InternalCache cache, TXManagerImpl managerImpl, TXId id,
-      boolean isjta) {
-    super(cache, managerImpl, id, isjta);
+      boolean isjta, StatisticsClock statisticsClock) {
+    super(cache, managerImpl, id, isjta, statisticsClock);
   }
 
   /*
@@ -159,7 +160,7 @@ public class DistTXStateProxyImplOnCoordinator extends DistTXStateProxyImpl {
                 DistTXCoordinatorInterface newTxStub = null;
                 if (currentNode.equals(dm)) {
                   // [DISTTX] TODO add a test case for this condition?
-                  newTxStub = new DistTXStateOnCoordinator(this, false);
+                  newTxStub = new DistTXStateOnCoordinator(this, false, getStatisticsClock());
                 } else {
                   newTxStub = new DistPeerTXStateStub(this, dm, onBehalfOfClientMember);
                 }
@@ -316,7 +317,7 @@ public class DistTXStateProxyImplOnCoordinator extends DistTXStateProxyImpl {
     if (this.realDeal == null) {
       // assert (r != null);
       if (r == null) { // TODO: stop gap to get tests working
-        this.realDeal = new DistTXStateOnCoordinator(this, false);
+        this.realDeal = new DistTXStateOnCoordinator(this, false, getStatisticsClock());
         target = this.txMgr.getDM().getId();
       } else {
         // Code to keep going forward
@@ -334,7 +335,7 @@ public class DistTXStateProxyImplOnCoordinator extends DistTXStateProxyImpl {
         } else {
           // (r != null) code block above
           if (target == null || target.equals(this.txMgr.getDM().getId())) {
-            this.realDeal = new DistTXStateOnCoordinator(this, false);
+            this.realDeal = new DistTXStateOnCoordinator(this, false, getStatisticsClock());
           } else {
             this.realDeal = new DistPeerTXStateStub(this, target, onBehalfOfClientMember);
           }
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/DistTXStateProxyImplOnDatanode.java b/geode-core/src/main/java/org/apache/geode/internal/cache/DistTXStateProxyImplOnDatanode.java
index 1e12082..e48f2e5 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/DistTXStateProxyImplOnDatanode.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/DistTXStateProxyImplOnDatanode.java
@@ -22,6 +22,7 @@ import org.apache.geode.cache.UnsupportedOperationInTransactionException;
 import org.apache.geode.distributed.DistributedMember;
 import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
 import org.apache.geode.internal.cache.TXEntryState.DistTxThinEntryState;
+import org.apache.geode.internal.statistics.StatisticsClock;
 
 public class DistTXStateProxyImplOnDatanode extends DistTXStateProxyImpl {
 
@@ -29,19 +30,19 @@ public class DistTXStateProxyImplOnDatanode extends DistTXStateProxyImpl {
   private boolean preCommitResponse = false;
 
   public DistTXStateProxyImplOnDatanode(InternalCache cache, TXManagerImpl managerImpl, TXId id,
-      InternalDistributedMember clientMember) {
-    super(cache, managerImpl, id, clientMember);
+      InternalDistributedMember clientMember, StatisticsClock statisticsClock) {
+    super(cache, managerImpl, id, clientMember, statisticsClock);
   }
 
   public DistTXStateProxyImplOnDatanode(InternalCache cache, TXManagerImpl managerImpl, TXId id,
-      boolean isjta) {
-    super(cache, managerImpl, id, isjta);
+      boolean isjta, StatisticsClock statisticsClock) {
+    super(cache, managerImpl, id, isjta, statisticsClock);
   }
 
   @Override
   public TXStateInterface getRealDeal(KeyInfo key, InternalRegion r) {
     if (this.realDeal == null) {
-      this.realDeal = new DistTXState(this, false);
+      this.realDeal = new DistTXState(this, false, getStatisticsClock());
       if (r != null) {
         // wait for the region to be initialized fixes bug 44652
         r.waitOnInitialization(r.getInitializationLatchBeforeGetInitialImage());
@@ -60,7 +61,7 @@ public class DistTXStateProxyImplOnDatanode extends DistTXStateProxyImpl {
     assert t != null;
     if (this.realDeal == null) {
       this.target = t;
-      this.realDeal = new DistTXState(this, false);
+      this.realDeal = new DistTXState(this, false, getStatisticsClock());
       if (logger.isDebugEnabled()) {
         logger.debug("Built a new DistTXState: {} me:{}", this.realDeal,
             this.txMgr.getDM().getId());
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/DistributedRegion.java b/geode-core/src/main/java/org/apache/geode/internal/cache/DistributedRegion.java
index 814867c..91a81f1 100755
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/DistributedRegion.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/DistributedRegion.java
@@ -130,6 +130,7 @@ import org.apache.geode.internal.logging.LoggingThread;
 import org.apache.geode.internal.offheap.annotations.Released;
 import org.apache.geode.internal.offheap.annotations.Retained;
 import org.apache.geode.internal.sequencelog.RegionLogger;
+import org.apache.geode.internal.statistics.StatisticsClock;
 import org.apache.geode.internal.util.concurrent.StoppableCountDownLatch;
 
 @SuppressWarnings("deprecation")
@@ -197,8 +198,9 @@ public class DistributedRegion extends LocalRegion implements InternalDistribute
 
   /** Creates a new instance of DistributedRegion */
   protected DistributedRegion(String regionName, RegionAttributes attrs, LocalRegion parentRegion,
-      InternalCache cache, InternalRegionArguments internalRegionArgs) {
-    super(regionName, attrs, parentRegion, cache, internalRegionArgs);
+      InternalCache cache, InternalRegionArguments internalRegionArgs,
+      StatisticsClock statisticsClock) {
+    super(regionName, attrs, parentRegion, cache, internalRegionArgs, statisticsClock);
     initializationLatchAfterMemberTimeout =
         new StoppableCountDownLatch(getCancelCriterion(), 1);
     distAdvisor = createDistributionAdvisor(internalRegionArgs);
@@ -2359,7 +2361,7 @@ public class DistributedRegion extends LocalRegion implements InternalDistribute
     // Set eventId. Required for interested clients.
     event.setNewEventId(cache.getDistributedSystem());
 
-    long startPut = CachePerfStats.getStatTime();
+    long startPut = getStatisticsClock().getTime();
     validateKey(event.getKey());
     // this next step also distributes the object to other processes, if necessary
     try {
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/DummyCachePerfStats.java b/geode-core/src/main/java/org/apache/geode/internal/cache/DummyCachePerfStats.java
index c4aadfc..3e2439e 100755
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/DummyCachePerfStats.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/DummyCachePerfStats.java
@@ -14,6 +14,8 @@
  */
 package org.apache.geode.internal.cache;
 
+import static org.apache.geode.internal.statistics.StatisticsClockFactory.disabledClock;
+
 import org.apache.geode.Statistics;
 import org.apache.geode.distributed.internal.PoolStatHelper;
 
@@ -23,7 +25,7 @@ import org.apache.geode.distributed.internal.PoolStatHelper;
 public class DummyCachePerfStats extends CachePerfStats {
 
   DummyCachePerfStats() {
-    super(null);
+    super(null, disabledClock());
   }
 
   @Override
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/EntryEventImpl.java b/geode-core/src/main/java/org/apache/geode/internal/cache/EntryEventImpl.java
index ded1553..0dfa798 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/EntryEventImpl.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/EntryEventImpl.java
@@ -1831,7 +1831,7 @@ public class EntryEventImpl implements InternalEntryEvent, InternalCacheEvent,
       }
       boolean deltaBytesApplied = false;
       try {
-        long start = CachePerfStats.getStatTime();
+        long start = getRegion().getCachePerfStats().getTime();
         ((org.apache.geode.Delta) value)
             .fromDelta(new ByteArrayDataInput(getDeltaBytes()));
         getRegion().getCachePerfStats().endDeltaUpdate(start);
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/GemFireCacheImpl.java b/geode-core/src/main/java/org/apache/geode/internal/cache/GemFireCacheImpl.java
index 406837a..fed201f 100755
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/GemFireCacheImpl.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/GemFireCacheImpl.java
@@ -158,7 +158,6 @@ import org.apache.geode.distributed.DistributedMember;
 import org.apache.geode.distributed.DistributedSystem;
 import org.apache.geode.distributed.DistributedSystemDisconnectedException;
 import org.apache.geode.distributed.Locator;
-import org.apache.geode.distributed.internal.CacheTime;
 import org.apache.geode.distributed.internal.ClusterDistributionManager;
 import org.apache.geode.distributed.internal.DistributionAdvisee;
 import org.apache.geode.distributed.internal.DistributionAdvisor;
@@ -233,6 +232,9 @@ import org.apache.geode.internal.security.SecurityService;
 import org.apache.geode.internal.security.SecurityServiceFactory;
 import org.apache.geode.internal.sequencelog.SequenceLoggerImpl;
 import org.apache.geode.internal.shared.StringPrintWriter;
+import org.apache.geode.internal.statistics.StatisticsClock;
+import org.apache.geode.internal.statistics.StatisticsClockFactory;
+import org.apache.geode.internal.statistics.StatisticsClockSupplier;
 import org.apache.geode.internal.tcp.ConnectionTable;
 import org.apache.geode.internal.util.BlobHelper;
 import org.apache.geode.internal.util.concurrent.FutureResult;
@@ -260,7 +262,7 @@ import org.apache.geode.pdx.internal.TypeRegistry;
  */
 @SuppressWarnings("deprecation")
 public class GemFireCacheImpl implements InternalCache, InternalClientCache, HasCachePerfStats,
-    DistributionAdvisee, CacheTime {
+    DistributionAdvisee, StatisticsClockSupplier {
   private static final Logger logger = LogService.getLogger();
 
   /** The default number of seconds to wait for a distributed lock */
@@ -601,6 +603,8 @@ public class GemFireCacheImpl implements InternalCache, InternalClientCache, Has
   private final MeterRegistry meterRegistry;
   private final Set<MeterRegistry> meterSubregistries;
 
+  private final StatisticsClock statisticsClock;
+
   static {
     // this works around jdk bug 6427854, reported in ticket #44434
     String propertyName = "sun.nio.ch.bugLevel";
@@ -843,10 +847,11 @@ public class GemFireCacheImpl implements InternalCache, InternalClientCache, Has
       cqService = CqServiceProvider.create(this);
 
       // Create the CacheStatistics
-      CachePerfStats.enableClockStats = system.getConfig().getEnableTimeStatistics();
-      cachePerfStats = new CachePerfStats(internalDistributedSystem.getStatisticsManager());
+      statisticsClock = StatisticsClockFactory.clock(system.getConfig().getEnableTimeStatistics());
+      cachePerfStats = new CachePerfStats(
+          internalDistributedSystem.getStatisticsManager(), statisticsClock);
 
-      transactionManager = new TXManagerImpl(cachePerfStats, this);
+      transactionManager = new TXManagerImpl(cachePerfStats, this, statisticsClock);
       dm.addMembershipListener(transactionManager);
 
       creationDate = new Date();
@@ -2008,7 +2013,7 @@ public class GemFireCacheImpl implements InternalCache, InternalClientCache, Has
     synchronized (heapEvictorLock) {
       stopper.checkCancelInProgress(null);
       if (heapEvictor == null) {
-        heapEvictor = new HeapEvictor(this);
+        heapEvictor = new HeapEvictor(this, statisticsClock);
       }
       return heapEvictor;
     }
@@ -2018,7 +2023,7 @@ public class GemFireCacheImpl implements InternalCache, InternalClientCache, Has
     synchronized (offHeapEvictorLock) {
       stopper.checkCancelInProgress(null);
       if (offHeapEvictor == null) {
-        offHeapEvictor = new OffHeapEvictor(this);
+        offHeapEvictor = new OffHeapEvictor(this, statisticsClock);
       }
       return offHeapEvictor;
     }
@@ -2996,7 +3001,8 @@ public class GemFireCacheImpl implements InternalCache, InternalClientCache, Has
             if (internalRegionArgs.getInternalMetaRegion() != null) {
               region = internalRegionArgs.getInternalMetaRegion();
             } else if (isPartitionedRegion) {
-              region = new PartitionedRegion(name, attrs, null, this, internalRegionArgs);
+              region = new PartitionedRegion(name, attrs, null, this, internalRegionArgs,
+                  statisticsClock);
             } else {
               // Abstract region depends on the default pool existing so lazily initialize it
               // if necessary.
@@ -3004,9 +3010,11 @@ public class GemFireCacheImpl implements InternalCache, InternalClientCache, Has
                 determineDefaultPool();
               }
               if (attrs.getScope().isLocal()) {
-                region = new LocalRegion(name, attrs, null, this, internalRegionArgs);
+                region =
+                    new LocalRegion(name, attrs, null, this, internalRegionArgs, statisticsClock);
               } else {
-                region = new DistributedRegion(name, attrs, null, this, internalRegionArgs);
+                region = new DistributedRegion(name, attrs, null, this, internalRegionArgs,
+                    statisticsClock);
               }
             }
 
@@ -3750,7 +3758,8 @@ public class GemFireCacheImpl implements InternalCache, InternalClientCache, Has
     throwIfClient();
     stopper.checkCancelInProgress(null);
 
-    InternalCacheServer server = new ServerBuilder(this, securityService).createServer();
+    InternalCacheServer server = new ServerBuilder(this, securityService,
+        StatisticsClockFactory.disabledClock()).createServer();
     allCacheServers.add(server);
 
     sendAddCacheServerProfileMessage();
@@ -3842,8 +3851,9 @@ public class GemFireCacheImpl implements InternalCache, InternalClientCache, Has
     requireNonNull(gatewayReceiver.get(),
         "GatewayReceiver must be added before adding a server endpoint.");
 
-    InternalCacheServer receiverServer = new ServerBuilder(this, securityService)
-        .forGatewayReceiver(receiver).createServer();
+    InternalCacheServer receiverServer = new ServerBuilder(this, securityService,
+        StatisticsClockFactory.disabledClock())
+            .forGatewayReceiver(receiver).createServer();
     gatewayReceiverServer.set(receiverServer);
 
     sendAddCacheServerProfileMessage();
@@ -5351,4 +5361,15 @@ public class GemFireCacheImpl implements InternalCache, InternalClientCache, Has
     }
   }
 
+  /**
+   * Feature factories may use this supplier to acquire the {@code StatisticsClock} which is
+   * created by the Cache as configured by {@link DistributionConfig#getEnableTimeStatistics()}.
+   *
+   * <p>
+   * Please pass the {@code StatisticsClock} through constructors where possible instead of
+   * accessing it from this supplier.
+   */
+  public StatisticsClock getStatisticsClock() {
+    return statisticsClock;
+  }
 }
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/HARegion.java b/geode-core/src/main/java/org/apache/geode/internal/cache/HARegion.java
index a957691..9f2e989 100755
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/HARegion.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/HARegion.java
@@ -46,6 +46,7 @@ import org.apache.geode.internal.cache.tier.sockets.ClientProxyMembershipID;
 import org.apache.geode.internal.cache.tier.sockets.HAEventWrapper;
 import org.apache.geode.internal.logging.LogService;
 import org.apache.geode.internal.offheap.annotations.Released;
+import org.apache.geode.internal.statistics.StatisticsClock;
 
 /**
  * This region is being implemented to suppress distribution of puts and to allow localDestroys on
@@ -90,10 +91,11 @@ public class HARegion extends DistributedRegion {
   private volatile HARegionQueue owningQueue;
 
   private HARegion(String regionName, RegionAttributes attrs, LocalRegion parentRegion,
-      InternalCache cache) {
+      InternalCache cache, StatisticsClock statisticsClock) {
     super(regionName, attrs, parentRegion, cache,
         new InternalRegionArguments().setDestroyLockFlag(true).setRecreateFlag(false)
-            .setSnapshotInputStream(null).setImageTarget(null));
+            .setSnapshotInputStream(null).setImageTarget(null),
+        statisticsClock);
     this.haRegionStats = new DummyCachePerfStats();
   }
 
@@ -244,10 +246,10 @@ public class HARegion extends DistributedRegion {
    * @throws RegionExistsException if a region of the same name exists in the same Cache
    */
   public static HARegion getInstance(String regionName, InternalCache cache, HARegionQueue hrq,
-      RegionAttributes ra)
+      RegionAttributes ra, StatisticsClock statisticsClock)
       throws TimeoutException, RegionExistsException, IOException, ClassNotFoundException {
 
-    HARegion haRegion = new HARegion(regionName, ra, null, cache);
+    HARegion haRegion = new HARegion(regionName, ra, null, cache, statisticsClock);
     haRegion.setOwner(hrq);
     Region region = cache.createVMRegion(regionName, ra,
         new InternalRegionArguments().setInternalMetaRegion(haRegion).setDestroyLockFlag(true)
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/InternalCache.java b/geode-core/src/main/java/org/apache/geode/internal/cache/InternalCache.java
index c413f28..598c80f 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/InternalCache.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/InternalCache.java
@@ -66,6 +66,7 @@ import org.apache.geode.internal.cache.tier.sockets.ClientProxyMembershipID;
 import org.apache.geode.internal.logging.InternalLogWriter;
 import org.apache.geode.internal.offheap.MemoryAllocator;
 import org.apache.geode.internal.security.SecurityService;
+import org.apache.geode.internal.statistics.StatisticsClockSupplier;
 import org.apache.geode.management.internal.JmxManagerAdvisor;
 import org.apache.geode.management.internal.RestAgent;
 import org.apache.geode.pdx.PdxInstanceFactory;
@@ -78,7 +79,8 @@ import org.apache.geode.pdx.internal.TypeRegistry;
  * @see org.apache.geode.cache.Cache
  * @since GemFire 7.0
  */
-public interface InternalCache extends Cache, Extensible<Cache>, CacheTime, ReconnectableCache {
+public interface InternalCache extends Cache, Extensible<Cache>, CacheTime, ReconnectableCache,
+    StatisticsClockSupplier {
 
   InternalDistributedMember getMyId();
 
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/InternalCacheForClientAccess.java b/geode-core/src/main/java/org/apache/geode/internal/cache/InternalCacheForClientAccess.java
index 1900ebd..8388276 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/InternalCacheForClientAccess.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/InternalCacheForClientAccess.java
@@ -89,6 +89,7 @@ import org.apache.geode.internal.cache.tier.sockets.ClientProxyMembershipID;
 import org.apache.geode.internal.logging.InternalLogWriter;
 import org.apache.geode.internal.offheap.MemoryAllocator;
 import org.apache.geode.internal.security.SecurityService;
+import org.apache.geode.internal.statistics.StatisticsClock;
 import org.apache.geode.management.internal.JmxManagerAdvisor;
 import org.apache.geode.management.internal.RestAgent;
 import org.apache.geode.pdx.JSONFormatter;
@@ -1256,4 +1257,9 @@ public class InternalCacheForClientAccess implements InternalCache {
   public void saveCacheXmlForReconnect() {
     delegate.saveCacheXmlForReconnect();
   }
+
+  @Override
+  public StatisticsClock getStatisticsClock() {
+    return delegate.getStatisticsClock();
+  }
 }
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/InternalCacheServer.java b/geode-core/src/main/java/org/apache/geode/internal/cache/InternalCacheServer.java
index 90090ba..64e745c 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/InternalCacheServer.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/InternalCacheServer.java
@@ -26,6 +26,7 @@ import org.apache.geode.internal.cache.tier.sockets.ConnectionListener;
 import org.apache.geode.internal.cache.tier.sockets.ServerConnectionFactory;
 import org.apache.geode.internal.net.SocketCreator;
 import org.apache.geode.internal.security.SecurityService;
+import org.apache.geode.internal.statistics.StatisticsClock;
 
 public interface InternalCacheServer extends CacheServer {
 
@@ -52,4 +53,6 @@ public interface InternalCacheServer extends CacheServer {
   ClientHealthMonitorProvider getClientHealthMonitorProvider();
 
   String[] getCombinedGroups();
+
+  StatisticsClock getStatisticsClock();
 }
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/LocalDataSet.java b/geode-core/src/main/java/org/apache/geode/internal/cache/LocalDataSet.java
index 42b1ba5..84c1fbe 100755
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/LocalDataSet.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/LocalDataSet.java
@@ -64,7 +64,6 @@ import org.apache.geode.internal.cache.execute.InternalRegionFunctionContext;
 import org.apache.geode.internal.cache.snapshot.RegionSnapshotServiceImpl;
 import org.apache.geode.internal.logging.LogService;
 
-
 public class LocalDataSet implements Region, QueryExecutor {
 
   private static final Logger logger = LogService.getLogger();
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/LocalRegion.java b/geode-core/src/main/java/org/apache/geode/internal/cache/LocalRegion.java
index 49ca506..883c85d8 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/LocalRegion.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/LocalRegion.java
@@ -214,6 +214,7 @@ import org.apache.geode.internal.offheap.annotations.Released;
 import org.apache.geode.internal.offheap.annotations.Retained;
 import org.apache.geode.internal.offheap.annotations.Unretained;
 import org.apache.geode.internal.sequencelog.EntryLogger;
+import org.apache.geode.internal.statistics.StatisticsClock;
 import org.apache.geode.internal.util.concurrent.CopyOnWriteHashMap;
 import org.apache.geode.internal.util.concurrent.FutureResult;
 import org.apache.geode.internal.util.concurrent.StoppableCountDownLatch;
@@ -531,20 +532,24 @@ public class LocalRegion extends AbstractRegion implements LoaderHelperFactory,
   }
 
   protected LocalRegion(String regionName, RegionAttributes attrs, LocalRegion parentRegion,
-      InternalCache cache, InternalRegionArguments internalRegionArgs) throws DiskAccessException {
-    this(regionName, attrs, parentRegion, cache, internalRegionArgs, new LocalRegionDataView());
+      InternalCache cache, InternalRegionArguments internalRegionArgs,
+      StatisticsClock statisticsClock) throws DiskAccessException {
+    this(regionName, attrs, parentRegion, cache, internalRegionArgs, new LocalRegionDataView(),
+        statisticsClock);
   }
 
   protected LocalRegion(String regionName, RegionAttributes attrs, LocalRegion parentRegion,
       InternalCache cache, InternalRegionArguments internalRegionArgs,
-      InternalDataView internalDataView) throws DiskAccessException {
+      InternalDataView internalDataView, StatisticsClock statisticsClock)
+      throws DiskAccessException {
     this(regionName, attrs, parentRegion, cache, internalRegionArgs, internalDataView,
         RegionMapFactory::createVM, new DefaultServerRegionProxyConstructor(),
         new DefaultEntryEventFactory(), poolName -> (PoolImpl) PoolManager.find(poolName),
         (LocalRegion region) -> new RegionPerfStats(
             cache.getInternalDistributedSystem().getStatisticsManager(),
             "RegionStats-" + regionName, cache.getCachePerfStats(),
-            region, cache.getMeterRegistry()));
+            region, cache.getMeterRegistry(), statisticsClock),
+        statisticsClock);
   }
 
   @VisibleForTesting
@@ -553,9 +558,10 @@ public class LocalRegion extends AbstractRegion implements LoaderHelperFactory,
       InternalDataView internalDataView, RegionMapConstructor regionMapConstructor,
       ServerRegionProxyConstructor serverRegionProxyConstructor,
       EntryEventFactory entryEventFactory, PoolFinder poolFinder,
-      java.util.function.Function<LocalRegion, RegionPerfStats> regionPerfStatsFactory)
+      java.util.function.Function<LocalRegion, RegionPerfStats> regionPerfStatsFactory,
+      StatisticsClock statisticsClock)
       throws DiskAccessException {
-    super(cache, attrs, regionName, internalRegionArgs, poolFinder);
+    super(cache, attrs, regionName, internalRegionArgs, poolFinder, statisticsClock);
 
     this.regionMapConstructor = regionMapConstructor;
     this.entryEventFactory = entryEventFactory;
@@ -915,21 +921,21 @@ public class LocalRegion extends AbstractRegion implements LoaderHelperFactory,
               internalRegionArgs.setUserAttribute(pr.getUserAttribute());
               if (pr.isShadowPR()) {
                 newRegion = new BucketRegionQueue(subregionName, regionAttributes, this, cache,
-                    internalRegionArgs);
+                    internalRegionArgs, getStatisticsClock());
               } else {
                 newRegion = new BucketRegion(subregionName, regionAttributes, this, cache,
-                    internalRegionArgs);
+                    internalRegionArgs, getStatisticsClock());
               }
             } else if (regionAttributes.getPartitionAttributes() != null) {
               newRegion = new PartitionedRegion(subregionName, regionAttributes, this, cache,
-                  internalRegionArgs);
+                  internalRegionArgs, getStatisticsClock());
             } else {
               boolean local = regionAttributes.getScope().isLocal();
               newRegion = local
                   ? new LocalRegion(subregionName, regionAttributes, this, cache,
-                      internalRegionArgs)
+                      internalRegionArgs, getStatisticsClock())
                   : new DistributedRegion(subregionName, regionAttributes, this, cache,
-                      internalRegionArgs);
+                      internalRegionArgs, getStatisticsClock());
             }
             Object previousValue = subregions.putIfAbsent(subregionName, newRegion);
 
@@ -1036,7 +1042,7 @@ public class LocalRegion extends AbstractRegion implements LoaderHelperFactory,
   @Override
   public void create(Object key, Object value, Object aCallbackArgument)
       throws TimeoutException, EntryExistsException, CacheWriterException {
-    long startPut = CachePerfStats.getStatTime();
+    long startPut = getStatisticsClock().getTime();
     @Released
     EntryEventImpl event = newCreateEntryEvent(key, value, aCallbackArgument);
     try {
@@ -1608,7 +1614,7 @@ public class LocalRegion extends AbstractRegion implements LoaderHelperFactory,
   @Override
   public Object put(Object key, Object value, Object aCallbackArgument)
       throws TimeoutException, CacheWriterException {
-    long startPut = CachePerfStats.getStatTime();
+    long startPut = getStatisticsClock().getTime();
     @Released
     EntryEventImpl event = newUpdateEntryEvent(key, value, aCallbackArgument);
     try {
@@ -2817,7 +2823,7 @@ public class LocalRegion extends AbstractRegion implements LoaderHelperFactory,
     RegionEntry re = null;
     if (value != null && !isMemoryThresholdReachedForLoad()) {
 
-      long startPut = CachePerfStats.getStatTime();
+      long startPut = getStatisticsClock().getTime();
       validateKey(key);
       Operation op;
       if (isCreate) {
@@ -5046,7 +5052,7 @@ public class LocalRegion extends AbstractRegion implements LoaderHelperFactory,
       List<EntryEventImpl> pendingCallbacks, FilterRoutingInfo filterRoutingInfo,
       ClientProxyMembershipID bridgeContext, TXEntryState txEntryState, VersionTag versionTag,
       long tailKey) {
-    long startPut = CachePerfStats.getStatTime();
+    long startPut = getStatisticsClock().getTime();
     entries.txApplyPut(putOp, key, newValue, didDestroy, transactionId, event, eventId,
         aCallbackArgument, pendingCallbacks, filterRoutingInfo, bridgeContext, txEntryState,
         versionTag, tailKey);
@@ -5101,7 +5107,7 @@ public class LocalRegion extends AbstractRegion implements LoaderHelperFactory,
     EventID eventId = clientEvent.getEventId();
     Object theCallbackArg = callbackArg;
 
-    long startPut = CachePerfStats.getStatTime();
+    long startPut = getStatisticsClock().getTime();
 
     @Released
     final EntryEventImpl event =
@@ -5169,7 +5175,7 @@ public class LocalRegion extends AbstractRegion implements LoaderHelperFactory,
 
     EventID eventID = clientEvent.getEventId();
     Object theCallbackArg = callbackArg;
-    long startPut = CachePerfStats.getStatTime();
+    long startPut = getStatisticsClock().getTime();
 
     @Released
     final EntryEventImpl event = entryEventFactory.create(this, Operation.UPDATE, key,
@@ -5249,7 +5255,7 @@ public class LocalRegion extends AbstractRegion implements LoaderHelperFactory,
     }
 
     concurrencyConfigurationCheck(event.getVersionTag());
-    long startPut = CachePerfStats.getStatTime();
+    long startPut = getStatisticsClock().getTime();
 
     // Generate EventID as it is possible that client is a cache server
     // in hierarchical cache
@@ -8648,7 +8654,7 @@ public class LocalRegion extends AbstractRegion implements LoaderHelperFactory,
           // The following basicPutEntry needs to be done
           // even if we do not have storage so that the
           // correct events will be delivered to any callbacks we have.
-          long startPut = CachePerfStats.getStatTime();
+          long startPut = getStatisticsClock().getTime();
           validateKey(key);
 
           @Released
@@ -8747,7 +8753,7 @@ public class LocalRegion extends AbstractRegion implements LoaderHelperFactory,
       ClientProxyMembershipID memberId, EventID eventId, boolean skipCallbacks, Object callbackArg)
       throws TimeoutException, CacheWriterException {
 
-    long startPut = CachePerfStats.getStatTime();
+    long startPut = getStatisticsClock().getTime();
 
     @Released
     final EntryEventImpl event =
@@ -8783,7 +8789,7 @@ public class LocalRegion extends AbstractRegion implements LoaderHelperFactory,
       ArrayList<VersionTag> retryVersions, ClientProxyMembershipID memberId, EventID eventId,
       Object callbackArg) throws TimeoutException, CacheWriterException {
 
-    long startOp = CachePerfStats.getStatTime();
+    long startOp = getStatisticsClock().getTime();
 
     @Released
     final EntryEventImpl event =
@@ -8809,7 +8815,7 @@ public class LocalRegion extends AbstractRegion implements LoaderHelperFactory,
 
   // TODO: return value is never used
   public VersionedObjectList basicImportPutAll(Map map, boolean skipCallbacks) {
-    long startPut = CachePerfStats.getStatTime();
+    long startPut = getStatisticsClock().getTime();
 
     @Released
     EntryEventImpl event = entryEventFactory.create(this, Operation.PUTALL_CREATE, null, null, null,
@@ -8832,7 +8838,7 @@ public class LocalRegion extends AbstractRegion implements LoaderHelperFactory,
 
   @Override
   public void putAll(Map map, Object aCallbackArgument) {
-    long startPut = CachePerfStats.getStatTime();
+    long startPut = getStatisticsClock().getTime();
     final DistributedPutAllOperation putAllOp = newPutAllOperation(map, aCallbackArgument);
     if (putAllOp != null) {
       try {
@@ -8858,7 +8864,7 @@ public class LocalRegion extends AbstractRegion implements LoaderHelperFactory,
 
   @Override
   public void removeAll(Collection keys, Object aCallbackArgument) {
-    long startOp = CachePerfStats.getStatTime();
+    long startOp = getStatisticsClock().getTime();
     DistributedRemoveAllOperation operation = newRemoveAllOperation(keys, aCallbackArgument);
     if (operation != null) {
       try {
@@ -10362,7 +10368,7 @@ public class LocalRegion extends AbstractRegion implements LoaderHelperFactory,
    * @throws PartitionedRegionStorageException if the operation could not be completed.
    */
   public Object putIfAbsent(Object key, Object value, Object callbackArgument) {
-    long startPut = CachePerfStats.getStatTime();
+    long startPut = getStatisticsClock().getTime();
 
     checkIfConcurrentMapOpsAllowed();
     validateArguments(key, value, callbackArgument);
@@ -10470,7 +10476,7 @@ public class LocalRegion extends AbstractRegion implements LoaderHelperFactory,
       throw new NullPointerException();
     }
 
-    long startPut = CachePerfStats.getStatTime();
+    long startPut = getStatisticsClock().getTime();
     validateArguments(key, newValue, callbackArg);
     checkReadiness();
     checkForLimitedOrNoAccess();
@@ -10523,7 +10529,7 @@ public class LocalRegion extends AbstractRegion implements LoaderHelperFactory,
    * TODO: callbackArg is always null but this method is for callbacks??
    */
   private Object replaceWithCallbackArgument(Object key, Object value, Object callbackArg) {
-    long startPut = CachePerfStats.getStatTime();
+    long startPut = getStatisticsClock().getTime();
 
     checkIfConcurrentMapOpsAllowed();
 
@@ -10570,7 +10576,7 @@ public class LocalRegion extends AbstractRegion implements LoaderHelperFactory,
       throws TimeoutException, EntryExistsException, CacheWriterException {
 
     EventID eventId = clientEvent.getEventId();
-    long startPut = CachePerfStats.getStatTime();
+    long startPut = getStatisticsClock().getTime();
 
     @Released
     final EntryEventImpl event =
@@ -10646,7 +10652,7 @@ public class LocalRegion extends AbstractRegion implements LoaderHelperFactory,
       throws TimeoutException, EntryExistsException, CacheWriterException {
 
     EventID eventId = clientEvent.getEventId();
-    long startPut = CachePerfStats.getStatTime();
+    long startPut = getStatisticsClock().getTime();
 
     @Released
     final EntryEventImpl event =
@@ -10704,7 +10710,7 @@ public class LocalRegion extends AbstractRegion implements LoaderHelperFactory,
       throws TimeoutException, EntryExistsException, CacheWriterException {
 
     EventID eventId = clientEvent.getEventId();
-    long startPut = CachePerfStats.getStatTime();
+    long startPut = getStatisticsClock().getTime();
 
     @Released
     final EntryEventImpl event =
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/PartitionedRegion.java b/geode-core/src/main/java/org/apache/geode/internal/cache/PartitionedRegion.java
index 02e90d6..cb920bd 100755
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/PartitionedRegion.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/PartitionedRegion.java
@@ -246,6 +246,7 @@ import org.apache.geode.internal.offheap.annotations.Released;
 import org.apache.geode.internal.offheap.annotations.Unretained;
 import org.apache.geode.internal.sequencelog.RegionLogger;
 import org.apache.geode.internal.size.Sizeable;
+import org.apache.geode.internal.statistics.StatisticsClock;
 import org.apache.geode.internal.util.TransformUtils;
 import org.apache.geode.internal.util.concurrent.StoppableCountDownLatch;
 
@@ -545,7 +546,7 @@ public class PartitionedRegion extends LocalRegion
 
   @Override
   public boolean remove(Object key, Object value, Object callbackArg) {
-    final long startTime = PartitionedRegionStats.startTime();
+    final long startTime = prStats.getTime();
     try {
       return super.remove(key, value, callbackArg);
     } finally {
@@ -739,12 +740,14 @@ public class PartitionedRegion extends LocalRegion
    * and also by invoking Cache.createRegion(). (Cache.xml etc to be added)
    */
   public PartitionedRegion(String regionName, RegionAttributes regionAttributes,
-      LocalRegion parentRegion, InternalCache cache, InternalRegionArguments internalRegionArgs) {
+      LocalRegion parentRegion, InternalCache cache, InternalRegionArguments internalRegionArgs,
+      StatisticsClock statisticsClock) {
     super(regionName, regionAttributes, parentRegion, cache, internalRegionArgs,
-        new PartitionedRegionDataView());
+        new PartitionedRegionDataView(), statisticsClock);
 
     this.node = initializeNode();
-    this.prStats = new PartitionedRegionStats(cache.getDistributedSystem(), getFullPath());
+    this.prStats = new PartitionedRegionStats(cache.getDistributedSystem(), getFullPath(),
+        statisticsClock);
     this.regionIdentifier = getFullPath().replace('/', '#');
 
     if (logger.isDebugEnabled()) {
@@ -1327,7 +1330,8 @@ public class PartitionedRegion extends LocalRegion
   private void initializeDataStore(RegionAttributes ra) {
 
     this.dataStore =
-        PartitionedRegionDataStore.createDataStore(cache, this, ra.getPartitionAttributes());
+        PartitionedRegionDataStore.createDataStore(cache, this, ra.getPartitionAttributes(),
+            getStatisticsClock());
   }
 
   protected DistributedLockService getPartitionedRegionLockService() {
@@ -1668,7 +1672,7 @@ public class PartitionedRegion extends LocalRegion
   @Override
   protected Region.Entry<?, ?> nonTXGetEntry(KeyInfo keyInfo, boolean access,
       boolean allowTombstones) {
-    final long startTime = PartitionedRegionStats.startTime();
+    final long startTime = prStats.getTime();
     final Object key = keyInfo.getKey();
     try {
       int bucketId = keyInfo.getBucketId();
@@ -2148,7 +2152,7 @@ public class PartitionedRegion extends LocalRegion
   public boolean virtualPut(EntryEventImpl event, boolean ifNew, boolean ifOld,
       Object expectedOldValue, boolean requireOldValue, long lastModified,
       boolean overwriteDestroyed) throws TimeoutException, CacheWriterException {
-    final long startTime = PartitionedRegionStats.startTime();
+    final long startTime = prStats.getTime();
     boolean result = false;
     final DistributedPutAllOperation putAllOp_save = event.setPutAllOperation(null);
 
@@ -2319,7 +2323,7 @@ public class PartitionedRegion extends LocalRegion
       throw cache.getCacheClosedException("Cache is shutting down");
     }
 
-    final long startTime = PartitionedRegionStats.startTime();
+    final long startTime = prStats.getTime();
     // build all the msgs by bucketid
     HashMap prMsgMap = putAllOp.createPRMessages();
     PutAllPartialResult partialKeys = new PutAllPartialResult(putAllOp.putAllDataSize);
@@ -2411,7 +2415,7 @@ public class PartitionedRegion extends LocalRegion
       throw cache.getCacheClosedException("Cache is shutting down");
     }
 
-    final long startTime = PartitionedRegionStats.startTime();
+    final long startTime = prStats.getTime();
     // build all the msgs by bucketid
     HashMap<Integer, RemoveAllPRMessage> prMsgMap = op.createPRMessages();
     PutAllPartialResult partialKeys = new PutAllPartialResult(op.removeAllDataSize);
@@ -3426,7 +3430,7 @@ public class PartitionedRegion extends LocalRegion
     }
     // Potentially no storage assigned, start bucket creation, be careful of race
     // conditions
-    final long startTime = PartitionedRegionStats.startTime();
+    final long startTime = prStats.getTime();
     if (isDataStore()) {
       ret = this.redundancyProvider.createBucketAtomically(bucketId, size, false,
           partitionName);
@@ -3444,7 +3448,7 @@ public class PartitionedRegion extends LocalRegion
     Object obj = null;
     final Object key = keyInfo.getKey();
     final Object aCallbackArgument = keyInfo.getCallbackArg();
-    final long startTime = PartitionedRegionStats.startTime();
+    final long startTime = prStats.getTime();
     try {
       int bucketId = keyInfo.getBucketId();
       if (bucketId == KeyInfo.UNKNOWN_BUCKET) {
@@ -5227,7 +5231,7 @@ public class PartitionedRegion extends LocalRegion
       final Object expectedOldValue)
       throws TimeoutException, EntryNotFoundException, CacheWriterException {
 
-    final long startTime = PartitionedRegionStats.startTime();
+    final long startTime = prStats.getTime();
     try {
       if (event.getEventId() == null) {
         event.setNewEventId(this.cache.getDistributedSystem());
@@ -5714,7 +5718,7 @@ public class PartitionedRegion extends LocalRegion
 
   @Override
   public void basicInvalidate(EntryEventImpl event) throws EntryNotFoundException {
-    final long startTime = PartitionedRegionStats.startTime();
+    final long startTime = prStats.getTime();
     try {
       if (event.getEventId() == null) {
         event.setNewEventId(this.cache.getDistributedSystem());
@@ -6363,7 +6367,7 @@ public class PartitionedRegion extends LocalRegion
 
   @Override
   boolean nonTXContainsKey(KeyInfo keyInfo) {
-    final long startTime = PartitionedRegionStats.startTime();
+    final long startTime = prStats.getTime();
     boolean contains = false;
     try {
       int bucketId = keyInfo.getBucketId();
@@ -6537,7 +6541,7 @@ public class PartitionedRegion extends LocalRegion
     // checkClosed();
     checkReadiness();
     validateKey(key);
-    final long startTime = PartitionedRegionStats.startTime();
+    final long startTime = prStats.getTime();
     boolean containsValueForKey = false;
     try {
       containsValueForKey = getDataView().containsValueForKey(getKeyInfo(key), this);
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/PartitionedRegionDataStore.java b/geode-core/src/main/java/org/apache/geode/internal/cache/PartitionedRegionDataStore.java
index 88a3ea2..7aec9ff 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/PartitionedRegionDataStore.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/PartitionedRegionDataStore.java
@@ -14,6 +14,8 @@
  */
 package org.apache.geode.internal.cache;
 
+import static org.apache.geode.internal.statistics.StatisticsClockFactory.disabledClock;
+
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collection;
@@ -95,6 +97,7 @@ import org.apache.geode.internal.cache.wan.AbstractGatewaySenderEventProcessor;
 import org.apache.geode.internal.cache.wan.GatewaySenderEventImpl;
 import org.apache.geode.internal.cache.wan.parallel.ConcurrentParallelGatewaySenderQueue;
 import org.apache.geode.internal.logging.LogService;
+import org.apache.geode.internal.statistics.StatisticsClock;
 import org.apache.geode.internal.util.concurrent.StoppableReentrantReadWriteLock;
 import org.apache.geode.internal.util.concurrent.StoppableReentrantReadWriteLock.StoppableReadLock;
 import org.apache.geode.internal.util.concurrent.StoppableReentrantReadWriteLock.StoppableWriteLock;
@@ -162,6 +165,8 @@ public class PartitionedRegionDataStore implements HasCachePerfStats {
 
   private final Object keysOfInterestLock = new Object();
 
+  private final StatisticsClock statisticsClock;
+
   /**
    * Update an entry's last access time if a client is interested in the entry.
    */
@@ -171,6 +176,7 @@ public class PartitionedRegionDataStore implements HasCachePerfStats {
 
   // Only for testing
   PartitionedRegionDataStore() {
+    statisticsClock = disabledClock();
     this.bucketCreationLock = null;
     bucketStats = null;
     partitionedRegion = null;
@@ -185,7 +191,8 @@ public class PartitionedRegionDataStore implements HasCachePerfStats {
    *
    * @param pr PartitionedRegion associated with this DataStore.
    */
-  PartitionedRegionDataStore(final PartitionedRegion pr) {
+  PartitionedRegionDataStore(final PartitionedRegion pr, StatisticsClock statisticsClock) {
+    this.statisticsClock = statisticsClock;
     final int bucketCount = pr.getTotalNumberOfBuckets();
     this.localBucket2RegionMap = new ConcurrentHashMap<Integer, BucketRegion>(bucketCount);
     this.partitionedRegion = pr;
@@ -204,9 +211,8 @@ public class PartitionedRegionDataStore implements HasCachePerfStats {
     // this.bucketStats = new CachePerfStats(pr.getSystem(), "partition-" + pr.getName());
     this.bucketStats =
         new RegionPerfStats(pr.getCache().getInternalDistributedSystem().getStatisticsManager(),
-            "RegionStats-partition-" + pr.getName(), pr.getCachePerfStats(),
-            pr,
-            pr.getCache().getMeterRegistry());
+            "RegionStats-partition-" + pr.getName(), pr.getCachePerfStats(), pr,
+            pr.getCache().getMeterRegistry(), statisticsClock);
     this.keysOfInterest = new ConcurrentHashMap();
   }
 
@@ -216,9 +222,8 @@ public class PartitionedRegionDataStore implements HasCachePerfStats {
    * @return @throws PartitionedRegionException
    */
   static PartitionedRegionDataStore createDataStore(Cache cache, PartitionedRegion pr,
-      PartitionAttributes pa) throws PartitionedRegionException {
-    PartitionedRegionDataStore prd = new PartitionedRegionDataStore(pr);
-    return prd;
+      PartitionAttributes pa, StatisticsClock statisticsClock) throws PartitionedRegionException {
+    return new PartitionedRegionDataStore(pr, statisticsClock);
   }
 
   ConcurrentMap<Integer, BucketRegion> getLocalBucket2RegionMap() {
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/PartitionedRegionHelper.java b/geode-core/src/main/java/org/apache/geode/internal/cache/PartitionedRegionHelper.java
index b83c4fa..c27b21c 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/PartitionedRegionHelper.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/PartitionedRegionHelper.java
@@ -268,7 +268,8 @@ public class PartitionedRegionHelper {
       final HasCachePerfStats prMetaStatsHolder = new HasCachePerfStats() {
         @Override
         public CachePerfStats getCachePerfStats() {
-          return new CachePerfStats(cache.getDistributedSystem(), "RegionStats-partitionMetaData");
+          return new CachePerfStats(cache.getDistributedSystem(), "RegionStats-partitionMetaData",
+              cache.getStatisticsClock());
         }
       };
 
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/PartitionedRegionStats.java b/geode-core/src/main/java/org/apache/geode/internal/cache/PartitionedRegionStats.java
index 65d0fcf..cae8752 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/PartitionedRegionStats.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/PartitionedRegionStats.java
@@ -12,7 +12,6 @@
  * or implied. See the License for the specific language governing permissions and limitations under
  * the License.
  */
-
 package org.apache.geode.internal.cache;
 
 import java.util.Collections;
@@ -26,6 +25,7 @@ import org.apache.geode.StatisticsType;
 import org.apache.geode.StatisticsTypeFactory;
 import org.apache.geode.annotations.Immutable;
 import org.apache.geode.cache.Region;
+import org.apache.geode.internal.statistics.StatisticsClock;
 import org.apache.geode.internal.statistics.StatisticsTypeFactoryImpl;
 
 /**
@@ -534,6 +534,8 @@ public class PartitionedRegionStats {
 
   private final Statistics stats;
 
+  private final StatisticsClock clock;
+
   /**
    * Utility map for temporarily holding stat start times.
    * <p>
@@ -545,22 +547,16 @@ public class PartitionedRegionStats {
    */
   private final Map startTimeMap;
 
-  public static long startTime() {
-    return CachePerfStats.getStatTime();
-  }
-
-  public static long getStatTime() {
-    return CachePerfStats.getStatTime();
-  }
+  public PartitionedRegionStats(StatisticsFactory factory, String name, StatisticsClock clock) {
+    stats = factory.createAtomicStatistics(type, name);
 
-  public PartitionedRegionStats(StatisticsFactory factory, String name) {
-    this.stats = factory.createAtomicStatistics(type, name /* fixes bug 42343 */);
-
-    if (CachePerfStats.enableClockStats) {
-      this.startTimeMap = new ConcurrentHashMap();
+    if (clock.isEnabled()) {
+      startTimeMap = new ConcurrentHashMap();
     } else {
-      this.startTimeMap = Collections.EMPTY_MAP;
+      startTimeMap = Collections.emptyMap();
     }
+
+    this.clock = clock;
   }
 
   public void close() {
@@ -571,6 +567,10 @@ public class PartitionedRegionStats {
     return this.stats;
   }
 
+  public long getTime() {
+    return clock.getTime();
+  }
+
   // ------------------------------------------------------------------------
   // region op stats
   // ------------------------------------------------------------------------
@@ -608,8 +608,8 @@ public class PartitionedRegionStats {
   }
 
   public void endPut(long start, int numInc) {
-    if (CachePerfStats.enableClockStats) {
-      long delta = CachePerfStats.getStatTime() - start;
+    if (clock.isEnabled()) {
+      long delta = clock.getTime() - start;
       this.stats.incLong(putTimeId, delta);
     }
     this.stats.incLong(putsCompletedId, numInc);
@@ -620,8 +620,8 @@ public class PartitionedRegionStats {
    *
    */
   public void endPutAll(long start, int numInc) {
-    if (CachePerfStats.enableClockStats) {
-      long delta = CachePerfStats.getStatTime() - start;
+    if (clock.isEnabled()) {
+      long delta = clock.getTime() - start;
       this.stats.incLong(fieldId_PUTALL_TIME, delta);
       // this.putStatsHistogram.endOp(delta);
 
@@ -630,52 +630,52 @@ public class PartitionedRegionStats {
   }
 
   public void endRemoveAll(long start, int numInc) {
-    if (CachePerfStats.enableClockStats) {
-      long delta = CachePerfStats.getStatTime() - start;
+    if (clock.isEnabled()) {
+      long delta = clock.getTime() - start;
       this.stats.incLong(fieldId_REMOVE_ALL_TIME, delta);
     }
     this.stats.incLong(fieldId_REMOVE_ALLS_COMPLETED, numInc);
   }
 
   public void endCreate(long start, int numInc) {
-    if (CachePerfStats.enableClockStats) {
-      this.stats.incLong(createTimeId, CachePerfStats.getStatTime() - start);
+    if (clock.isEnabled()) {
+      this.stats.incLong(createTimeId, clock.getTime() - start);
     }
     this.stats.incLong(createsCompletedId, numInc);
   }
 
   public void endGet(long start, int numInc) {
-    if (CachePerfStats.enableClockStats) {
-      final long delta = CachePerfStats.getStatTime() - start;
+    if (clock.isEnabled()) {
+      final long delta = clock.getTime() - start;
       this.stats.incLong(getTimeId, delta);
     }
     this.stats.incLong(getsCompletedId, numInc);
   }
 
   public void endDestroy(long start) {
-    if (CachePerfStats.enableClockStats) {
-      this.stats.incLong(destroyTimeId, CachePerfStats.getStatTime() - start);
+    if (clock.isEnabled()) {
+      this.stats.incLong(destroyTimeId, clock.getTime() - start);
     }
     this.stats.incLong(destroysCompletedId, 1);
   }
 
   public void endInvalidate(long start) {
-    if (CachePerfStats.enableClockStats) {
-      this.stats.incLong(invalidateTimeId, CachePerfStats.getStatTime() - start);
+    if (clock.isEnabled()) {
+      this.stats.incLong(invalidateTimeId, clock.getTime() - start);
     }
     this.stats.incLong(invalidatesCompletedId, 1);
   }
 
   public void endContainsKey(long start, int numInc) {
-    if (CachePerfStats.enableClockStats) {
-      this.stats.incLong(containsKeyTimeId, CachePerfStats.getStatTime() - start);
+    if (clock.isEnabled()) {
+      this.stats.incLong(containsKeyTimeId, clock.getTime() - start);
     }
     this.stats.incLong(containsKeyCompletedId, numInc);
   }
 
   public void endContainsValueForKey(long start, int numInc) {
-    if (CachePerfStats.enableClockStats) {
-      this.stats.incLong(containsValueForKeyTimeId, CachePerfStats.getStatTime() - start);
+    if (clock.isEnabled()) {
+      this.stats.incLong(containsValueForKeyTimeId, clock.getTime() - start);
     }
     this.stats.incLong(containsValueForKeyCompletedId, numInc);
   }
@@ -746,12 +746,12 @@ public class PartitionedRegionStats {
 
   public long startPartitionMessageProcessing() {
     this.stats.incLong(partitionMessagesReceivedId, 1);
-    return startTime();
+    return getTime();
   }
 
   public void endPartitionMessagesProcessing(long start) {
-    if (CachePerfStats.enableClockStats) {
-      long delta = CachePerfStats.getStatTime() - start;
+    if (clock.isEnabled()) {
+      long delta = clock.getTime() - start;
       this.stats.incLong(partitionMessagesProcessingTimeId, delta);
     }
     this.stats.incLong(partitionMessagesProcessedId, 1);
@@ -843,34 +843,34 @@ public class PartitionedRegionStats {
 
   public long startVolunteering() {
     this.stats.incLong(volunteeringInProgressId, 1);
-    return CachePerfStats.getStatTime();
+    return clock.getTime();
   }
 
   public void endVolunteeringBecamePrimary(long start) {
-    long ts = CachePerfStats.getStatTime();
+    long ts = clock.getTime();
     this.stats.incLong(volunteeringInProgressId, -1);
     this.stats.incLong(volunteeringBecamePrimaryId, 1);
-    if (CachePerfStats.enableClockStats) {
+    if (clock.isEnabled()) {
       long time = ts - start;
       this.stats.incLong(volunteeringBecamePrimaryTimeId, time);
     }
   }
 
   public void endVolunteeringOtherPrimary(long start) {
-    long ts = CachePerfStats.getStatTime();
+    long ts = clock.getTime();
     this.stats.incLong(volunteeringInProgressId, -1);
     this.stats.incLong(volunteeringOtherPrimaryId, 1);
-    if (CachePerfStats.enableClockStats) {
+    if (clock.isEnabled()) {
       long time = ts - start;
       this.stats.incLong(volunteeringOtherPrimaryTimeId, time);
     }
   }
 
   public void endVolunteeringClosed(long start) {
-    long ts = CachePerfStats.getStatTime();
+    long ts = clock.getTime();
     this.stats.incLong(volunteeringInProgressId, -1);
     this.stats.incLong(volunteeringClosedId, 1);
-    if (CachePerfStats.enableClockStats) {
+    if (clock.isEnabled()) {
       long time = ts - start;
       this.stats.incLong(volunteeringClosedTimeId, time);
     }
@@ -938,7 +938,7 @@ public class PartitionedRegionStats {
 
   /** Put stat start time in holding map for later removal and use by caller */
   public void putStartTime(Object key, long startTime) {
-    if (CachePerfStats.enableClockStats) {
+    if (clock.isEnabled()) {
       this.startTimeMap.put(key, Long.valueOf(startTime));
     }
   }
@@ -963,8 +963,8 @@ public class PartitionedRegionStats {
    *
    */
   public void endGetEntry(long start, int numInc) {
-    if (CachePerfStats.enableClockStats) {
-      this.stats.incLong(getEntryTimeId, CachePerfStats.getStatTime() - start);
+    if (clock.isEnabled()) {
+      this.stats.incLong(getEntryTimeId, clock.getTime() - start);
     }
     this.stats.incLong(getEntriesCompletedId, numInc);
   }
@@ -974,13 +974,13 @@ public class PartitionedRegionStats {
   // ------------------------------------------------------------------------
   public long startRecovery() {
     this.stats.incLong(recoveriesInProgressId, 1);
-    return PartitionedRegionStats.getStatTime();
+    return clock.getTime();
   }
 
   public void endRecovery(long start) {
-    long ts = PartitionedRegionStats.getStatTime();
+    long ts = clock.getTime();
     this.stats.incLong(recoveriesInProgressId, -1);
-    if (CachePerfStats.enableClockStats) {
+    if (clock.isEnabled()) {
       this.stats.incLong(recoveriesTimeId, ts - start);
     }
     this.stats.incLong(recoveriesCompletedId, 1);
@@ -991,13 +991,13 @@ public class PartitionedRegionStats {
     if (isRebalance) {
       startRebalanceBucketCreate();
     }
-    return PartitionedRegionStats.getStatTime();
+    return clock.getTime();
   }
 
   public void endBucketCreate(long start, boolean success, boolean isRebalance) {
-    long ts = PartitionedRegionStats.getStatTime();
+    long ts = clock.getTime();
     this.stats.incLong(bucketCreatesInProgressId, -1);
-    if (CachePerfStats.enableClockStats) {
+    if (clock.isEnabled()) {
       this.stats.incLong(bucketCreateTimeId, ts - start);
     }
     if (success) {
@@ -1015,13 +1015,13 @@ public class PartitionedRegionStats {
     if (isRebalance) {
       startRebalancePrimaryTransfer();
     }
-    return PartitionedRegionStats.getStatTime();
+    return clock.getTime();
   }
 
   public void endPrimaryTransfer(long start, boolean success, boolean isRebalance) {
-    long ts = PartitionedRegionStats.getStatTime();
+    long ts = clock.getTime();
     this.stats.incLong(primaryTransfersInProgressId, -1);
-    if (CachePerfStats.enableClockStats) {
+    if (clock.isEnabled()) {
       this.stats.incLong(primaryTransferTimeId, ts - start);
     }
     if (success) {
@@ -1076,7 +1076,7 @@ public class PartitionedRegionStats {
 
   private void endRebalanceBucketCreate(long start, long end, boolean success) {
     this.stats.incLong(rebalanceBucketCreatesInProgressId, -1);
-    if (CachePerfStats.enableClockStats) {
+    if (clock.isEnabled()) {
       this.stats.incLong(rebalanceBucketCreateTimeId, end - start);
     }
     if (success) {
@@ -1092,7 +1092,7 @@ public class PartitionedRegionStats {
 
   private void endRebalancePrimaryTransfer(long start, long end, boolean success) {
     this.stats.incLong(rebalancePrimaryTransfersInProgressId, -1);
-    if (CachePerfStats.enableClockStats) {
+    if (clock.isEnabled()) {
       this.stats.incLong(rebalancePrimaryTransferTimeId, end - start);
     }
     if (success) {
@@ -1136,11 +1136,11 @@ public class PartitionedRegionStats {
 
   public long startApplyReplication() {
     stats.incLong(applyReplicationInProgressId, 1);
-    return CachePerfStats.getStatTime();
+    return clock.getTime();
   }
 
   public void endApplyReplication(long start) {
-    long delta = CachePerfStats.getStatTime() - start;
+    long delta = clock.getTime() - start;
     stats.incLong(applyReplicationInProgressId, -1);
     stats.incLong(applyReplicationCompletedId, 1);
     stats.incLong(applyReplicationTimeId, delta);
@@ -1148,11 +1148,11 @@ public class PartitionedRegionStats {
 
   public long startSendReplication() {
     stats.incLong(sendReplicationInProgressId, 1);
-    return CachePerfStats.getStatTime();
+    return clock.getTime();
   }
 
   public void endSendReplication(long start) {
-    long delta = CachePerfStats.getStatTime() - start;
+    long delta = clock.getTime() - start;
     stats.incLong(sendReplicationInProgressId, -1);
     stats.incLong(sendReplicationCompletedId, 1);
     stats.incLong(sendReplicationTimeId, delta);
@@ -1160,11 +1160,11 @@ public class PartitionedRegionStats {
 
   public long startPutRemote() {
     stats.incLong(putRemoteInProgressId, 1);
-    return CachePerfStats.getStatTime();
+    return clock.getTime();
   }
 
   public void endPutRemote(long start) {
-    long delta = CachePerfStats.getStatTime() - start;
+    long delta = clock.getTime() - start;
     stats.incLong(putRemoteInProgressId, -1);
     stats.incLong(putRemoteCompletedId, 1);
     stats.incLong(putRemoteTimeId, delta);
@@ -1172,11 +1172,11 @@ public class PartitionedRegionStats {
 
   public long startPutLocal() {
     stats.incLong(putLocalInProgressId, 1);
-    return CachePerfStats.getStatTime();
+    return clock.getTime();
   }
 
   public void endPutLocal(long start) {
-    long delta = CachePerfStats.getStatTime() - start;
+    long delta = clock.getTime() - start;
     stats.incLong(putLocalInProgressId, -1);
     stats.incLong(putLocalCompletedId, 1);
     stats.incLong(putLocalTimeId, delta);
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/PoolFactoryImpl.java b/geode-core/src/main/java/org/apache/geode/internal/cache/PoolFactoryImpl.java
index 323020c..b5b7bee 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/PoolFactoryImpl.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/PoolFactoryImpl.java
@@ -57,7 +57,7 @@ public class PoolFactoryImpl implements PoolFactory {
    */
   private PoolAttributes attributes = new PoolAttributes();
 
-  private List<HostAddress> locatorAddresses = new ArrayList<>();
+  private final List<HostAddress> locatorAddresses = new ArrayList<>();
 
   /**
    * The cache that created this factory
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/RegionPerfStats.java b/geode-core/src/main/java/org/apache/geode/internal/cache/RegionPerfStats.java
index ccc8a36..ca0756e 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/RegionPerfStats.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/RegionPerfStats.java
@@ -14,33 +14,32 @@
  */
 package org.apache.geode.internal.cache;
 
-import java.util.function.LongSupplier;
-
 import io.micrometer.core.instrument.Gauge;
 import io.micrometer.core.instrument.MeterRegistry;
 
 import org.apache.geode.StatisticsFactory;
 import org.apache.geode.annotations.VisibleForTesting;
-import org.apache.geode.internal.NanoTimer;
+import org.apache.geode.internal.statistics.StatisticsClock;
 
-class RegionPerfStats extends CachePerfStats {
+class RegionPerfStats extends CachePerfStats implements RegionStats {
 
   private final CachePerfStats cachePerfStats;
+  private final StatisticsClock clock;
   private final MeterRegistry meterRegistry;
   private final Gauge entriesGauge;
 
   RegionPerfStats(StatisticsFactory statisticsFactory, String textId, CachePerfStats cachePerfStats,
-      InternalRegion region,
-      MeterRegistry meterRegistry) {
-    this(statisticsFactory, textId, createClock(), cachePerfStats, region,
+      InternalRegion region, MeterRegistry meterRegistry, StatisticsClock clock) {
+    this(statisticsFactory, textId, clock, cachePerfStats, region,
         meterRegistry);
   }
 
   @VisibleForTesting
-  RegionPerfStats(StatisticsFactory statisticsFactory, String textId, LongSupplier clock,
-      CachePerfStats cachePerfStats,
-      InternalRegion region, MeterRegistry meterRegistry) {
+  RegionPerfStats(StatisticsFactory statisticsFactory, String textId, StatisticsClock clock,
+      CachePerfStats cachePerfStats, InternalRegion region,
+      MeterRegistry meterRegistry) {
     super(statisticsFactory, textId, clock);
+    this.clock = clock;
     this.cachePerfStats = cachePerfStats;
     this.meterRegistry = meterRegistry;
     entriesGauge = Gauge.builder("member.region.entries", region::getLocalSize)
@@ -52,10 +51,6 @@ class RegionPerfStats extends CachePerfStats {
     stats.setLongSupplier(entryCountId, region::getLocalSize);
   }
 
-  private static LongSupplier createClock() {
-    return enableClockStats ? NanoTimer::getTime : () -> 0;
-  }
-
   @Override
   protected void close() {
     meterRegistry.remove(entriesGauge);
@@ -122,22 +117,16 @@ class RegionPerfStats extends CachePerfStats {
     cachePerfStats.incQueuedEvents(inc);
   }
 
-  /**
-   * @return the timestamp that marks the start of the operation
-   */
   @Override
   public long startLoad() {
     stats.incInt(loadsInProgressId, 1);
     return cachePerfStats.startLoad();
   }
 
-  /**
-   * @param start the timestamp taken when the operation started
-   */
   @Override
   public void endLoad(long start) {
     // note that load times are used in health checks and
-    // should not be disabled by enableClockStats==false
+    // should not be disabled by clock.isEnabled()==false
 
     // don't use getStatTime so always enabled
     long ts = getTime();
@@ -149,9 +138,6 @@ class RegionPerfStats extends CachePerfStats {
     cachePerfStats.endLoad(start);
   }
 
-  /**
-   * @return the timestamp that marks the start of the operation
-   */
   @Override
   public long startNetload() {
     stats.incInt(netloadsInProgressId, 1);
@@ -159,12 +145,9 @@ class RegionPerfStats extends CachePerfStats {
     return getTime();
   }
 
-  /**
-   * @param start the timestamp taken when the operation started
-   */
   @Override
   public void endNetload(long start) {
-    if (enableClockStats) {
+    if (clock.isEnabled()) {
       stats.incLong(netloadTimeId, getTime() - start);
     }
     stats.incInt(netloadsInProgressId, -1);
@@ -172,22 +155,16 @@ class RegionPerfStats extends CachePerfStats {
     cachePerfStats.endNetload(start);
   }
 
-  /**
-   * @return the timestamp that marks the start of the operation
-   */
   @Override
   public long startNetsearch() {
     stats.incInt(netsearchesInProgressId, 1);
     return cachePerfStats.startNetsearch();
   }
 
-  /**
-   * @param start the timestamp taken when the operation started
-   */
   @Override
   public void endNetsearch(long start) {
     // note that netsearch is used in health checks and timings should
-    // not be disabled by enableClockStats==false
+    // not be disabled by clock.isEnabled()==false
 
     // don't use getStatTime so always enabled
     long ts = getTime();
@@ -197,9 +174,6 @@ class RegionPerfStats extends CachePerfStats {
     cachePerfStats.endNetsearch(start);
   }
 
-  /**
-   * @return the timestamp that marks the start of the operation
-   */
   @Override
   public long startCacheWriterCall() {
     stats.incInt(cacheWriterCallsInProgressId, 1);
@@ -207,12 +181,9 @@ class RegionPerfStats extends CachePerfStats {
     return getTime();
   }
 
-  /**
-   * @param start the timestamp taken when the operation started
-   */
   @Override
   public void endCacheWriterCall(long start) {
-    if (enableClockStats) {
+    if (clock.isEnabled()) {
       stats.incLong(cacheWriterCallTimeId, getTime() - start);
     }
     stats.incInt(cacheWriterCallsInProgressId, -1);
@@ -220,10 +191,6 @@ class RegionPerfStats extends CachePerfStats {
     cachePerfStats.endCacheWriterCall(start);
   }
 
-  /**
-   * @return the timestamp that marks the start of the operation
-   * @since GemFire 3.5
-   */
   @Override
   public long startCacheListenerCall() {
     stats.incInt(cacheListenerCallsInProgressId, 1);
@@ -231,13 +198,9 @@ class RegionPerfStats extends CachePerfStats {
     return getTime();
   }
 
-  /**
-   * @param start the timestamp taken when the operation started
-   * @since GemFire 3.5
-   */
   @Override
   public void endCacheListenerCall(long start) {
-    if (enableClockStats) {
+    if (clock.isEnabled()) {
       stats.incLong(cacheListenerCallTimeId, getTime() - start);
     }
     stats.incInt(cacheListenerCallsInProgressId, -1);
@@ -245,9 +208,6 @@ class RegionPerfStats extends CachePerfStats {
     cachePerfStats.endCacheListenerCall(start);
   }
 
-  /**
-   * @return the timestamp that marks the start of the operation
-   */
   @Override
   public long startGetInitialImage() {
     stats.incInt(getInitialImagesInProgressId, 1);
@@ -255,12 +215,9 @@ class RegionPerfStats extends CachePerfStats {
     return getTime();
   }
 
-  /**
-   * @param start the timestamp taken when the operation started
-   */
   @Override
   public void endGetInitialImage(long start) {
-    if (enableClockStats) {
+    if (clock.isEnabled()) {
       stats.incLong(getInitialImageTimeId, getTime() - start);
     }
     stats.incInt(getInitialImagesInProgressId, -1);
@@ -268,12 +225,9 @@ class RegionPerfStats extends CachePerfStats {
     cachePerfStats.endGetInitialImage(start);
   }
 
-  /**
-   * @param start the timestamp taken when the operation started
-   */
   @Override
   public void endNoGIIDone(long start) {
-    if (enableClockStats) {
+    if (clock.isEnabled()) {
       stats.incLong(getInitialImageTimeId, getTime() - start);
     }
     stats.incInt(getInitialImagesInProgressId, -1);
@@ -357,12 +311,9 @@ class RegionPerfStats extends CachePerfStats {
     cachePerfStats.incConflatedEventsCount();
   }
 
-  /**
-   * @param start the timestamp taken when the operation started
-   */
   @Override
   public void endGet(long start, boolean miss) {
-    if (enableClockStats) {
+    if (clock.isEnabled()) {
       long totalNanos = getTime() - start;
       stats.incLong(getTimeId, totalNanos);
     }
@@ -373,22 +324,18 @@ class RegionPerfStats extends CachePerfStats {
     cachePerfStats.endGet(start, miss);
   }
 
-  /**
-   * @param start the timestamp taken when the operation started
-   * @param isUpdate true if the put was an update (origin remote)
-   */
   @Override
   public long endPut(long start, boolean isUpdate) {
     long totalNanos = 0;
     if (isUpdate) {
       stats.incLong(updatesId, 1L);
-      if (enableClockStats) {
+      if (clock.isEnabled()) {
         totalNanos = getTime() - start;
         stats.incLong(updateTimeId, totalNanos);
       }
     } else {
       stats.incLong(putsId, 1L);
-      if (enableClockStats) {
+      if (clock.isEnabled()) {
         totalNanos = getTime() - start;
         stats.incLong(putTimeId, totalNanos);
       }
@@ -400,7 +347,7 @@ class RegionPerfStats extends CachePerfStats {
   @Override
   public void endPutAll(long start) {
     stats.incInt(putAllsId, 1);
-    if (enableClockStats) {
+    if (clock.isEnabled()) {
       stats.incLong(putAllTimeId, getTime() - start);
     }
     cachePerfStats.endPutAll(start);
@@ -409,7 +356,7 @@ class RegionPerfStats extends CachePerfStats {
   @Override
   public void endQueryExecution(long executionTime) {
     stats.incInt(queryExecutionsId, 1);
-    if (enableClockStats) {
+    if (clock.isEnabled()) {
       stats.incLong(queryExecutionTimeId, executionTime);
     }
     cachePerfStats.endQueryExecution(executionTime);
@@ -417,7 +364,7 @@ class RegionPerfStats extends CachePerfStats {
 
   @Override
   public void endQueryResultsHashCollisionProbe(long start) {
-    if (enableClockStats) {
+    if (clock.isEnabled()) {
       stats.incLong(queryResultsHashCollisionProbeTimeId, getTime() - start);
     }
     cachePerfStats.endQueryResultsHashCollisionProbe(start);
@@ -475,13 +422,13 @@ class RegionPerfStats extends CachePerfStats {
   }
 
   @Override
-  protected void incEventQueueThrottleTime(long nanos) {
+  public void incEventQueueThrottleTime(long nanos) {
     stats.incLong(eventQueueThrottleTimeId, nanos);
     cachePerfStats.incEventQueueThrottleTime(nanos);
   }
 
   @Override
-  protected void incEventThreads(int items) {
+  public void incEventThreads(int items) {
     stats.incInt(eventThreadsId, items);
     cachePerfStats.incEventThreads(items);
   }
@@ -560,7 +507,7 @@ class RegionPerfStats extends CachePerfStats {
   @Override
   public void endImport(long entryCount, long start) {
     stats.incLong(importedEntriesCountId, entryCount);
-    if (enableClockStats) {
+    if (clock.isEnabled()) {
       stats.incLong(importTimeId, getTime() - start);
     }
     cachePerfStats.endImport(entryCount, start);
@@ -569,7 +516,7 @@ class RegionPerfStats extends CachePerfStats {
   @Override
   public void endExport(long entryCount, long start) {
     stats.incLong(exportedEntriesCountId, entryCount);
-    if (enableClockStats) {
+    if (clock.isEnabled()) {
       stats.incLong(exportTimeId, getTime() - start);
     }
     cachePerfStats.endExport(entryCount, start);
@@ -584,7 +531,7 @@ class RegionPerfStats extends CachePerfStats {
 
   @Override
   public void endCompression(long startTime, long startSize, long endSize) {
-    if (enableClockStats) {
+    if (clock.isEnabled()) {
       long time = getTime() - startTime;
       stats.incLong(compressionCompressTimeId, time);
       cachePerfStats.stats.incLong(compressionCompressTimeId, time);
@@ -606,7 +553,7 @@ class RegionPerfStats extends CachePerfStats {
 
   @Override
   public void endDecompression(long startTime) {
-    if (enableClockStats) {
+    if (clock.isEnabled()) {
       long time = getTime() - startTime;
       stats.incLong(compressionDecompressTimeId, time);
       cachePerfStats.stats.incLong(compressionDecompressTimeId, time);
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/RegionStats.java b/geode-core/src/main/java/org/apache/geode/internal/cache/RegionStats.java
new file mode 100644
index 0000000..b0f9f06
--- /dev/null
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/RegionStats.java
@@ -0,0 +1,152 @@
+/*
+ * 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.geode.internal.cache;
+
+public interface RegionStats {
+
+  void incReliableQueuedOps(int inc);
+
+  void incReliableQueueSize(int inc);
+
+  void incReliableQueueMax(int inc);
+
+  void incReliableRegions(int inc);
+
+  void incReliableRegionsMissing(int inc);
+
+  void incReliableRegionsQueuing(int inc);
+
+  void incReliableRegionsMissingFullAccess(int inc);
+
+  void incReliableRegionsMissingLimitedAccess(int inc);
+
+  void incReliableRegionsMissingNoAccess(int inc);
+
+  void incQueuedEvents(int inc);
+
+  long startLoad();
+
+  void endLoad(long start);
+
+  long startNetload();
+
+  void endNetload(long start);
+
+  long startNetsearch();
+
+  void endNetsearch(long start);
+
+  long startCacheWriterCall();
+
+  void endCacheWriterCall(long start);
+
+  long startCacheListenerCall();
+
+  void endCacheListenerCall(long start);
+
+  long startGetInitialImage();
+
+  void endGetInitialImage(long start);
+
+  void endNoGIIDone(long start);
+
+  void incGetInitialImageKeysReceived();
+
+  long startIndexUpdate();
+
+  void endIndexUpdate(long start);
+
+  void incRegions(int inc);
+
+  void incPartitionedRegions(int inc);
+
+  void incDestroys();
+
+  void incCreates();
+
+  void incInvalidates();
+
+  void incTombstoneCount(int amount);
+
+  void incTombstoneGCCount();
+
+  void incClearTimeouts();
+
+  void incConflatedEventsCount();
+
+  void endGet(long start, boolean miss);
+
+  long endPut(long start, boolean isUpdate);
+
+  void endPutAll(long start);
+
+  void endQueryExecution(long executionTime);
+
+  void endQueryResultsHashCollisionProbe(long start);
+
+  void incQueryResultsHashCollisions();
+
+  void incTxConflictCheckTime(long delta);
+
+  void txSuccess(long opTime, long txLifeTime, int txChanges);
+
+  void txFailure(long opTime, long txLifeTime, int txChanges);
+
+  void txRollback(long opTime, long txLifeTime, int txChanges);
+
+  void incEventQueueSize(int items);
+
+  void incEventQueueThrottleCount(int items);
+
+  void incEventQueueThrottleTime(long nanos);
+
+  void incEventThreads(int items);
+
+  void incEntryCount(int delta);
+
+  void incRetries();
+
+  void incDiskTasksWaiting();
+
+  void decDiskTasksWaiting();
+
+  void decDiskTasksWaiting(int count);
+
+  void incEvictorJobsStarted();
+
+  void incEvictorJobsCompleted();
+
+  void incEvictorQueueSize(int delta);
+
+  void incEvictWorkTime(long delta);
+
+  void incClearCount();
+
+  void incPRQueryRetries();
+
+  void incMetaDataRefreshCount();
+
+  void endImport(long entryCount, long start);
+
+  void endExport(long entryCount, long start);
+
+  long startCompression();
+
+  void endCompression(long startTime, long startSize, long endSize);
+
+  long startDecompression();
+
+  void endDecompression(long startTime);
+}
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/ServerBuilder.java b/geode-core/src/main/java/org/apache/geode/internal/cache/ServerBuilder.java
index 13e859b..d0657e8 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/ServerBuilder.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/ServerBuilder.java
@@ -35,6 +35,7 @@ import org.apache.geode.internal.cache.tier.sockets.ClientHealthMonitor.ClientHe
 import org.apache.geode.internal.net.SocketCreator;
 import org.apache.geode.internal.security.SecurableCommunicationChannel;
 import org.apache.geode.internal.security.SecurityService;
+import org.apache.geode.internal.statistics.StatisticsClock;
 
 /**
  * Builds instances of {@link InternalCacheServer}.
@@ -43,6 +44,7 @@ class ServerBuilder implements ServerFactory {
 
   private final InternalCache cache;
   private final SecurityService securityService;
+  private final StatisticsClock statisticsClock;
 
   private boolean sendResourceEvents = true;
   private boolean includeMemberGroups = true;
@@ -58,9 +60,11 @@ class ServerBuilder implements ServerFactory {
 
   private List<GatewayTransportFilter> gatewayTransportFilters = Collections.emptyList();
 
-  ServerBuilder(InternalCache cache, SecurityService securityService) {
+  ServerBuilder(InternalCache cache, SecurityService securityService,
+      StatisticsClock statisticsClock) {
     this.cache = cache;
     this.securityService = securityService;
+    this.statisticsClock = statisticsClock;
   }
 
   /**
@@ -109,8 +113,8 @@ class ServerBuilder implements ServerFactory {
     acceptorBuilder.setIsGatewayReceiver(socketCreatorType.isGateway());
     acceptorBuilder.setGatewayTransportFilters(gatewayTransportFilters);
 
-    return new CacheServerImpl(cache, securityService, acceptorBuilder, sendResourceEvents,
-        includeMemberGroups, socketCreatorSupplier, cacheClientNotifierProvider,
+    return new CacheServerImpl(cache, securityService, statisticsClock, acceptorBuilder,
+        sendResourceEvents, includeMemberGroups, socketCreatorSupplier, cacheClientNotifierProvider,
         clientHealthMonitorProvider, cacheServerAdvisorProvider);
   }
 
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/TXManagerImpl.java b/geode-core/src/main/java/org/apache/geode/internal/cache/TXManagerImpl.java
index 7c96047..e545d9f 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/TXManagerImpl.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/TXManagerImpl.java
@@ -63,6 +63,7 @@ import org.apache.geode.internal.cache.tier.MessageType;
 import org.apache.geode.internal.cache.tier.sockets.Message;
 import org.apache.geode.internal.concurrent.ConcurrentHashSet;
 import org.apache.geode.internal.logging.LogService;
+import org.apache.geode.internal.statistics.StatisticsClock;
 import org.apache.geode.internal.util.concurrent.CustomEntryConcurrentHashMap;
 import org.apache.geode.internal.util.concurrent.CustomEntryConcurrentHashMap.HashEntry;
 import org.apache.geode.internal.util.concurrent.CustomEntryConcurrentHashMap.MapCallback;
@@ -182,11 +183,14 @@ public class TXManagerImpl implements CacheTransactionManager, MembershipListene
    */
   private int transactionTimeToLive;
 
+  private final StatisticsClock statisticsClock;
+
   /**
    * Constructor that implements the {@link CacheTransactionManager} interface. Only only one
    * instance per {@link org.apache.geode.cache.Cache}
    */
-  public TXManagerImpl(CachePerfStats cachePerfStats, InternalCache cache) {
+  public TXManagerImpl(CachePerfStats cachePerfStats, InternalCache cache,
+      StatisticsClock statisticsClock) {
     this.cache = cache;
     this.dm = ((InternalDistributedSystem) cache.getDistributedSystem()).getDistributionManager();
     this.distributionMgrId = this.dm.getDistributionManagerId();
@@ -199,6 +203,7 @@ public class TXManagerImpl implements CacheTransactionManager, MembershipListene
     this.transactionTimeToLive = Integer
         .getInteger(DistributionConfig.GEMFIRE_PREFIX + "cacheServer.transactionTimeToLive", 180);
     currentInstance = this;
+    this.statisticsClock = statisticsClock;
   }
 
   public static TXManagerImpl getCurrentInstanceForTest() {
@@ -352,9 +357,9 @@ public class TXManagerImpl implements CacheTransactionManager, MembershipListene
     TXId id = new TXId(this.distributionMgrId, this.uniqId.incrementAndGet());
     TXStateProxyImpl proxy = null;
     if (isDistributed()) {
-      proxy = new DistTXStateProxyImplOnCoordinator(cache, this, id, null);
+      proxy = new DistTXStateProxyImplOnCoordinator(cache, this, id, null, statisticsClock);
     } else {
-      proxy = new TXStateProxyImpl(cache, this, id, null);
+      proxy = new TXStateProxyImpl(cache, this, id, null, statisticsClock);
     }
     setTXState(proxy);
     if (logger.isDebugEnabled()) {
@@ -375,9 +380,9 @@ public class TXManagerImpl implements CacheTransactionManager, MembershipListene
     TXStateProxy newState = null;
 
     if (isDistributed()) {
-      newState = new DistTXStateProxyImplOnCoordinator(cache, this, id, true);
+      newState = new DistTXStateProxyImplOnCoordinator(cache, this, id, true, statisticsClock);
     } else {
-      newState = new TXStateProxyImpl(cache, this, id, true);
+      newState = new TXStateProxyImpl(cache, this, id, true, statisticsClock);
     }
     setTXState(newState);
     return newState;
@@ -419,7 +424,7 @@ public class TXManagerImpl implements CacheTransactionManager, MembershipListene
     tx.checkJTA(
         "Can not commit this transaction because it is enlisted with a JTA transaction, use the JTA manager to perform the commit.");
 
-    final long opStart = CachePerfStats.getStatTime();
+    final long opStart = statisticsClock.getTime();
     final long lifeTime = opStart - tx.getBeginTime();
     try {
       setTXState(null);
@@ -448,7 +453,7 @@ public class TXManagerImpl implements CacheTransactionManager, MembershipListene
   }
 
   void noteCommitFailure(long opStart, long lifeTime, TXStateInterface tx) {
-    long opEnd = CachePerfStats.getStatTime();
+    long opEnd = statisticsClock.getTime();
     this.cachePerfStats.txFailure(opEnd - opStart, lifeTime, tx.getChanges());
     TransactionListener[] listeners = getListeners();
     if (tx.isFireCallbacks() && listeners.length > 0) {
@@ -479,7 +484,7 @@ public class TXManagerImpl implements CacheTransactionManager, MembershipListene
   }
 
   void noteCommitSuccess(long opStart, long lifeTime, TXStateInterface tx) {
-    long opEnd = CachePerfStats.getStatTime();
+    long opEnd = statisticsClock.getTime();
     this.cachePerfStats.txSuccess(opEnd - opStart, lifeTime, tx.getChanges());
     TransactionListener[] listeners = getListeners();
     if (tx.isFireCallbacks() && listeners.length > 0) {
@@ -536,7 +541,7 @@ public class TXManagerImpl implements CacheTransactionManager, MembershipListene
     tx.checkJTA(
         "Can not rollback this transaction is enlisted with a JTA transaction, use the JTA manager to perform the rollback.");
 
-    final long opStart = CachePerfStats.getStatTime();
+    final long opStart = statisticsClock.getTime();
     final long lifeTime = opStart - tx.getBeginTime();
     setTXState(null);
     tx.rollback();
@@ -546,7 +551,7 @@ public class TXManagerImpl implements CacheTransactionManager, MembershipListene
   }
 
   void noteRollbackSuccess(long opStart, long lifeTime, TXStateInterface tx) {
-    long opEnd = CachePerfStats.getStatTime();
+    long opEnd = statisticsClock.getTime();
     this.cachePerfStats.txRollback(opEnd - opStart, lifeTime, tx.getChanges());
     TransactionListener[] listeners = getListeners();
     if (tx.isFireCallbacks() && listeners.length > 0) {
@@ -918,11 +923,13 @@ public class TXManagerImpl implements CacheTransactionManager, MembershipListene
         val = this.hostedTXStates.get(key);
         if (val == null && msg.canStartRemoteTransaction()) {
           if (msg.isTransactionDistributed()) {
-            val = new DistTXStateProxyImplOnDatanode(cache, this, key, msg.getTXOriginatorClient());
-            val.setLocalTXState(new DistTXState(val, true));
+            val = new DistTXStateProxyImplOnDatanode(cache, this, key, msg.getTXOriginatorClient(),
+                statisticsClock);
+            val.setLocalTXState(new DistTXState(val, true, statisticsClock));
           } else {
-            val = new TXStateProxyImpl(cache, this, key, msg.getTXOriginatorClient());
-            val.setLocalTXState(new TXState(val, true));
+            val = new TXStateProxyImpl(cache, this, key, msg.getTXOriginatorClient(),
+                statisticsClock);
+            val.setLocalTXState(new TXState(val, true, statisticsClock));
             val.setTarget(cache.getDistributedSystem().getDistributedMember());
           }
           this.hostedTXStates.put(key, val);
@@ -984,10 +991,10 @@ public class TXManagerImpl implements CacheTransactionManager, MembershipListene
           // TODO: Conditionally create object based on distributed or non-distributed tx mode
           if (msg instanceof TransactionMessage
               && ((TransactionMessage) msg).isTransactionDistributed()) {
-            val = new DistTXStateProxyImplOnDatanode(cache, this, key, memberId);
+            val = new DistTXStateProxyImplOnDatanode(cache, this, key, memberId, statisticsClock);
             // val.setLocalTXState(new DistTXState(val,true));
           } else {
-            val = new TXStateProxyImpl(cache, this, key, memberId);
+            val = new TXStateProxyImpl(cache, this, key, memberId, statisticsClock);
             // val.setLocalTXState(new TXState(val,true));
           }
           this.hostedTXStates.put(key, val);
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/TXState.java b/geode-core/src/main/java/org/apache/geode/internal/cache/TXState.java
index 1611da4..9ec621b 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/TXState.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/TXState.java
@@ -66,6 +66,7 @@ import org.apache.geode.internal.cache.tx.TransactionalOperation.ServerRegionOpe
 import org.apache.geode.internal.logging.LogService;
 import org.apache.geode.internal.offheap.annotations.Released;
 import org.apache.geode.internal.offheap.annotations.Retained;
+import org.apache.geode.internal.statistics.StatisticsClock;
 
 /**
  * TXState is the entity that tracks the transaction state on a per thread basis, noting changes to
@@ -151,13 +152,16 @@ public class TXState implements TXStateInterface {
 
   private volatile DistributedMember proxyServer;
 
-  public TXState(TXStateProxy proxy, boolean onBehalfOfRemoteStub) {
-    this(proxy, onBehalfOfRemoteStub, new SingleThreadJTAExecutor());
+  private final StatisticsClock statisticsClock;
+
+  public TXState(TXStateProxy proxy, boolean onBehalfOfRemoteStub,
+      StatisticsClock statisticsClock) {
+    this(proxy, onBehalfOfRemoteStub, new SingleThreadJTAExecutor(), statisticsClock);
   }
 
   public TXState(TXStateProxy proxy, boolean onBehalfOfRemoteStub,
-      SingleThreadJTAExecutor singleThreadJTAExecutor) {
-    this.beginTime = CachePerfStats.getStatTime();
+      SingleThreadJTAExecutor singleThreadJTAExecutor, StatisticsClock statisticsClock) {
+    this.beginTime = statisticsClock.getTime();
     this.regions = new IdentityHashMap<>();
 
     this.internalAfterConflictCheck = null;
@@ -170,6 +174,7 @@ public class TXState implements TXStateInterface {
     this.proxy = proxy;
     this.onBehalfOfRemoteStub = onBehalfOfRemoteStub;
     this.singleThreadJTAExecutor = singleThreadJTAExecutor;
+    this.statisticsClock = statisticsClock;
   }
 
   private boolean hasSeenEvent(EntryEventImpl event) {
@@ -350,14 +355,14 @@ public class TXState implements TXStateInterface {
       return;
     }
 
-    final long conflictStart = CachePerfStats.getStatTime();
+    final long conflictStart = statisticsClock.getTime();
     this.locks = createLockRequest();
     this.locks.obtain(getCache().getInternalDistributedSystem());
     // for now check account the dlock service time
     // later this stat end should be moved to a finally block
-    if (CachePerfStats.enableClockStats)
+    if (statisticsClock.isEnabled())
       this.proxy.getTxMgr().getCachePerfStats()
-          .incTxConflictCheckTime(CachePerfStats.getStatTime() - conflictStart);
+          .incTxConflictCheckTime(statisticsClock.getTime() - conflictStart);
     if (this.internalAfterReservation != null) {
       this.internalAfterReservation.run();
     }
@@ -883,15 +888,15 @@ public class TXState implements TXStateInterface {
       this.seenResults.clear();
       freePendingCallbacks();
       if (this.locks != null) {
-        final long conflictStart = CachePerfStats.getStatTime();
+        final long conflictStart = statisticsClock.getTime();
         try {
           this.locks.cleanup(getCache().getInternalDistributedSystem());
         } catch (IllegalArgumentException | IllegalMonitorStateException e) {
           exception = e;
         }
-        if (CachePerfStats.enableClockStats)
+        if (statisticsClock.isEnabled())
           this.proxy.getTxMgr().getCachePerfStats()
-              .incTxConflictCheckTime(CachePerfStats.getStatTime() - conflictStart);
+              .incTxConflictCheckTime(statisticsClock.getTime() - conflictStart);
       }
       Iterator<Map.Entry<InternalRegion, TXRegionState>> it = this.regions.entrySet().iterator();
       while (it.hasNext()) {
@@ -1047,7 +1052,7 @@ public class TXState implements TXStateInterface {
   }
 
   void doBeforeCompletion() {
-    final long opStart = CachePerfStats.getStatTime();
+    final long opStart = statisticsClock.getTime();
     this.jtaLifeTime = opStart - getBeginTime();
 
     try {
@@ -1128,7 +1133,7 @@ public class TXState implements TXStateInterface {
   }
 
   void doAfterCompletionCommit() {
-    final long opStart = CachePerfStats.getStatTime();
+    final long opStart = statisticsClock.getTime();
     try {
       Assert.assertTrue(this.locks != null,
           "Gemfire Transaction afterCompletion called with illegal state.");
@@ -1148,7 +1153,7 @@ public class TXState implements TXStateInterface {
   }
 
   void doAfterCompletionRollback() {
-    final long opStart = CachePerfStats.getStatTime();
+    final long opStart = statisticsClock.getTime();
     this.jtaLifeTime = opStart - getBeginTime();
     try {
       rollback();
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/TXStateProxyImpl.java b/geode-core/src/main/java/org/apache/geode/internal/cache/TXStateProxyImpl.java
index 05e1956..86ac1a6 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/TXStateProxyImpl.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/TXStateProxyImpl.java
@@ -44,6 +44,7 @@ import org.apache.geode.internal.cache.tx.ClientTXStateStub;
 import org.apache.geode.internal.cache.tx.TransactionalOperation.ServerRegionOperation;
 import org.apache.geode.internal.lang.SystemPropertyHelper;
 import org.apache.geode.internal.logging.LogService;
+import org.apache.geode.internal.statistics.StatisticsClock;
 
 public class TXStateProxyImpl implements TXStateProxy {
   private static final Logger logger = LogService.getLogger();
@@ -77,21 +78,25 @@ public class TXStateProxyImpl implements TXStateProxy {
 
   private final InternalCache cache;
   private long lastOperationTimeFromClient;
+  private final StatisticsClock statisticsClock;
 
   public TXStateProxyImpl(InternalCache cache, TXManagerImpl managerImpl, TXId id,
-      InternalDistributedMember clientMember) {
+      InternalDistributedMember clientMember, StatisticsClock statisticsClock) {
     this.cache = cache;
     this.txMgr = managerImpl;
     this.txId = id;
     this.isJTA = false;
     this.onBehalfOfClientMember = clientMember;
+    this.statisticsClock = statisticsClock;
   }
 
-  public TXStateProxyImpl(InternalCache cache, TXManagerImpl managerImpl, TXId id, boolean isjta) {
+  public TXStateProxyImpl(InternalCache cache, TXManagerImpl managerImpl, TXId id, boolean isjta,
+      StatisticsClock statisticsClock) {
     this.cache = cache;
     this.txMgr = managerImpl;
     this.txId = id;
     this.isJTA = isjta;
+    this.statisticsClock = statisticsClock;
   }
 
   @Override
@@ -99,6 +104,10 @@ public class TXStateProxyImpl implements TXStateProxy {
     return this.lock;
   }
 
+  protected StatisticsClock getStatisticsClock() {
+    return statisticsClock;
+  }
+
   boolean isJTA() {
     return isJTA;
   }
@@ -124,7 +133,7 @@ public class TXStateProxyImpl implements TXStateProxy {
   public TXStateInterface getRealDeal(KeyInfo key, InternalRegion r) {
     if (this.realDeal == null) {
       if (r == null) { // TODO: stop gap to get tests working
-        this.realDeal = new TXState(this, false);
+        this.realDeal = new TXState(this, false, statisticsClock);
       } else {
         // Code to keep going forward
         if (r.hasServerProxy()) {
@@ -143,7 +152,7 @@ public class TXStateProxyImpl implements TXStateProxy {
           r.waitOnInitialization(r.getInitializationLatchBeforeGetInitialImage());
           target = r.getOwnerForKey(key);
           if (target == null || target.equals(this.txMgr.getDM().getId())) {
-            this.realDeal = new TXState(this, false);
+            this.realDeal = new TXState(this, false, statisticsClock);
           } else {
             this.realDeal = new PeerTXStateStub(this, target, onBehalfOfClientMember);
           }
@@ -161,7 +170,7 @@ public class TXStateProxyImpl implements TXStateProxy {
     if (this.realDeal == null) {
       this.target = t;
       if (target.equals(getCache().getDistributedSystem().getDistributedMember())) {
-        this.realDeal = new TXState(this, false);
+        this.realDeal = new TXState(this, false, statisticsClock);
       } else {
         /*
          * txtodo: // what to do!! We don't know if this is client or server!!!
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/eviction/HeapEvictor.java b/geode-core/src/main/java/org/apache/geode/internal/cache/eviction/HeapEvictor.java
index 89ff04e..5b350b6 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/eviction/HeapEvictor.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/eviction/HeapEvictor.java
@@ -42,6 +42,7 @@ import org.apache.geode.internal.cache.control.MemoryEvent;
 import org.apache.geode.internal.cache.control.ResourceListener;
 import org.apache.geode.internal.logging.LogService;
 import org.apache.geode.internal.logging.LoggingExecutors;
+import org.apache.geode.internal.statistics.StatisticsClock;
 
 /**
  * Triggers centralized eviction(asynchronously) when the ResourceManager sends an eviction event
@@ -87,6 +88,8 @@ public class HeapEvictor implements ResourceListener<MemoryEvent> {
 
   private final AtomicBoolean isRunning = new AtomicBoolean(true);
 
+  private final StatisticsClock statisticsClock;
+
   private volatile int testAbortAfterLoopCount = Integer.MAX_VALUE;
 
   /*
@@ -102,11 +105,12 @@ public class HeapEvictor implements ResourceListener<MemoryEvent> {
   private volatile int numEvictionLoopsCompleted = 0;
   private volatile int numFastLoops;
 
-  public HeapEvictor(final InternalCache cache) {
-    this(cache, EVICTOR_THREAD_NAME);
+  public HeapEvictor(final InternalCache cache, StatisticsClock statisticsClock) {
+    this(cache, EVICTOR_THREAD_NAME, statisticsClock);
   }
 
-  public HeapEvictor(final InternalCache cache, final String threadName) {
+  public HeapEvictor(final InternalCache cache, final String threadName,
+      StatisticsClock statisticsClock) {
     this.cache = cache;
 
     if (!DISABLE_HEAP_EVICTOR_THREAD_POOL) {
@@ -117,6 +121,8 @@ public class HeapEvictor implements ResourceListener<MemoryEvent> {
       // disabled
       this.evictorThreadPool = null;
     }
+
+    this.statisticsClock = statisticsClock;
   }
 
   protected InternalCache cache() {
@@ -236,7 +242,7 @@ public class HeapEvictor implements ResourceListener<MemoryEvent> {
       regionsForSingleTask.add(region);
       if (mustEvict()) {
         executeInThreadPool(new RegionEvictorTask(cache.getCachePerfStats(), regionsForSingleTask,
-            this, bytesToEvictPerTask));
+            this, bytesToEvictPerTask, statisticsClock));
       } else {
         break;
       }
@@ -264,7 +270,8 @@ public class HeapEvictor implements ResourceListener<MemoryEvent> {
         List<LocalRegion> regionList = new ArrayList<>(1);
         regionList.add(region);
         RegionEvictorTask task =
-            new RegionEvictorTask(cache.getCachePerfStats(), regionList, this, bytesToEvictPerTask);
+            new RegionEvictorTask(cache.getCachePerfStats(), regionList, this, bytesToEvictPerTask,
+                statisticsClock);
         evictorTaskSet.add(task);
       }
       for (RegionEvictorTask regionEvictorTask : evictorTaskSet) {
@@ -286,7 +293,7 @@ public class HeapEvictor implements ResourceListener<MemoryEvent> {
         count++;
       }
       evictorTaskSet.add(new RegionEvictorTask(cache.getCachePerfStats(), regionsForSingleTask,
-          this, bytesToEvictPerTask));
+          this, bytesToEvictPerTask, statisticsClock));
     }
 
     // Add leftover regions to last task
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/eviction/OffHeapEvictor.java b/geode-core/src/main/java/org/apache/geode/internal/cache/eviction/OffHeapEvictor.java
index b2133c9..63145e8 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/eviction/OffHeapEvictor.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/eviction/OffHeapEvictor.java
@@ -20,6 +20,7 @@ import org.apache.geode.internal.cache.LocalRegion;
 import org.apache.geode.internal.cache.PartitionedRegion;
 import org.apache.geode.internal.cache.control.InternalResourceManager.ResourceType;
 import org.apache.geode.internal.offheap.MemoryAllocator;
+import org.apache.geode.internal.statistics.StatisticsClock;
 
 /**
  * Triggers centralized eviction(asynchronously) when the ResourceManager sends an eviction event
@@ -33,8 +34,8 @@ public class OffHeapEvictor extends HeapEvictor {
 
   private long bytesToEvictWithEachBurst;
 
-  public OffHeapEvictor(final InternalCache cache) {
-    super(cache, EVICTOR_THREAD_NAME);
+  public OffHeapEvictor(final InternalCache cache, StatisticsClock statisticsClock) {
+    super(cache, EVICTOR_THREAD_NAME, statisticsClock);
     calculateEvictionBurst();
   }
 
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/eviction/RegionEvictorTask.java b/geode-core/src/main/java/org/apache/geode/internal/cache/eviction/RegionEvictorTask.java
index 5994fea..3e7484f 100755
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/eviction/RegionEvictorTask.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/eviction/RegionEvictorTask.java
@@ -23,6 +23,7 @@ import org.apache.geode.cache.RegionDestroyedException;
 import org.apache.geode.internal.cache.CachePerfStats;
 import org.apache.geode.internal.cache.LocalRegion;
 import org.apache.geode.internal.logging.LogService;
+import org.apache.geode.internal.statistics.StatisticsClock;
 
 /**
  * Takes delta to be evicted and tries to evict the least no of LRU entry which would make
@@ -42,12 +43,15 @@ public class RegionEvictorTask implements Runnable {
 
   private final long bytesToEvictPerTask;
 
+  private final StatisticsClock statisticsClock;
+
   RegionEvictorTask(final CachePerfStats stats, final List<LocalRegion> regions,
-      final HeapEvictor evictor, final long bytesToEvictPerTask) {
+      final HeapEvictor evictor, final long bytesToEvictPerTask, StatisticsClock statisticsClock) {
     this.stats = stats;
     this.evictor = evictor;
     this.regions = regions;
     this.bytesToEvictPerTask = bytesToEvictPerTask;
+    this.statisticsClock = statisticsClock;
   }
 
   List<LocalRegion> getRegionList() {
@@ -66,7 +70,7 @@ public class RegionEvictorTask implements Runnable {
     try {
       long totalBytesEvicted = 0;
       while (true) {
-        final long start = CachePerfStats.getStatTime();
+        final long start = statisticsClock.getTime();
         synchronized (this.regions) {
           if (this.regions.isEmpty()) {
             return;
@@ -91,7 +95,7 @@ public class RegionEvictorTask implements Runnable {
               logger.warn(String.format("Exception: %s occurred during eviction ",
                   new Object[] {e.getMessage()}), e);
             } finally {
-              long end = CachePerfStats.getStatTime();
+              long end = statisticsClock.getTime();
               this.stats.incEvictWorkTime(end - start);
             }
           }
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/ha/HARegionQueue.java b/geode-core/src/main/java/org/apache/geode/internal/cache/ha/HARegionQueue.java
index 5b72dfe..97fe7fe 100755
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/ha/HARegionQueue.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/ha/HARegionQueue.java
@@ -49,10 +49,10 @@ import org.apache.logging.log4j.Logger;
 import org.apache.geode.CancelCriterion;
 import org.apache.geode.CancelException;
 import org.apache.geode.InternalGemFireError;
-import org.apache.geode.InternalGemFireException;
 import org.apache.geode.StatisticsFactory;
 import org.apache.geode.SystemFailure;
 import org.apache.geode.annotations.Immutable;
+import org.apache.geode.annotations.VisibleForTesting;
 import org.apache.geode.annotations.internal.MakeNotStatic;
 import org.apache.geode.annotations.internal.MutableForTesting;
 import org.apache.geode.cache.AttributesFactory;
@@ -100,6 +100,7 @@ import org.apache.geode.internal.cache.tier.sockets.Handshake;
 import org.apache.geode.internal.logging.LogService;
 import org.apache.geode.internal.logging.LoggingThread;
 import org.apache.geode.internal.logging.log4j.LogMarker;
+import org.apache.geode.internal.statistics.StatisticsClock;
 import org.apache.geode.internal.util.BlobHelper;
 import org.apache.geode.internal.util.concurrent.StoppableCondition;
 import org.apache.geode.internal.util.concurrent.StoppableReentrantLock;
@@ -325,21 +326,63 @@ public class HARegionQueue implements RegionQueue {
     return regionName.replace('/', '#');
   }
 
-  HARegionQueue(String regionName, HARegion haRegion, InternalCache cache, Map haContainer,
+  /**
+   * @param isPrimary whether this is the primary queue for a client
+   */
+  protected HARegionQueue(String regionName, InternalCache cache, Map haContainer,
       ClientProxyMembershipID clientProxyId, final byte clientConflation, boolean isPrimary,
-      HARegionQueueStats stats, StoppableReentrantReadWriteLock giiLock,
-      StoppableReentrantReadWriteLock rwLock, CancelCriterion cancelCriterion,
-      boolean puttingGIIDataInQueue)
+      StatisticsClock statisticsClock)
       throws IOException, ClassNotFoundException, CacheException, InterruptedException {
-    initializeHARegionQueue(regionName, haRegion, haContainer, clientProxyId, clientConflation,
-        isPrimary, stats, giiLock, rwLock, cancelCriterion, puttingGIIDataInQueue);
+
+    String processedRegionName = createRegionName(regionName);
+
+    // Initialize the statistics
+    StatisticsFactory factory = cache.getInternalDistributedSystem().getStatisticsManager();
+
+    AttributesFactory af = new AttributesFactory();
+    af.setMirrorType(MirrorType.KEYS_VALUES);
+    af.addCacheListener(createCacheListenerForHARegion());
+    af.setStatisticsEnabled(true);
+    RegionAttributes ra = af.create();
+
+    this.region = HARegion.getInstance(processedRegionName, cache, this, ra, statisticsClock);
+
+    if (this.isPrimary) {// fix for 41878
+      // since it's primary queue, we will disable the EntryExpiryTask
+      // this should be done after region creation
+      disableEntryExpiryTasks();
+    }
+
+    this.regionName = processedRegionName;
+    this.threadIdToSeqId = new MapWrapper();
+    this.idsAvailable = new LinkedHashSet();
+    setClientConflation(clientConflation);
+    this.isPrimary = isPrimary;
+    // Initialize the statistics
+    this.stats = new HARegionQueueStats(factory, processedRegionName);
+    this.haContainer = haContainer;
+    this.giiLock = new StoppableReentrantReadWriteLock(cache.getCancelCriterion());
+    this.clientProxyID = clientProxyId;
+
+    this.stopper = this.region.getCancelCriterion();
+    this.rwLock = new StoppableReentrantReadWriteLock(region.getCancelCriterion());
+    this.readLock = this.rwLock.readLock();
+    this.writeLock = this.rwLock.writeLock();
+
+    putGIIDataInRegion();
+
+    if (this.getClass() == HARegionQueue.class) {
+      initialized.set(true);
+    }
   }
 
-  private void initializeHARegionQueue(String regionName, HARegion haRegion, Map haContainer,
-      ClientProxyMembershipID clientProxyId, byte clientConflation, boolean isPrimary,
+  @VisibleForTesting
+  HARegionQueue(String regionName, HARegion haRegion, InternalCache cache, Map haContainer,
+      ClientProxyMembershipID clientProxyId, final byte clientConflation, boolean isPrimary,
       HARegionQueueStats stats, StoppableReentrantReadWriteLock giiLock,
       StoppableReentrantReadWriteLock rwLock, CancelCriterion cancelCriterion,
-      boolean putGIIDataInQueue) throws InterruptedException {
+      boolean puttingGIIDataInQueue, StatisticsClock statisticsClock)
+      throws IOException, ClassNotFoundException, CacheException, InterruptedException {
     this.regionName = regionName;
     this.region = haRegion;
     this.threadIdToSeqId = new MapWrapper();
@@ -358,7 +401,7 @@ public class HARegionQueue implements RegionQueue {
     this.writeLock = this.rwLock.writeLock();
 
     // false specifically set in tests only
-    if (putGIIDataInQueue) {
+    if (puttingGIIDataInQueue) {
       putGIIDataInRegion();
     }
     if (this.getClass() == HARegionQueue.class) {
@@ -367,68 +410,6 @@ public class HARegionQueue implements RegionQueue {
   }
 
   /**
-   * @param isPrimary whether this is the primary queue for a client
-   */
-  protected HARegionQueue(String regionName, InternalCache cache, Map haContainer,
-      ClientProxyMembershipID clientProxyId, final byte clientConflation, boolean isPrimary)
-      throws IOException, ClassNotFoundException, CacheException, InterruptedException {
-
-    String processedRegionName = createRegionName(regionName);
-
-    // Initialize the statistics
-    StatisticsFactory factory = cache.getDistributedSystem();
-    createHARegion(processedRegionName, cache);
-
-    initializeHARegionQueue(processedRegionName, this.region, haContainer, clientProxyId,
-        clientConflation, isPrimary, new HARegionQueueStats(factory, processedRegionName),
-        new StoppableReentrantReadWriteLock(cache.getCancelCriterion()),
-        new StoppableReentrantReadWriteLock(region.getCancelCriterion()),
-        this.region.getCancelCriterion(), true);
-  }
-
-  private void createHARegion(String processedRegionName, InternalCache cache)
-      throws IOException, ClassNotFoundException {
-    AttributesFactory af = new AttributesFactory();
-    af.setMirrorType(MirrorType.KEYS_VALUES);
-    af.addCacheListener(createCacheListenerForHARegion());
-    af.setStatisticsEnabled(true);
-    RegionAttributes ra = af.create();
-    this.region = HARegion.getInstance(processedRegionName, cache, this, ra);
-
-    if (isPrimary) {// fix for 41878
-      // since it's primary queue, we will disable the EntryExpiryTask
-      // this should be done after region creation
-      disableEntryExpiryTasks();
-    }
-  }
-
-  /**
-   * reinitialize the queue, presumably pulling current information from seconaries
-   */
-  public void reinitializeRegion() {
-    InternalCache cache = this.region.getCache();
-    String regionName = this.region.getName();
-    this.region.destroyRegion();
-    Exception problem = null;
-    try {
-      createHARegion(regionName, cache);
-    } catch (IOException | ClassNotFoundException e) {
-      problem = e;
-    }
-    if (problem != null) {
-      throw new InternalGemFireException("Problem recreating region queue '" + regionName + "'");
-    }
-    try {
-      this.putGIIDataInRegion();
-    } catch (InterruptedException e) {
-      cache.getCancelCriterion().checkCancelInProgress(e);
-      Thread.currentThread().interrupt();
-    }
-  }
-
-
-
-  /**
    * install DACE information from an initial image provider
    */
   @SuppressWarnings("synthetic-access")
@@ -1964,7 +1945,7 @@ public class HARegionQueue implements RegionQueue {
    * @return an instance of HARegionQueue
    */
   public static HARegionQueue getHARegionQueueInstance(String regionName, InternalCache cache,
-      final int haRgnQType, final boolean isDurable)
+      final int haRgnQType, final boolean isDurable, StatisticsClock statisticsClock)
       throws IOException, ClassNotFoundException, CacheException, InterruptedException {
     Map container = null;
     if (haRgnQType == HARegionQueue.BLOCKING_HA_QUEUE) {
@@ -1977,7 +1958,7 @@ public class HARegionQueue implements RegionQueue {
 
     return getHARegionQueueInstance(regionName, cache,
         HARegionQueueAttributes.DEFAULT_HARQ_ATTRIBUTES, haRgnQType, isDurable, container, null,
-        Handshake.CONFLATION_DEFAULT, false, Boolean.FALSE);
+        Handshake.CONFLATION_DEFAULT, false, Boolean.FALSE, statisticsClock);
   }
 
   /**
@@ -1995,7 +1976,7 @@ public class HARegionQueue implements RegionQueue {
   public static HARegionQueue getHARegionQueueInstance(String regionName, InternalCache cache,
       HARegionQueueAttributes hrqa, final int haRgnQType, final boolean isDurable, Map haContainer,
       ClientProxyMembershipID clientProxyId, final byte clientConflation, boolean isPrimary,
-      boolean canHandleDelta)
+      boolean canHandleDelta, StatisticsClock statisticsClock)
       throws IOException, ClassNotFoundException, CacheException, InterruptedException {
 
     HARegionQueue hrq = null;
@@ -2003,15 +1984,15 @@ public class HARegionQueue implements RegionQueue {
       case BLOCKING_HA_QUEUE:
         if (!isDurable && !canHandleDelta) {
           hrq = new BlockingHARegionQueue(regionName, cache, hrqa, haContainer, clientProxyId,
-              clientConflation, isPrimary);
+              clientConflation, isPrimary, statisticsClock);
         } else {
           hrq = new DurableHARegionQueue(regionName, cache, hrqa, haContainer, clientProxyId,
-              clientConflation, isPrimary);
+              clientConflation, isPrimary, statisticsClock);
         }
         break;
       case NON_BLOCKING_HA_QUEUE:
         hrq = new HARegionQueue(regionName, cache, haContainer, clientProxyId, clientConflation,
-            isPrimary);
+            isPrimary, statisticsClock);
         break;
       default:
         throw new IllegalArgumentException(
@@ -2036,7 +2017,8 @@ public class HARegionQueue implements RegionQueue {
    * @since GemFire 5.7
    */
   public static HARegionQueue getHARegionQueueInstance(String regionName, InternalCache cache,
-      HARegionQueueAttributes hrqa, final int haRgnQType, final boolean isDurable)
+      HARegionQueueAttributes hrqa, final int haRgnQType, final boolean isDurable,
+      StatisticsClock statisticsClock)
       throws IOException, ClassNotFoundException, CacheException, InterruptedException {
     Map container = null;
     if (haRgnQType == HARegionQueue.BLOCKING_HA_QUEUE) {
@@ -2048,7 +2030,7 @@ public class HARegionQueue implements RegionQueue {
     }
 
     return getHARegionQueueInstance(regionName, cache, hrqa, haRgnQType, isDurable, container, null,
-        Handshake.CONFLATION_DEFAULT, false, Boolean.FALSE);
+        Handshake.CONFLATION_DEFAULT, false, Boolean.FALSE, statisticsClock);
   }
 
   public boolean isEmptyAckList() {
@@ -2220,9 +2202,10 @@ public class HARegionQueue implements RegionQueue {
      */
     protected BlockingHARegionQueue(String regionName, InternalCache cache,
         HARegionQueueAttributes hrqa, Map haContainer, ClientProxyMembershipID clientProxyId,
-        final byte clientConflation, boolean isPrimary)
+        final byte clientConflation, boolean isPrimary, StatisticsClock statisticsClock)
         throws IOException, ClassNotFoundException, CacheException, InterruptedException {
-      super(regionName, cache, haContainer, clientProxyId, clientConflation, isPrimary);
+      super(regionName, cache, haContainer, clientProxyId, clientConflation, isPrimary,
+          statisticsClock);
       this.capacity = hrqa.getBlockingQueueCapacity();
       this.putPermits = this.capacity;
       this.lock = new StoppableReentrantLock(this.region.getCancelCriterion());
@@ -2449,9 +2432,10 @@ public class HARegionQueue implements RegionQueue {
 
     protected DurableHARegionQueue(String regionName, InternalCache cache,
         HARegionQueueAttributes hrqa, Map haContainer, ClientProxyMembershipID clientProxyId,
-        final byte clientConflation, boolean isPrimary)
+        final byte clientConflation, boolean isPrimary, StatisticsClock statisticsClock)
         throws IOException, ClassNotFoundException, CacheException, InterruptedException {
-      super(regionName, cache, hrqa, haContainer, clientProxyId, clientConflation, isPrimary);
+      super(regionName, cache, hrqa, haContainer, clientProxyId, clientConflation, isPrimary,
+          statisticsClock);
 
       this.threadIdToSeqId.keepPrevAcks = true;
       this.durableIDsList = new LinkedHashSet();
@@ -2673,54 +2657,6 @@ public class HARegionQueue implements RegionQueue {
   }
 
   /**
-   * A static class which is created only for for testing prposes as some existing tests extend the
-   * HARegionQueue. Since the constructors of HAregionQueue are private , this class can act as a
-   * bridge between the user defined HARegionQueue class & the actual class. This class object will
-   * be buggy as it will tend to publish the Object o QRM thread & the expiry thread before the
-   * complete creation of the HARegionQueue instance
-   */
-  static class TestOnlyHARegionQueue extends HARegionQueue {
-    /**
-     * Overloaded constructor to accept haContainer.
-     *
-     * @since GemFire 5.7
-     */
-    TestOnlyHARegionQueue(String regionName, InternalCache cache, Map haContainer)
-        throws IOException, ClassNotFoundException, CacheException, InterruptedException {
-      this(regionName, cache, HARegionQueueAttributes.DEFAULT_HARQ_ATTRIBUTES, haContainer,
-          Handshake.CONFLATION_DEFAULT, false);
-      this.initialized.set(true);
-    }
-
-    TestOnlyHARegionQueue(String regionName, InternalCache cache)
-        throws IOException, ClassNotFoundException, CacheException, InterruptedException {
-      this(regionName, cache, HARegionQueueAttributes.DEFAULT_HARQ_ATTRIBUTES, new HashMap(),
-          Handshake.CONFLATION_DEFAULT, false);
-    }
-
-    TestOnlyHARegionQueue(String regionName, InternalCache cache, HARegionQueueAttributes hrqa,
-        Map haContainer, final byte clientConflation, boolean isPrimary)
-        throws IOException, ClassNotFoundException, CacheException, InterruptedException {
-      super(regionName, cache, haContainer, null, clientConflation, isPrimary);
-      ExpirationAttributes ea =
-          new ExpirationAttributes(hrqa.getExpiryTime(), ExpirationAction.LOCAL_INVALIDATE);
-      this.region.setOwner(this);
-      this.region.getAttributesMutator().setEntryTimeToLive(ea);
-      this.initialized.set(true);
-    }
-
-    /**
-     * Overloaded constructor to pass an {@code HashMap} instance as a haContainer.
-     *
-     * @since GemFire 5.7
-     */
-    TestOnlyHARegionQueue(String regionName, InternalCache cache, HARegionQueueAttributes hrqa)
-        throws IOException, ClassNotFoundException, CacheException, InterruptedException {
-      this(regionName, cache, hrqa, new HashMap(), Handshake.CONFLATION_DEFAULT, false);
-    }
-  }
-
-  /**
    * This thread will check for messages which have been dispatched. After a configurable time or
    * size is reached, it will create a new {@code QueueRemovalMessage} and send it to all the nodes
    * in the DistributedSystem
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/partitioned/CreateMissingBucketsTask.java b/geode-core/src/main/java/org/apache/geode/internal/cache/partitioned/CreateMissingBucketsTask.java
index 1066bde..ea60204 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/partitioned/CreateMissingBucketsTask.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/partitioned/CreateMissingBucketsTask.java
@@ -18,7 +18,6 @@ import org.apache.geode.internal.cache.ColocationHelper;
 import org.apache.geode.internal.cache.PRHARedundancyProvider;
 import org.apache.geode.internal.cache.PartitionedRegion;
 import org.apache.geode.internal.cache.PartitionedRegion.RecoveryLock;
-import org.apache.geode.internal.cache.PartitionedRegionStats;
 
 /**
  * A task for creating buckets in a child colocated region that are present in the leader region.
@@ -47,7 +46,7 @@ public class CreateMissingBucketsTask extends RecoveryRunnable {
     if (parentRegion == null) {
       return;
     }
-    // Fix for 48954 - Make sure the parent region has created missing buckets
+    // Make sure the parent region has created missing buckets
     // before we create missing buckets for this child region.
     createMissingBuckets(parentRegion);
 
@@ -55,8 +54,6 @@ public class CreateMissingBucketsTask extends RecoveryRunnable {
 
       if (parentRegion.getRegionAdvisor().getBucketAdvisor(i).getBucketRedundancy() != region
           .getRegionAdvisor().getBucketAdvisor(i).getBucketRedundancy()) {
-        /* if (leaderRegion.getRegionAdvisor().isStorageAssignedForBucket(i)) { */
-        final long startTime = PartitionedRegionStats.startTime();
         region.getRedundancyProvider().createBucketAtomically(i, 0, true, null);
       }
     }
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/snapshot/RegionSnapshotServiceImpl.java b/geode-core/src/main/java/org/apache/geode/internal/cache/snapshot/RegionSnapshotServiceImpl.java
index 844ea0f..b565da2 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/snapshot/RegionSnapshotServiceImpl.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/snapshot/RegionSnapshotServiceImpl.java
@@ -50,7 +50,6 @@ import org.apache.geode.distributed.DistributedMember;
 import org.apache.geode.distributed.internal.DistributionConfig;
 import org.apache.geode.internal.DSCODE;
 import org.apache.geode.internal.InternalEntity;
-import org.apache.geode.internal.cache.CachePerfStats;
 import org.apache.geode.internal.cache.CachedDeserializable;
 import org.apache.geode.internal.cache.CachedDeserializableFactory;
 import org.apache.geode.internal.cache.InternalCache;
@@ -234,7 +233,7 @@ public class RegionSnapshotServiceImpl<K, V> implements RegionSnapshotService<K,
       throws IOException, ClassNotFoundException {
     long count = 0;
     long bytes = 0;
-    long start = CachePerfStats.getStatTime();
+    long start = local.getCachePerfStats().getTime();
 
     // Would be interesting to use a PriorityQueue ordered on isDone()
     // but this is probably close enough in practice.
@@ -349,7 +348,7 @@ public class RegionSnapshotServiceImpl<K, V> implements RegionSnapshotService<K,
     }
 
     long count = 0;
-    long start = CachePerfStats.getStatTime();
+    long start = local.getCachePerfStats().getTime();
     SnapshotWriter writer =
         GFSnapshot.create(snapshot, region.getFullPath(), (InternalCache) region.getCache());
     try {
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/AcceptorBuilder.java b/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/AcceptorBuilder.java
index df9cd3c..f2769d4 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/AcceptorBuilder.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/AcceptorBuilder.java
@@ -29,6 +29,7 @@ import org.apache.geode.internal.cache.tier.sockets.CacheClientNotifier.CacheCli
 import org.apache.geode.internal.cache.tier.sockets.ClientHealthMonitor.ClientHealthMonitorProvider;
 import org.apache.geode.internal.net.SocketCreator;
 import org.apache.geode.internal.security.SecurityService;
+import org.apache.geode.internal.statistics.StatisticsClock;
 
 /**
  * Builds an instance of {@link Acceptor}.
@@ -51,6 +52,7 @@ public class AcceptorBuilder implements AcceptorFactory {
   private ServerConnectionFactory serverConnectionFactory;
   private long timeLimitMillis;
   private SecurityService securityService;
+  private StatisticsClock statisticsClock;
 
   private boolean isGatewayReceiver;
   private List<GatewayTransportFilter> gatewayTransportFilters = Collections.emptyList();
@@ -79,6 +81,7 @@ public class AcceptorBuilder implements AcceptorFactory {
     serverConnectionFactory = server.getServerConnectionFactory();
     timeLimitMillis = server.getTimeLimitMillis();
     securityService = server.getSecurityService();
+    statisticsClock = server.getStatisticsClock();
 
     socketCreatorSupplier = server.getSocketCreatorSupplier();
     cacheClientNotifierProvider = server.getCacheClientNotifierProvider();
@@ -285,6 +288,16 @@ public class AcceptorBuilder implements AcceptorFactory {
     return this;
   }
 
+  /**
+   * Sets {@code statisticsClock}. Must be invoked after or instead of
+   * {@link #forServer(InternalCacheServer)}.
+   */
+  @VisibleForTesting
+  AcceptorBuilder setStatisticsClock(StatisticsClock statisticsClock) {
+    this.statisticsClock = statisticsClock;
+    return this;
+  }
+
   @Override
   public Acceptor create(OverflowAttributes overflowAttributes) throws IOException {
     return new AcceptorImpl(port, bindAddress, notifyBySubscription, socketBufferSize,
@@ -292,7 +305,7 @@ public class AcceptorBuilder implements AcceptorFactory {
         messageTimeToLive, connectionListener, overflowAttributes, tcpNoDelay,
         serverConnectionFactory, timeLimitMillis, securityService, socketCreatorSupplier,
         cacheClientNotifierProvider, clientHealthMonitorProvider, isGatewayReceiver,
-        gatewayTransportFilters);
+        gatewayTransportFilters, statisticsClock);
   }
 
   @VisibleForTesting
@@ -394,4 +407,9 @@ public class AcceptorBuilder implements AcceptorFactory {
   ClientHealthMonitorProvider getClientHealthMonitorProvider() {
     return clientHealthMonitorProvider;
   }
+
+  @VisibleForTesting
+  StatisticsClock getStatisticsClock() {
+    return statisticsClock;
+  }
 }
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/AcceptorImpl.java b/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/AcceptorImpl.java
index 8fdd78a..ec297ad 100755
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/AcceptorImpl.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/AcceptorImpl.java
@@ -98,6 +98,8 @@ import org.apache.geode.internal.logging.LoggingThreadFactory.ThreadInitializer;
 import org.apache.geode.internal.monitoring.ThreadsMonitoring;
 import org.apache.geode.internal.net.SocketCreator;
 import org.apache.geode.internal.security.SecurityService;
+import org.apache.geode.internal.statistics.StatisticsClock;
+import org.apache.geode.internal.statistics.StatisticsClockFactory;
 import org.apache.geode.internal.tcp.ConnectionTable;
 import org.apache.geode.internal.util.ArrayUtils;
 
@@ -336,6 +338,7 @@ public class AcceptorImpl implements Acceptor, Runnable {
   private final boolean isGatewayReceiver;
 
   private final List<GatewayTransportFilter> gatewayTransportFilters;
+  private final StatisticsClock statisticsClock;
 
   private final SocketCreator socketCreator;
 
@@ -380,7 +383,8 @@ public class AcceptorImpl implements Acceptor, Runnable {
         internalCache, maxConnections, maxThreads, maximumMessageCount, messageTimeToLive,
         connectionListener, overflowAttributes, tcpNoDelay, serverConnectionFactory,
         timeLimitMillis, securityService, socketCreatorSupplier, cacheClientNotifierProvider,
-        clientHealthMonitorProvider, false, Collections.emptyList());
+        clientHealthMonitorProvider, false, Collections.emptyList(),
+        StatisticsClockFactory.disabledClock());
   }
 
   /**
@@ -389,6 +393,8 @@ public class AcceptorImpl implements Acceptor, Runnable {
    * <p>
    * Initializes this acceptor thread to listen for connections on the given port.
    *
+   * @param gatewayReceiver the GatewayReceiver that will use this AcceptorImpl instance
+   * @param gatewayReceiverMetrics the GatewayReceiverMetrics to use for exposing metrics
    * @param port The port on which this acceptor listens for connections. If {@code 0}, a
    *        random port will be chosen.
    * @param bindHostName The ip address or host name this acceptor listens on for connections. If
@@ -400,23 +406,24 @@ public class AcceptorImpl implements Acceptor, Runnable {
    * @param maxConnections the maximum number of connections allowed in the server pool
    * @param maxThreads the maximum number of threads allowed in the server pool
    * @param securityService the SecurityService to use for authentication and authorization
-   * @param gatewayReceiver the GatewayReceiver that will use this AcceptorImpl instance
-   * @param gatewayReceiverMetrics the GatewayReceiverMetrics to use for exposing metrics
    * @param gatewayTransportFilters List of GatewayTransportFilters
    */
   AcceptorImpl(final int port, final String bindHostName, final boolean notifyBySubscription,
       final int socketBufferSize, final int maximumTimeBetweenPings,
       final InternalCache internalCache, final int maxConnections, final int maxThreads,
       final int maximumMessageCount, final int messageTimeToLive,
-      final ConnectionListener connectionListener, final OverflowAttributes overflowAttributes,
+      final ConnectionListener connectionListener,
+      final OverflowAttributes overflowAttributes,
       final boolean tcpNoDelay, final ServerConnectionFactory serverConnectionFactory,
       final long timeLimitMillis, final SecurityService securityService,
       final Supplier<SocketCreator> socketCreatorSupplier,
       final CacheClientNotifierProvider cacheClientNotifierProvider,
       final ClientHealthMonitorProvider clientHealthMonitorProvider,
       final boolean isGatewayReceiver,
-      final List<GatewayTransportFilter> gatewayTransportFilters) throws IOException {
+      final List<GatewayTransportFilter> gatewayTransportFilters,
+      final StatisticsClock statisticsClock) throws IOException {
     this.securityService = securityService;
+    this.statisticsClock = statisticsClock;
 
     this.isGatewayReceiver = isGatewayReceiver;
     this.gatewayTransportFilters = gatewayTransportFilters;
@@ -609,8 +616,9 @@ public class AcceptorImpl implements Acceptor, Runnable {
     cache = internalCache;
     crHelper = new CachedRegionHelper(cache);
 
-    clientNotifier = cacheClientNotifierProvider.get(internalCache, stats, maximumMessageCount,
-        messageTimeToLive, this.connectionListener, overflowAttributes, isGatewayReceiver());
+    clientNotifier =
+        cacheClientNotifierProvider.get(internalCache, statisticsClock, stats, maximumMessageCount,
+            messageTimeToLive, this.connectionListener, overflowAttributes, isGatewayReceiver());
 
     this.socketBufferSize = socketBufferSize;
 
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/CacheClientNotifier.java b/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/CacheClientNotifier.java
index e7eb977..24f36fc 100755
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/CacheClientNotifier.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/CacheClientNotifier.java
@@ -107,6 +107,7 @@ import org.apache.geode.internal.logging.InternalLogWriter;
 import org.apache.geode.internal.logging.LogService;
 import org.apache.geode.internal.net.SocketCloser;
 import org.apache.geode.internal.statistics.DummyStatisticsFactory;
+import org.apache.geode.internal.statistics.StatisticsClock;
 import org.apache.geode.security.AccessControl;
 import org.apache.geode.security.AuthenticationFailedException;
 import org.apache.geode.security.AuthenticationRequiredException;
@@ -135,12 +136,12 @@ public class CacheClientNotifier {
    * @return A {@code CacheClientNotifier} instance
    */
   public static synchronized CacheClientNotifier getInstance(InternalCache cache,
-      CacheServerStats acceptorStats, int maximumMessageCount, int messageTimeToLive,
-      ConnectionListener listener, OverflowAttributes overflowAttributes,
+      StatisticsClock statisticsClock, CacheServerStats acceptorStats, int maximumMessageCount,
+      int messageTimeToLive, ConnectionListener listener, OverflowAttributes overflowAttributes,
       boolean isGatewayReceiver) {
     if (ccnSingleton == null) {
-      ccnSingleton = new CacheClientNotifier(cache, acceptorStats, maximumMessageCount,
-          messageTimeToLive, listener, isGatewayReceiver);
+      ccnSingleton = new CacheClientNotifier(cache, statisticsClock, acceptorStats,
+          maximumMessageCount, messageTimeToLive, listener, isGatewayReceiver);
     }
 
     if (!isGatewayReceiver && ccnSingleton.getHaContainer() == null) {
@@ -287,8 +288,8 @@ public class CacheClientNotifier {
         }
         cacheClientProxy =
             new CacheClientProxy(this, socket, clientProxyMembershipID, isPrimary, clientConflation,
-                clientVersion,
-                acceptorId, notifyBySubscription, cache.getSecurityService(), subject);
+                clientVersion, acceptorId, notifyBySubscription, cache.getSecurityService(),
+                subject, statisticsClock);
         successful = initializeProxy(cacheClientProxy);
       } else {
         cacheClientProxy.setSubject(subject);
@@ -363,7 +364,8 @@ public class CacheClientNotifier {
       // Create the new proxy for this non-durable client
       cacheClientProxy =
           new CacheClientProxy(this, socket, clientProxyMembershipID, isPrimary, clientConflation,
-              clientVersion, acceptorId, notifyBySubscription, cache.getSecurityService(), subject);
+              clientVersion, acceptorId, notifyBySubscription, cache.getSecurityService(), subject,
+              statisticsClock);
       successful = initializeProxy(cacheClientProxy);
     }
 
@@ -1692,11 +1694,12 @@ public class CacheClientNotifier {
    * @param cache The GemFire {@code InternalCache}
    * @param listener a listener which should receive notifications abouts queues being added or
    */
-  private CacheClientNotifier(InternalCache cache, CacheServerStats acceptorStats,
-      int maximumMessageCount, int messageTimeToLive, ConnectionListener listener,
-      boolean isGatewayReceiver) {
+  private CacheClientNotifier(InternalCache cache, StatisticsClock statisticsClock,
+      CacheServerStats acceptorStats, int maximumMessageCount, int messageTimeToLive,
+      ConnectionListener listener, boolean isGatewayReceiver) {
     // Set the Cache
     setCache(cache);
+    this.statisticsClock = statisticsClock;
     this.acceptorStats = acceptorStats;
     // we only need one thread per client and wait 50ms for close
     socketCloser = new SocketCloser(1, 50);
@@ -1921,6 +1924,8 @@ public class CacheClientNotifier {
    */
   private final ConnectionListener _connectionListener;
 
+  private final StatisticsClock statisticsClock;
+
   private final CacheServerStats acceptorStats;
 
   /**
@@ -2091,9 +2096,10 @@ public class CacheClientNotifier {
   @FunctionalInterface
   @VisibleForTesting
   public interface CacheClientNotifierProvider {
-    CacheClientNotifier get(InternalCache cache, CacheServerStats acceptorStats,
-        int maximumMessageCount, int messageTimeToLive, ConnectionListener listener,
-        OverflowAttributes overflowAttributes, boolean isGatewayReceiver);
+    CacheClientNotifier get(InternalCache cache, StatisticsClock statisticsClock,
+        CacheServerStats acceptorStats, int maximumMessageCount, int messageTimeToLive,
+        ConnectionListener listener, OverflowAttributes overflowAttributes,
+        boolean isGatewayReceiver);
   }
 
   @VisibleForTesting
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/CacheClientProxy.java b/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/CacheClientProxy.java
index 5c12281..cb235bf 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/CacheClientProxy.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/CacheClientProxy.java
@@ -100,6 +100,7 @@ import org.apache.geode.internal.logging.LoggingThread;
 import org.apache.geode.internal.logging.log4j.LogMarker;
 import org.apache.geode.internal.security.AuthorizeRequestPP;
 import org.apache.geode.internal.security.SecurityService;
+import org.apache.geode.internal.statistics.StatisticsClock;
 import org.apache.geode.security.AccessControl;
 
 /**
@@ -315,6 +316,7 @@ public class CacheClientProxy implements ClientSession {
   private final Object drainsInProgressLock = new Object();
 
   private final SecurityService securityService;
+  private final StatisticsClock statisticsClock;
 
   /**
    * Constructor.
@@ -328,7 +330,8 @@ public class CacheClientProxy implements ClientSession {
   protected CacheClientProxy(CacheClientNotifier ccn, Socket socket,
       ClientProxyMembershipID proxyID, boolean isPrimary, byte clientConflation,
       Version clientVersion, long acceptorId, boolean notifyBySubscription,
-      SecurityService securityService, Subject subject) throws CacheException {
+      SecurityService securityService, Subject subject, StatisticsClock statisticsClock)
+      throws CacheException {
 
     initializeTransientFields(socket, proxyID, isPrimary, clientConflation, clientVersion);
     this._cacheClientNotifier = ccn;
@@ -338,7 +341,8 @@ public class CacheClientProxy implements ClientSession {
     this._messageTimeToLive = ccn.getMessageTimeToLive();
     this._acceptorId = acceptorId;
     this.notifyBySubscription = notifyBySubscription;
-    StatisticsFactory factory = this._cache.getDistributedSystem();
+    StatisticsFactory factory = this._cache.getInternalDistributedSystem().getStatisticsManager();
+    this.statisticsClock = statisticsClock;
     this._statistics =
         new CacheClientProxyStats(factory, "id_" + this.proxyID.getDistributedMember().getId()
             + "_at_" + this._remoteHostAddress + ":" + this._socket.getPort());
@@ -1704,7 +1708,7 @@ public class CacheClientProxy implements ClientSession {
   }
 
   MessageDispatcher createMessageDispatcher(String name) {
-    return new MessageDispatcher(this, name);
+    return new MessageDispatcher(this, name, statisticsClock);
   }
 
   protected void startOrResumeMessageDispatcher(boolean processedMarker) {
@@ -2225,7 +2229,8 @@ public class CacheClientProxy implements ClientSession {
      *        messages
      * @param name thread name for this dispatcher
      */
-    protected MessageDispatcher(CacheClientProxy proxy, String name) throws CacheException {
+    protected MessageDispatcher(CacheClientProxy proxy, String name,
+        StatisticsClock statisticsClock) throws CacheException {
       super(name);
 
       this._proxy = proxy;
@@ -2252,7 +2257,7 @@ public class CacheClientProxy implements ClientSession {
         this._messageQueue = HARegionQueue.getHARegionQueueInstance(getProxy().getHARegionName(),
             getCache(), harq, HARegionQueue.BLOCKING_HA_QUEUE, createDurableQueue,
             proxy._cacheClientNotifier.getHaContainer(), proxy.getProxyID(),
-            this._proxy.clientConflation, this._proxy.isPrimary(), canHandleDelta);
+            this._proxy.clientConflation, this._proxy.isPrimary(), canHandleDelta, statisticsClock);
         // Check if interests were registered during HARegion GII.
         if (this._proxy.hasRegisteredInterested()) {
           this._messageQueue.setHasRegisteredInterest(true);
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/tx/PartitionedTXRegionStub.java b/geode-core/src/main/java/org/apache/geode/internal/cache/tx/PartitionedTXRegionStub.java
index 4645589..5e38c08 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/tx/PartitionedTXRegionStub.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/tx/PartitionedTXRegionStub.java
@@ -40,7 +40,6 @@ import org.apache.geode.internal.cache.InternalRegion;
 import org.apache.geode.internal.cache.KeyInfo;
 import org.apache.geode.internal.cache.PartitionedRegion;
 import org.apache.geode.internal.cache.PartitionedRegion.RetryTimeKeeper;
-import org.apache.geode.internal.cache.PartitionedRegionStats;
 import org.apache.geode.internal.cache.PrimaryBucketException;
 import org.apache.geode.internal.cache.PutAllPartialResultException;
 import org.apache.geode.internal.cache.PutAllPartialResultException.PutAllPartialResult;
@@ -401,7 +400,7 @@ public class PartitionedTXRegionStub extends AbstractPeerTXRegionStub {
     }
 
     PartitionedRegion pr = (PartitionedRegion) r;
-    final long startTime = PartitionedRegionStats.startTime();
+    final long startTime = pr.prStats.getTime();
     // build all the msgs by bucketid
     HashMap prMsgMap = putallO.createPRMessages();
     PutAllPartialResult partialKeys = new PutAllPartialResult(putallO.putAllDataSize);
@@ -461,7 +460,7 @@ public class PartitionedTXRegionStub extends AbstractPeerTXRegionStub {
     }
 
     PartitionedRegion pr = (PartitionedRegion) r;
-    final long startTime = PartitionedRegionStats.startTime();
+    final long startTime = pr.prStats.getTime();
     // build all the msgs by bucketid
     HashMap<Integer, RemoveAllPRMessage> prMsgMap = op.createPRMessages();
     PutAllPartialResult partialKeys = new PutAllPartialResult(op.removeAllDataSize);
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/wan/AbstractGatewaySender.java b/geode-core/src/main/java/org/apache/geode/internal/cache/wan/AbstractGatewaySender.java
index 3c4411f..b02026b 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/wan/AbstractGatewaySender.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/wan/AbstractGatewaySender.java
@@ -14,6 +14,8 @@
  */
 package org.apache.geode.internal.cache.wan;
 
+import static org.apache.geode.internal.statistics.StatisticsClockFactory.disabledClock;
+
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.HashSet;
@@ -77,6 +79,7 @@ import org.apache.geode.internal.offheap.Releasable;
 import org.apache.geode.internal.offheap.annotations.Released;
 import org.apache.geode.internal.offheap.annotations.Retained;
 import org.apache.geode.internal.offheap.annotations.Unretained;
+import org.apache.geode.internal.statistics.StatisticsClock;
 
 /**
  * Abstract implementation of both Serial and Parallel GatewaySender. It handles common
@@ -229,10 +232,16 @@ public abstract class AbstractGatewaySender implements InternalGatewaySender, Di
 
   final Object lockForConcurrentDispatcher = new Object();
 
-  protected AbstractGatewaySender() {}
+  private final StatisticsClock statisticsClock;
+
+  protected AbstractGatewaySender() {
+    statisticsClock = disabledClock();
+  }
 
-  public AbstractGatewaySender(InternalCache cache, GatewaySenderAttributes attrs) {
+  public AbstractGatewaySender(InternalCache cache, StatisticsClock statisticsClock,
+      GatewaySenderAttributes attrs) {
     this.cache = cache;
+    this.statisticsClock = statisticsClock;
     this.id = attrs.getId();
     this.socketBufferSize = attrs.getSocketBufferSize();
     this.socketReadTimeout = attrs.getSocketReadTimeout();
@@ -268,7 +277,8 @@ public abstract class AbstractGatewaySender implements InternalGatewaySender, Di
       this.stopper = new Stopper(cache.getCancelCriterion());
       this.senderAdvisor = GatewaySenderAdvisor.createGatewaySenderAdvisor(this);
       if (!this.isForInternalUse()) {
-        this.statistics = new GatewaySenderStats(cache.getDistributedSystem(), id);
+        this.statistics = new GatewaySenderStats(cache.getDistributedSystem(),
+            "gatewaySenderStats-", id, statisticsClock);
       }
       initializeEventIdIndex();
     }
@@ -285,6 +295,11 @@ public abstract class AbstractGatewaySender implements InternalGatewaySender, Di
     return statistics;
   }
 
+  @Override
+  public StatisticsClock getStatisticsClock() {
+    return statisticsClock;
+  }
+
   public void initProxy() {
     // no op
   }
@@ -1257,7 +1272,7 @@ public abstract class AbstractGatewaySender implements InternalGatewaySender, Di
         @Override
         public CachePerfStats getCachePerfStats() {
           return new CachePerfStats(cache.getDistributedSystem(),
-              "RegionStats-" + META_DATA_REGION_NAME);
+              "RegionStats-" + META_DATA_REGION_NAME, sender.statisticsClock);
         }
       };
 
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/wan/GatewaySenderStats.java b/geode-core/src/main/java/org/apache/geode/internal/cache/wan/GatewaySenderStats.java
index e066373..94d4b1c 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/wan/GatewaySenderStats.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/wan/GatewaySenderStats.java
@@ -21,12 +21,11 @@ import org.apache.geode.StatisticsType;
 import org.apache.geode.StatisticsTypeFactory;
 import org.apache.geode.annotations.Immutable;
 import org.apache.geode.distributed.internal.DistributionStats;
-import org.apache.geode.internal.cache.CachePerfStats;
+import org.apache.geode.internal.statistics.StatisticsClock;
 import org.apache.geode.internal.statistics.StatisticsTypeFactoryImpl;
 
 public class GatewaySenderStats {
 
-
   public static final String typeName = "GatewaySenderStatistics";
 
 
@@ -293,6 +292,8 @@ public class GatewaySenderStats {
   /** The <code>Statistics</code> instance to which most behavior is delegated */
   private final Statistics stats;
 
+  private final StatisticsClock statisticsClock;
+
   /////////////////////// Constructors ///////////////////////
 
   /**
@@ -302,8 +303,9 @@ public class GatewaySenderStats {
    * @param gatewaySenderId The id of the <code>GatewaySender</code> used to generate the name of
    *        the <code>Statistics</code>
    */
-  public GatewaySenderStats(StatisticsFactory f, String gatewaySenderId) {
-    this.stats = f.createAtomicStatistics(type, "gatewaySenderStats-" + gatewaySenderId);
+  public GatewaySenderStats(StatisticsFactory f, String textIdPrefix, String gatewaySenderId,
+      StatisticsClock statisticsClock) {
+    this(f, textIdPrefix + gatewaySenderId, type, statisticsClock);
   }
 
   /**
@@ -314,8 +316,15 @@ public class GatewaySenderStats {
    *        <code>Statistics</code>
    * @param statType The StatisticsTYpe
    */
-  public GatewaySenderStats(StatisticsFactory f, String asyncQueueId, StatisticsType statType) {
-    this.stats = f.createAtomicStatistics(statType, "asyncEventQueueStats-" + asyncQueueId);
+  public GatewaySenderStats(StatisticsFactory f, String textIdPrefix, String asyncQueueId,
+      StatisticsType statType, StatisticsClock statisticsClock) {
+    this(f, textIdPrefix + asyncQueueId, statType, statisticsClock);
+  }
+
+  private GatewaySenderStats(StatisticsFactory f, String textId, StatisticsType statType,
+      StatisticsClock statisticsClock) {
+    stats = f.createAtomicStatistics(statType, textId);
+    this.statisticsClock = statisticsClock;
   }
 
   ///////////////////// Instance Methods /////////////////////
@@ -866,11 +875,11 @@ public class GatewaySenderStats {
 
   public long startLoadBalance() {
     stats.incInt(loadBalancesInProgressId, 1);
-    return CachePerfStats.getStatTime();
+    return statisticsClock.getTime();
   }
 
   public void endLoadBalance(long start) {
-    long delta = CachePerfStats.getStatTime() - start;
+    long delta = statisticsClock.getTime() - start;
     stats.incInt(loadBalancesInProgressId, -1);
     stats.incInt(loadBalancesCompletedId, 1);
     stats.incLong(loadBalanceTimeId, delta);
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/wan/InternalGatewaySender.java b/geode-core/src/main/java/org/apache/geode/internal/cache/wan/InternalGatewaySender.java
index 783e1d9..5c91b8d 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/wan/InternalGatewaySender.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/wan/InternalGatewaySender.java
@@ -22,6 +22,7 @@ import java.util.concurrent.TimeUnit;
 import org.apache.geode.cache.wan.GatewaySender;
 import org.apache.geode.internal.cache.InternalCache;
 import org.apache.geode.internal.cache.RegionQueue;
+import org.apache.geode.internal.statistics.StatisticsClock;
 
 public interface InternalGatewaySender extends GatewaySender {
 
@@ -33,6 +34,8 @@ public interface InternalGatewaySender extends GatewaySender {
 
   GatewaySenderStats getStatistics();
 
+  StatisticsClock getStatisticsClock();
+
   boolean waitUntilFlushed(long timeout, TimeUnit unit) throws InterruptedException;
 
   boolean isForwardExpirationDestroy();
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/wan/parallel/ParallelGatewaySenderQueue.java b/geode-core/src/main/java/org/apache/geode/internal/cache/wan/parallel/ParallelGatewaySenderQueue.java
index 0e9c2f3..6bb0020 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/wan/parallel/ParallelGatewaySenderQueue.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/wan/parallel/ParallelGatewaySenderQueue.java
@@ -84,6 +84,7 @@ import org.apache.geode.internal.cache.wan.GatewaySenderStats;
 import org.apache.geode.internal.logging.LogService;
 import org.apache.geode.internal.logging.LoggingExecutors;
 import org.apache.geode.internal.size.SingleObjectSizer;
+import org.apache.geode.internal.statistics.StatisticsClock;
 import org.apache.geode.internal.util.concurrent.StoppableCondition;
 import org.apache.geode.internal.util.concurrent.StoppableReentrantLock;
 import org.apache.geode.management.ManagementService;
@@ -359,7 +360,8 @@ public class ParallelGatewaySenderQueue implements RegionQueue {
         }
 
         ParallelGatewaySenderQueueMetaRegion meta =
-            new ParallelGatewaySenderQueueMetaRegion(prQName, ra, null, cache, sender);
+            new ParallelGatewaySenderQueueMetaRegion(prQName, ra, null, cache, sender,
+                sender.getStatisticsClock());
 
         try {
           prQ = (PartitionedRegion) cache.createVMRegion(prQName, ra,
@@ -1789,12 +1791,14 @@ public class ParallelGatewaySenderQueue implements RegionQueue {
     AbstractGatewaySender sender = null;
 
     public ParallelGatewaySenderQueueMetaRegion(String regionName, RegionAttributes attrs,
-        LocalRegion parentRegion, InternalCache cache, AbstractGatewaySender pgSender) {
+        LocalRegion parentRegion, InternalCache cache, AbstractGatewaySender pgSender,
+        StatisticsClock statisticsClock) {
       super(regionName, attrs, parentRegion, cache,
           new InternalRegionArguments().setDestroyLockFlag(true).setRecreateFlag(false)
               .setSnapshotInputStream(null).setImageTarget(null)
               .setIsUsedForParallelGatewaySenderQueue(true)
-              .setParallelGatewaySender((AbstractGatewaySender) pgSender));
+              .setParallelGatewaySender((AbstractGatewaySender) pgSender),
+          statisticsClock);
       this.sender = (AbstractGatewaySender) pgSender;
 
     }
@@ -1855,7 +1859,8 @@ public class ParallelGatewaySenderQueue implements RegionQueue {
     ParallelGatewaySenderQueueMetaRegion newMetataRegion(InternalCache cache, final String prQName,
         final RegionAttributes ra, AbstractGatewaySender sender) {
       ParallelGatewaySenderQueueMetaRegion meta =
-          new ParallelGatewaySenderQueueMetaRegion(prQName, ra, null, cache, sender);
+          new ParallelGatewaySenderQueueMetaRegion(prQName, ra, null, cache, sender,
+              sender.getStatisticsClock());
       return meta;
     }
   }
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/wan/serial/SerialGatewaySenderQueue.java b/geode-core/src/main/java/org/apache/geode/internal/cache/wan/serial/SerialGatewaySenderQueue.java
index d6704fe..8064f80 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/wan/serial/SerialGatewaySenderQueue.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/wan/serial/SerialGatewaySenderQueue.java
@@ -69,6 +69,7 @@ import org.apache.geode.internal.cache.wan.GatewaySenderEventImpl;
 import org.apache.geode.internal.cache.wan.GatewaySenderStats;
 import org.apache.geode.internal.logging.LogService;
 import org.apache.geode.internal.offheap.OffHeapRegionEntryHelper;
+import org.apache.geode.internal.statistics.StatisticsClock;
 import org.apache.geode.management.ManagementService;
 import org.apache.geode.management.internal.beans.AsyncEventQueueMBean;
 import org.apache.geode.management.internal.beans.GatewaySenderMBean;
@@ -876,7 +877,8 @@ public class SerialGatewaySenderQueue implements RegionQueue {
       final RegionAttributes<Long, AsyncEvent> ra = factory.create();
       try {
         SerialGatewaySenderQueueMetaRegion meta =
-            new SerialGatewaySenderQueueMetaRegion(this.regionName, ra, null, gemCache, sender);
+            new SerialGatewaySenderQueueMetaRegion(this.regionName, ra, null, gemCache, sender,
+                sender.getStatisticsClock());
         try {
           this.region = gemCache.createVMRegion(this.regionName, ra,
               new InternalRegionArguments().setInternalMetaRegion(meta).setDestroyLockFlag(true)
@@ -1116,11 +1118,13 @@ public class SerialGatewaySenderQueue implements RegionQueue {
     AbstractGatewaySender sender = null;
 
     protected SerialGatewaySenderQueueMetaRegion(String regionName, RegionAttributes attrs,
-        LocalRegion parentRegion, InternalCache cache, AbstractGatewaySender sender) {
+        LocalRegion parentRegion, InternalCache cache, AbstractGatewaySender sender,
+        StatisticsClock statisticsClock) {
       super(regionName, attrs, parentRegion, cache,
           new InternalRegionArguments().setDestroyLockFlag(true).setRecreateFlag(false)
               .setSnapshotInputStream(null).setImageTarget(null)
-              .setIsUsedForSerialGatewaySenderQueue(true).setSerialGatewaySender(sender));
+              .setIsUsedForSerialGatewaySenderQueue(true).setSerialGatewaySender(sender),
+          statisticsClock);
       this.sender = sender;
     }
 
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/xmlcache/CacheCreation.java b/geode-core/src/main/java/org/apache/geode/internal/cache/xmlcache/CacheCreation.java
index 83b3386..ae071e3 100755
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/xmlcache/CacheCreation.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/xmlcache/CacheCreation.java
@@ -17,6 +17,7 @@ package org.apache.geode.internal.cache.xmlcache;
 import static java.lang.String.format;
 import static org.apache.geode.internal.logging.LogWriterFactory.toSecurityLogWriter;
 import static org.apache.geode.internal.logging.LogWriterLevel.ALL;
+import static org.apache.geode.internal.statistics.StatisticsClockFactory.disabledClock;
 
 import java.io.File;
 import java.io.IOException;
@@ -157,6 +158,7 @@ import org.apache.geode.internal.logging.LogService;
 import org.apache.geode.internal.offheap.MemoryAllocator;
 import org.apache.geode.internal.security.SecurityService;
 import org.apache.geode.internal.security.SecurityServiceFactory;
+import org.apache.geode.internal.statistics.StatisticsClock;
 import org.apache.geode.management.internal.JmxManagerAdvisor;
 import org.apache.geode.management.internal.RestAgent;
 import org.apache.geode.pdx.JSONFormatter;
@@ -2426,6 +2428,11 @@ public class CacheCreation implements InternalCache {
     throw new UnsupportedOperationException("Should not be invoked");
   }
 
+  @Override
+  public StatisticsClock getStatisticsClock() {
+    return disabledClock();
+  }
+
   CacheTransactionManagerCreation getCacheTransactionManagerCreation() {
     return cacheTransactionManagerCreation;
   }
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/xmlcache/CacheServerCreation.java b/geode-core/src/main/java/org/apache/geode/internal/cache/xmlcache/CacheServerCreation.java
index fa6c518..eaaa3e8 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/xmlcache/CacheServerCreation.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/xmlcache/CacheServerCreation.java
@@ -34,6 +34,7 @@ import org.apache.geode.internal.cache.tier.sockets.ConnectionListener;
 import org.apache.geode.internal.cache.tier.sockets.ServerConnectionFactory;
 import org.apache.geode.internal.net.SocketCreator;
 import org.apache.geode.internal.security.SecurityService;
+import org.apache.geode.internal.statistics.StatisticsClock;
 
 /**
  * Represents a {@link CacheServer} that is created declaratively.
@@ -309,4 +310,9 @@ public class CacheServerCreation extends AbstractCacheServer {
   public String[] getCombinedGroups() {
     throw new UnsupportedOperationException("Shouldn't be invoked");
   }
+
+  @Override
+  public StatisticsClock getStatisticsClock() {
+    throw new UnsupportedOperationException("Shouldn't be invoked");
+  }
 }
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/xmlcache/ParallelAsyncEventQueueCreation.java b/geode-core/src/main/java/org/apache/geode/internal/cache/xmlcache/ParallelAsyncEventQueueCreation.java
index 4a5fff1..ee06f69 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/xmlcache/ParallelAsyncEventQueueCreation.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/xmlcache/ParallelAsyncEventQueueCreation.java
@@ -14,6 +14,8 @@
  */
 package org.apache.geode.internal.cache.xmlcache;
 
+import static org.apache.geode.internal.statistics.StatisticsClockFactory.disabledClock;
+
 import java.util.List;
 
 import org.apache.geode.CancelCriterion;
@@ -33,7 +35,7 @@ public class ParallelAsyncEventQueueCreation extends AbstractGatewaySender
     implements GatewaySender {
 
   public ParallelAsyncEventQueueCreation(InternalCache cache, GatewaySenderAttributes attrs) {
-    super(cache, attrs);
+    super(cache, disabledClock(), attrs);
   }
 
   @Override
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/xmlcache/ParallelGatewaySenderCreation.java b/geode-core/src/main/java/org/apache/geode/internal/cache/xmlcache/ParallelGatewaySenderCreation.java
index 502015e..bfeb1f9 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/xmlcache/ParallelGatewaySenderCreation.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/xmlcache/ParallelGatewaySenderCreation.java
@@ -14,6 +14,8 @@
  */
 package org.apache.geode.internal.cache.xmlcache;
 
+import static org.apache.geode.internal.statistics.StatisticsClockFactory.disabledClock;
+
 import java.util.List;
 
 import org.apache.geode.CancelCriterion;
@@ -33,7 +35,7 @@ import org.apache.geode.internal.cache.wan.GatewaySenderAttributes;
 public class ParallelGatewaySenderCreation extends AbstractGatewaySender implements GatewaySender {
 
   public ParallelGatewaySenderCreation(InternalCache cache, GatewaySenderAttributes attrs) {
-    super(cache, attrs);
+    super(cache, disabledClock(), attrs);
   }
 
   @Override
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/xmlcache/SerialAsyncEventQueueCreation.java b/geode-core/src/main/java/org/apache/geode/internal/cache/xmlcache/SerialAsyncEventQueueCreation.java
index 8423a89..4582142 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/xmlcache/SerialAsyncEventQueueCreation.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/xmlcache/SerialAsyncEventQueueCreation.java
@@ -14,6 +14,8 @@
  */
 package org.apache.geode.internal.cache.xmlcache;
 
+import static org.apache.geode.internal.statistics.StatisticsClockFactory.disabledClock;
+
 import java.util.List;
 
 import org.apache.geode.CancelCriterion;
@@ -32,7 +34,7 @@ import org.apache.geode.internal.cache.wan.GatewaySenderAttributes;
 public class SerialAsyncEventQueueCreation extends AbstractGatewaySender implements GatewaySender {
 
   public SerialAsyncEventQueueCreation(InternalCache cache, GatewaySenderAttributes attrs) {
-    super(cache, attrs);
+    super(cache, disabledClock(), attrs);
   }
 
   @Override
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/xmlcache/SerialGatewaySenderCreation.java b/geode-core/src/main/java/org/apache/geode/internal/cache/xmlcache/SerialGatewaySenderCreation.java
index c92482f..155d3c6 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/xmlcache/SerialGatewaySenderCreation.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/xmlcache/SerialGatewaySenderCreation.java
@@ -14,6 +14,8 @@
  */
 package org.apache.geode.internal.cache.xmlcache;
 
+import static org.apache.geode.internal.statistics.StatisticsClockFactory.disabledClock;
+
 import java.util.List;
 
 import org.apache.geode.CancelCriterion;
@@ -33,7 +35,7 @@ import org.apache.geode.internal.cache.wan.GatewaySenderAttributes;
 public class SerialGatewaySenderCreation extends AbstractGatewaySender implements GatewaySender {
 
   public SerialGatewaySenderCreation(InternalCache cache, GatewaySenderAttributes attrs) {
-    super(cache, attrs);
+    super(cache, disabledClock(), attrs);
   }
 
   @Override
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/DistTXStateProxyImpl.java b/geode-core/src/main/java/org/apache/geode/internal/statistics/DisabledStatisticsClock.java
similarity index 52%
copy from geode-core/src/main/java/org/apache/geode/internal/cache/DistTXStateProxyImpl.java
copy to geode-core/src/main/java/org/apache/geode/internal/statistics/DisabledStatisticsClock.java
index 59aeaa2..47f2b85 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/DistTXStateProxyImpl.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/statistics/DisabledStatisticsClock.java
@@ -12,29 +12,17 @@
  * or implied. See the License for the specific language governing permissions and limitations under
  * the License.
  */
-package org.apache.geode.internal.cache;
+package org.apache.geode.internal.statistics;
 
-import org.apache.logging.log4j.Logger;
+public class DisabledStatisticsClock implements StatisticsClock {
 
-import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
-import org.apache.geode.internal.logging.LogService;
-
-public abstract class DistTXStateProxyImpl extends TXStateProxyImpl {
-
-  protected static final Logger logger = LogService.getLogger();
-
-  public DistTXStateProxyImpl(InternalCache cache, TXManagerImpl managerImpl, TXId id,
-      InternalDistributedMember clientMember) {
-    super(cache, managerImpl, id, clientMember);
-  }
-
-  public DistTXStateProxyImpl(InternalCache cache, TXManagerImpl managerImpl, TXId id,
-      boolean isjta) {
-    super(cache, managerImpl, id, isjta);
+  @Override
+  public long getTime() {
+    return 0;
   }
 
   @Override
-  public boolean isDistTx() {
-    return true;
+  public boolean isEnabled() {
+    return false;
   }
 }
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/DistTXStateProxyImpl.java b/geode-core/src/main/java/org/apache/geode/internal/statistics/EnabledStatisticsClock.java
similarity index 53%
copy from geode-core/src/main/java/org/apache/geode/internal/cache/DistTXStateProxyImpl.java
copy to geode-core/src/main/java/org/apache/geode/internal/statistics/EnabledStatisticsClock.java
index 59aeaa2..eb68953 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/DistTXStateProxyImpl.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/statistics/EnabledStatisticsClock.java
@@ -12,29 +12,19 @@
  * or implied. See the License for the specific language governing permissions and limitations under
  * the License.
  */
-package org.apache.geode.internal.cache;
+package org.apache.geode.internal.statistics;
 
-import org.apache.logging.log4j.Logger;
+import org.apache.geode.internal.NanoTimer;
 
-import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
-import org.apache.geode.internal.logging.LogService;
+public class EnabledStatisticsClock implements StatisticsClock {
 
-public abstract class DistTXStateProxyImpl extends TXStateProxyImpl {
-
-  protected static final Logger logger = LogService.getLogger();
-
-  public DistTXStateProxyImpl(InternalCache cache, TXManagerImpl managerImpl, TXId id,
-      InternalDistributedMember clientMember) {
-    super(cache, managerImpl, id, clientMember);
-  }
-
-  public DistTXStateProxyImpl(InternalCache cache, TXManagerImpl managerImpl, TXId id,
-      boolean isjta) {
-    super(cache, managerImpl, id, isjta);
+  @Override
+  public long getTime() {
+    return NanoTimer.getTime();
   }
 
   @Override
-  public boolean isDistTx() {
+  public boolean isEnabled() {
     return true;
   }
 }
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/DistTXStateProxyImpl.java b/geode-core/src/main/java/org/apache/geode/internal/statistics/StatisticsClock.java
similarity index 52%
copy from geode-core/src/main/java/org/apache/geode/internal/cache/DistTXStateProxyImpl.java
copy to geode-core/src/main/java/org/apache/geode/internal/statistics/StatisticsClock.java
index 59aeaa2..350bc4e 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/DistTXStateProxyImpl.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/statistics/StatisticsClock.java
@@ -12,29 +12,27 @@
  * or implied. See the License for the specific language governing permissions and limitations under
  * the License.
  */
-package org.apache.geode.internal.cache;
+package org.apache.geode.internal.statistics;
 
-import org.apache.logging.log4j.Logger;
+@FunctionalInterface
+public interface StatisticsClock {
 
-import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
-import org.apache.geode.internal.logging.LogService;
+  /**
+   * Returns the current value of the running Java Virtual Machine's high-resolution time source,
+   * in nanoseconds.
+   *
+   * <p>
+   * See {@code java.lang.System#nanoTime()}.
+   */
+  long getTime();
 
-public abstract class DistTXStateProxyImpl extends TXStateProxyImpl {
-
-  protected static final Logger logger = LogService.getLogger();
-
-  public DistTXStateProxyImpl(InternalCache cache, TXManagerImpl managerImpl, TXId id,
-      InternalDistributedMember clientMember) {
-    super(cache, managerImpl, id, clientMember);
-  }
-
-  public DistTXStateProxyImpl(InternalCache cache, TXManagerImpl managerImpl, TXId id,
-      boolean isjta) {
-    super(cache, managerImpl, id, isjta);
-  }
-
-  @Override
-  public boolean isDistTx() {
+  /**
+   * Returns true if this clock is enabled. If disabled then {@code getTime()} will return zero.
+   *
+   * <p>
+   * Default returns {@code true}.
+   */
+  default boolean isEnabled() {
     return true;
   }
 }
diff --git a/geode-core/src/main/java/org/apache/geode/internal/statistics/StatisticsClockFactory.java b/geode-core/src/main/java/org/apache/geode/internal/statistics/StatisticsClockFactory.java
new file mode 100644
index 0000000..d4234e3
--- /dev/null
+++ b/geode-core/src/main/java/org/apache/geode/internal/statistics/StatisticsClockFactory.java
@@ -0,0 +1,85 @@
+/*
+ * 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.geode.internal.statistics;
+
+import java.util.function.BooleanSupplier;
+import java.util.function.LongSupplier;
+
+import org.apache.geode.annotations.Immutable;
+
+public class StatisticsClockFactory {
+
+  @Immutable
+  public static final String ENABLE_CLOCK_STATS_PROPERTY = "enableClockStats";
+
+  @Immutable
+  public static final boolean enableClockStats = Boolean.getBoolean(ENABLE_CLOCK_STATS_PROPERTY);
+
+  /**
+   * TODO: delete getTimeIfEnabled
+   */
+  @Deprecated
+  public static long getTimeIfEnabled() {
+    return enableClockStats ? getTime() : 0;
+  }
+
+  public static long getTime() {
+    return System.nanoTime();
+  }
+
+  /**
+   * Creates new {@code StatisticsClock} using {@code enableClockStats} system property.
+   */
+  public static StatisticsClock clock() {
+    return clock(Boolean.getBoolean(ENABLE_CLOCK_STATS_PROPERTY));
+  }
+
+  /**
+   * Creates new {@code StatisticsClock} using specified boolean value.
+   */
+  public static StatisticsClock clock(boolean enabled) {
+    if (enabled) {
+      return enabledClock(() -> getTime());
+    }
+    return disabledClock();
+  }
+
+  public static StatisticsClock enabledClock() {
+    return clock(() -> getTime(), () -> true);
+  }
+
+  public static StatisticsClock enabledClock(LongSupplier time) {
+    return clock(() -> time.getAsLong(), () -> true);
+  }
+
+  public static StatisticsClock disabledClock() {
+    return clock(() -> 0, () -> false);
+  }
+
+  public static StatisticsClock clock(LongSupplier time, BooleanSupplier isEnabled) {
+    return new StatisticsClock() {
+      @Override
+      public long getTime() {
+        return time.getAsLong();
+      }
+
+      @Override
+      public boolean isEnabled() {
+        return isEnabled.getAsBoolean();
+      }
+    };
+  }
+
+}
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/DistTXStateProxyImpl.java b/geode-core/src/main/java/org/apache/geode/internal/statistics/StatisticsClockSupplier.java
similarity index 50%
copy from geode-core/src/main/java/org/apache/geode/internal/cache/DistTXStateProxyImpl.java
copy to geode-core/src/main/java/org/apache/geode/internal/statistics/StatisticsClockSupplier.java
index 59aeaa2..6dbfa3f 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/DistTXStateProxyImpl.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/statistics/StatisticsClockSupplier.java
@@ -12,29 +12,9 @@
  * or implied. See the License for the specific language governing permissions and limitations under
  * the License.
  */
-package org.apache.geode.internal.cache;
+package org.apache.geode.internal.statistics;
 
-import org.apache.logging.log4j.Logger;
+public interface StatisticsClockSupplier {
 
-import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
-import org.apache.geode.internal.logging.LogService;
-
-public abstract class DistTXStateProxyImpl extends TXStateProxyImpl {
-
-  protected static final Logger logger = LogService.getLogger();
-
-  public DistTXStateProxyImpl(InternalCache cache, TXManagerImpl managerImpl, TXId id,
-      InternalDistributedMember clientMember) {
-    super(cache, managerImpl, id, clientMember);
-  }
-
-  public DistTXStateProxyImpl(InternalCache cache, TXManagerImpl managerImpl, TXId id,
-      boolean isjta) {
-    super(cache, managerImpl, id, isjta);
-  }
-
-  @Override
-  public boolean isDistTx() {
-    return true;
-  }
+  StatisticsClock getStatisticsClock();
 }
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/FederatingManager.java b/geode-core/src/main/java/org/apache/geode/management/internal/FederatingManager.java
index 12e70ba..6689695 100755
--- a/geode-core/src/main/java/org/apache/geode/management/internal/FederatingManager.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/FederatingManager.java
@@ -31,6 +31,7 @@ import javax.management.ObjectName;
 
 import org.apache.logging.log4j.Logger;
 
+import org.apache.geode.StatisticsFactory;
 import org.apache.geode.annotations.VisibleForTesting;
 import org.apache.geode.cache.AttributesFactory;
 import org.apache.geode.cache.DataPolicy;
@@ -50,6 +51,7 @@ import org.apache.geode.internal.cache.InternalCache;
 import org.apache.geode.internal.cache.InternalRegionArguments;
 import org.apache.geode.internal.logging.LogService;
 import org.apache.geode.internal.logging.LoggingExecutors;
+import org.apache.geode.internal.statistics.StatisticsClock;
 import org.apache.geode.management.ManagementException;
 
 /**
@@ -84,9 +86,9 @@ public class FederatingManager extends Manager {
   private final AtomicReference<Exception> latestException = new AtomicReference<>(null);
 
   FederatingManager(MBeanJMXAdapter jmxAdapter, ManagementResourceRepo repo,
-      InternalDistributedSystem system, SystemManagementService service,
-      InternalCache cache) {
-    super(repo, system, cache);
+      InternalDistributedSystem system, SystemManagementService service, InternalCache cache,
+      StatisticsFactory statisticsFactory, StatisticsClock statisticsClock) {
+    super(repo, system, cache, statisticsFactory, statisticsClock);
     this.service = service;
     proxyFactory = new MBeanProxyFactory(jmxAdapter, service);
     messenger = new MemberMessenger(jmxAdapter, system);
@@ -214,7 +216,7 @@ public class FederatingManager extends Manager {
       notificationRegion.localDestroyRegion();
     }
 
-    if (!cache.getDistributedSystem().getDistributedMember().equals(member)) {
+    if (!system.getDistributedMember().equals(member)) {
       service.memberDeparted((InternalDistributedMember) member, crashed);
     }
   }
@@ -242,7 +244,7 @@ public class FederatingManager extends Manager {
 
     List<Callable<DistributedMember>> giiTaskList = new ArrayList<>();
 
-    for (DistributedMember member : cache.getDistributionManager()
+    for (DistributedMember member : system.getDistributionManager()
         .getOtherDistributionManagerIds()) {
       giiTaskList.add(new GIITask(member));
     }
@@ -357,7 +359,6 @@ public class FederatingManager extends Manager {
         String monitoringRegionName = ManagementConstants.MONITORING_REGION + "_" + appender;
         String notificationRegionName = ManagementConstants.NOTIFICATION_REGION + "_" + appender;
 
-
         if (cache.getInternalRegion(monitoringRegionName) != null
             && cache.getInternalRegion(notificationRegionName) != null) {
           return member;
@@ -375,7 +376,7 @@ public class FederatingManager extends Manager {
             // Create anonymous stats holder for Management Regions
             HasCachePerfStats monitoringRegionStats =
                 () -> new CachePerfStats(cache.getDistributedSystem(),
-                    "RegionStats-managementRegionStats");
+                    "RegionStats-managementRegionStats", statisticsClock);
 
             internalRegionArguments.setCachePerfStatsHolder(monitoringRegionStats);
 
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/LocalManager.java b/geode-core/src/main/java/org/apache/geode/management/internal/LocalManager.java
index bb0cc86..3f9c84e 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/LocalManager.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/LocalManager.java
@@ -31,6 +31,7 @@ import org.apache.logging.log4j.Logger;
 
 import org.apache.geode.CancelException;
 import org.apache.geode.GemFireException;
+import org.apache.geode.StatisticsFactory;
 import org.apache.geode.SystemFailure;
 import org.apache.geode.cache.AttributesFactory;
 import org.apache.geode.cache.DataPolicy;
@@ -45,6 +46,7 @@ import org.apache.geode.internal.cache.InternalCache;
 import org.apache.geode.internal.cache.InternalRegionArguments;
 import org.apache.geode.internal.logging.LogService;
 import org.apache.geode.internal.logging.LoggingExecutors;
+import org.apache.geode.internal.statistics.StatisticsClock;
 import org.apache.geode.management.ManagementException;
 
 /**
@@ -72,19 +74,20 @@ public class LocalManager extends Manager {
    * This map holds all the components which are eligible for federation. Although filters might
    * prevent any of the component from getting federated.
    */
-  private Map<ObjectName, FederationComponent> federatedComponentMap;
+  private final Map<ObjectName, FederationComponent> federatedComponentMap;
 
-  private Object lock = new Object();
+  private final Object lock = new Object();
 
-  private SystemManagementService service;
+  private final SystemManagementService service;
 
   /**
    * @param repo management resource repo
    * @param system internal distributed system
    */
   public LocalManager(ManagementResourceRepo repo, InternalDistributedSystem system,
-      SystemManagementService service, InternalCache cache) {
-    super(repo, system, cache);
+      SystemManagementService service, InternalCache cache, StatisticsFactory statisticsFactory,
+      StatisticsClock statisticsClock) {
+    super(repo, system, cache, statisticsFactory, statisticsClock);
     this.service = service;
     this.federatedComponentMap = new ConcurrentHashMap<ObjectName, FederationComponent>();
   }
@@ -115,13 +118,9 @@ public class LocalManager extends Manager {
         internalArgs.setIsUsedForMetaRegion(true);
 
         // Create anonymous stats holder for Management Regions
-        final HasCachePerfStats monitoringRegionStats = new HasCachePerfStats() {
-          @Override
-          public CachePerfStats getCachePerfStats() {
-            return new CachePerfStats(cache.getDistributedSystem(),
-                "RegionStats-managementRegionStats");
-          }
-        };
+        final HasCachePerfStats monitoringRegionStats =
+            () -> new CachePerfStats(cache.getDistributedSystem(),
+                "RegionStats-managementRegionStats", statisticsClock);
 
         internalArgs.setCachePerfStatsHolder(monitoringRegionStats);
 
@@ -145,8 +144,7 @@ public class LocalManager extends Manager {
         RegionAttributes<NotificationKey, Notification> notifRegionAttrs =
             notificationRegionAttributeFactory.create();
 
-        String appender = MBeanJMXAdapter
-            .getUniqueIDForMember(cache.getDistributedSystem().getDistributedMember());
+        String appender = MBeanJMXAdapter.getUniqueIDForMember(system.getDistributedMember());
 
         boolean monitoringRegionCreated = false;
         boolean notifRegionCreated = false;
@@ -192,7 +190,7 @@ public class LocalManager extends Manager {
         managementTask.run();
         // All local resources are created for the ManagementTask
         // Now Management tasks can proceed.
-        int updateRate = cache.getInternalDistributedSystem().getConfig().getJmxManagerUpdateRate();
+        int updateRate = system.getConfig().getJmxManagerUpdateRate();
         singleThreadFederationScheduler.scheduleAtFixedRate(managementTask, updateRate, updateRate,
             TimeUnit.MILLISECONDS);
 
@@ -296,7 +294,7 @@ public class LocalManager extends Manager {
    */
   private class ManagementTask implements Runnable {
 
-    private Map<String, FederationComponent> replicaMap;
+    private final Map<String, FederationComponent> replicaMap;
 
     public ManagementTask(Map<ObjectName, FederationComponent> federatedComponentMap)
         throws ManagementException {
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/Manager.java b/geode-core/src/main/java/org/apache/geode/management/internal/Manager.java
index 20ea590..5962d29 100755
--- a/geode-core/src/main/java/org/apache/geode/management/internal/Manager.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/Manager.java
@@ -14,9 +14,11 @@
  */
 package org.apache.geode.management.internal;
 
+import org.apache.geode.StatisticsFactory;
 import org.apache.geode.distributed.internal.InternalDistributedSystem;
 import org.apache.geode.internal.cache.InternalCache;
 import org.apache.geode.internal.cache.InternalCacheForClientAccess;
+import org.apache.geode.internal.statistics.StatisticsClock;
 
 /**
  * The Manager is a 7.0 JMX Agent which is hosted within a GemFire process. Only one instance is
@@ -27,7 +29,7 @@ import org.apache.geode.internal.cache.InternalCacheForClientAccess;
  */
 public abstract class Manager {
 
-  protected InternalCacheForClientAccess cache;
+  protected final InternalCacheForClientAccess cache;
 
   /**
    * depicts whether this node is a Managing node or not
@@ -42,18 +44,24 @@ public abstract class Manager {
   /**
    * This is a single window to manipulate region resources for management
    */
-  protected ManagementResourceRepo repo;
+  protected final ManagementResourceRepo repo;
 
   /**
    * The concrete implementation of DistributedSystem that provides internal-only functionality.
    */
-  protected InternalDistributedSystem system;
+  protected final InternalDistributedSystem system;
+
+  protected final StatisticsFactory statisticsFactory;
+
+  protected final StatisticsClock statisticsClock;
 
   public Manager(ManagementResourceRepo repo, InternalDistributedSystem system,
-      InternalCache cache) {
+      InternalCache cache, StatisticsFactory statisticsFactory, StatisticsClock statisticsClock) {
     this.repo = repo;
     this.cache = cache.getCacheForProcessingClientRequests();
     this.system = system;
+    this.statisticsFactory = statisticsFactory;
+    this.statisticsClock = statisticsClock;
   }
 
   public abstract boolean isRunning();
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/SystemManagementService.java b/geode-core/src/main/java/org/apache/geode/management/internal/SystemManagementService.java
index 023f182..79d99d4 100755
--- a/geode-core/src/main/java/org/apache/geode/management/internal/SystemManagementService.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/SystemManagementService.java
@@ -26,6 +26,7 @@ import javax.management.ObjectName;
 import org.apache.logging.log4j.Logger;
 
 import org.apache.geode.CancelException;
+import org.apache.geode.StatisticsFactory;
 import org.apache.geode.cache.execute.FunctionService;
 import org.apache.geode.distributed.DistributedMember;
 import org.apache.geode.distributed.DistributedSystemDisconnectedException;
@@ -35,6 +36,7 @@ import org.apache.geode.distributed.internal.membership.InternalDistributedMembe
 import org.apache.geode.internal.cache.InternalCache;
 import org.apache.geode.internal.cache.InternalCacheForClientAccess;
 import org.apache.geode.internal.logging.LogService;
+import org.apache.geode.internal.statistics.StatisticsClock;
 import org.apache.geode.management.AlreadyRunningException;
 import org.apache.geode.management.AsyncEventQueueMXBean;
 import org.apache.geode.management.CacheServerMXBean;
@@ -118,6 +120,9 @@ public class SystemManagementService extends BaseManagementService {
   private final UniversalListenerContainer universalListenerContainer =
       new UniversalListenerContainer();
 
+  private final StatisticsFactory statisticsFactory;
+  private final StatisticsClock statisticsClock;
+
   public static BaseManagementService newSystemManagementService(
       InternalCacheForClientAccess cache) {
     return new SystemManagementService(cache).init();
@@ -125,7 +130,7 @@ public class SystemManagementService extends BaseManagementService {
 
   protected SystemManagementService(InternalCacheForClientAccess cache) {
     this.cache = cache;
-    this.system = (InternalDistributedSystem) cache.getDistributedSystem();
+    this.system = cache.getInternalDistributedSystem();
     // This is a safe check to ensure Management service does not start for a
     // system which is disconnected.
     // Most likely scenario when this will happen is when a cache is closed and we are at this
@@ -135,6 +140,9 @@ public class SystemManagementService extends BaseManagementService {
           "This connection to a distributed system has been disconnected.");
     }
 
+    statisticsFactory = system.getStatisticsManager();
+    statisticsClock = cache.getStatisticsClock();
+
     this.jmxAdapter = new MBeanJMXAdapter(this.system.getDistributedMember());
     this.repo = new ManagementResourceRepo();
 
@@ -157,7 +165,8 @@ public class SystemManagementService extends BaseManagementService {
    */
   private SystemManagementService init() {
     try {
-      this.localManager = new LocalManager(repo, system, this, cache);
+      this.localManager =
+          new LocalManager(repo, system, this, cache, statisticsFactory, statisticsClock);
       this.localManager.startManager();
       this.listener = new ManagementMembershipListener(this);
       system.getDistributionManager().addMembershipListener(listener);
@@ -342,7 +351,7 @@ public class SystemManagementService extends BaseManagementService {
     if (!isStartedAndOpen()) {
       return Collections.emptySet();
     }
-    if (cache.getDistributedSystem().getDistributedMember().equals(member)) {
+    if (system.getDistributedMember().equals(member)) {
       return jmxAdapter.getLocalGemFireMBean().keySet();
     } else {
       if (federatingManager == null) {
@@ -462,7 +471,8 @@ public class SystemManagementService extends BaseManagementService {
       }
       system.handleResourceEvent(ResourceEvent.MANAGER_CREATE, null);
       // An initialised copy of federating manager
-      federatingManager = new FederatingManager(jmxAdapter, repo, system, this, cache);
+      federatingManager = new FederatingManager(jmxAdapter, repo, system, this, cache,
+          statisticsFactory, statisticsClock);
       getInternalCache().getJmxManagerAdvisor().broadcastChange();
       return true;
     }
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/beans/QueryDataFunction.java b/geode-core/src/main/java/org/apache/geode/management/internal/beans/QueryDataFunction.java
index d750eb3..7b64581 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/beans/QueryDataFunction.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/beans/QueryDataFunction.java
@@ -153,7 +153,8 @@ public class QueryDataFunction implements Function, InternalEntity {
             for (BucketRegion bRegion : localPrimaryBucketRegions) {
               localPrimaryBucketSet.add(bRegion.getId());
             }
-            LocalDataSet lds = new LocalDataSet(parRegion, localPrimaryBucketSet);
+            LocalDataSet lds =
+                new LocalDataSet(parRegion, localPrimaryBucketSet);
             DefaultQuery query = (DefaultQuery) cache.getQueryService().newQuery(queryString);
             final ExecutionContext executionContext = new QueryExecutionContext(null, cache, query);
             results = lds.executeQuery(query, executionContext, null, localPrimaryBucketSet);
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/AbstractDistributedRegionJUnitTest.java b/geode-core/src/test/java/org/apache/geode/internal/cache/AbstractDistributedRegionJUnitTest.java
index 92d3ecb..273a8a4 100755
--- a/geode-core/src/test/java/org/apache/geode/internal/cache/AbstractDistributedRegionJUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/internal/cache/AbstractDistributedRegionJUnitTest.java
@@ -14,6 +14,7 @@
  */
 package org.apache.geode.internal.cache;
 
+import static org.apache.geode.internal.statistics.StatisticsClockFactory.disabledClock;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
@@ -33,6 +34,7 @@ import org.apache.geode.cache.Scope;
 import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
 import org.apache.geode.internal.cache.versions.VersionTag;
 import org.apache.geode.internal.logging.LogService;
+import org.apache.geode.internal.statistics.StatisticsClock;
 import org.apache.geode.test.fake.Fakes;
 
 public abstract class AbstractDistributedRegionJUnitTest {
@@ -96,7 +98,8 @@ public abstract class AbstractDistributedRegionJUnitTest {
   protected abstract void setInternalRegionArguments(InternalRegionArguments ira);
 
   protected abstract DistributedRegion createAndDefineRegion(boolean isConcurrencyChecksEnabled,
-      RegionAttributes ra, InternalRegionArguments ira, GemFireCacheImpl cache);
+      RegionAttributes ra, InternalRegionArguments ira, GemFireCacheImpl cache,
+      StatisticsClock statisticsClock);
 
   protected abstract void verifyDistributeUpdate(DistributedRegion region, EntryEventImpl event,
       int cnt);
@@ -121,7 +124,8 @@ public abstract class AbstractDistributedRegionJUnitTest {
     setInternalRegionArguments(ira);
 
     // create a region object
-    DistributedRegion region = createAndDefineRegion(isConcurrencyChecksEnabled, ra, ira, cache);
+    DistributedRegion region =
+        createAndDefineRegion(isConcurrencyChecksEnabled, ra, ira, cache, disabledClock());
     if (isConcurrencyChecksEnabled) {
       region.enableConcurrencyChecks();
     }
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/AbstractRegionJUnitTest.java b/geode-core/src/test/java/org/apache/geode/internal/cache/AbstractRegionJUnitTest.java
index 09c7c5f..67f9355 100644
--- a/geode-core/src/test/java/org/apache/geode/internal/cache/AbstractRegionJUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/internal/cache/AbstractRegionJUnitTest.java
@@ -14,6 +14,7 @@
  */
 package org.apache.geode.internal.cache;
 
+import static org.apache.geode.internal.statistics.StatisticsClockFactory.disabledClock;
 import static org.assertj.core.api.Assertions.assertThat;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.spy;
@@ -118,7 +119,7 @@ public class AbstractRegionJUnitTest {
         (localRegion) -> mock(RegionPerfStats.class);
     AbstractRegion region = new LocalRegion("regionName", regionAttributes, null, Fakes.cache(),
         new InternalRegionArguments(), null, regionMapConstructor, null, null, null,
-        regionPerfStatsFactory);
+        regionPerfStatsFactory, disabledClock());
     return region;
   }
 }
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/BucketRegionJUnitTest.java b/geode-core/src/test/java/org/apache/geode/internal/cache/BucketRegionJUnitTest.java
index 8435c1b..72e6657 100755
--- a/geode-core/src/test/java/org/apache/geode/internal/cache/BucketRegionJUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/internal/cache/BucketRegionJUnitTest.java
@@ -32,6 +32,7 @@ import java.util.concurrent.locks.ReadWriteLock;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
 
 import org.apache.geode.cache.RegionAttributes;
+import org.apache.geode.internal.statistics.StatisticsClock;
 
 public class BucketRegionJUnitTest extends DistributedRegionJUnitTest {
 
@@ -51,8 +52,9 @@ public class BucketRegionJUnitTest extends DistributedRegionJUnitTest {
 
   @Override
   protected DistributedRegion createAndDefineRegion(boolean isConcurrencyChecksEnabled,
-      RegionAttributes ra, InternalRegionArguments ira, GemFireCacheImpl cache) {
-    BucketRegion br = new BucketRegion("testRegion", ra, null, cache, ira);
+      RegionAttributes ra, InternalRegionArguments ira, GemFireCacheImpl cache,
+      StatisticsClock statisticsClock) {
+    BucketRegion br = new BucketRegion("testRegion", ra, null, cache, ira, statisticsClock);
     // it is necessary to set the event tracker to initialized, since initialize() in not being
     // called on the instantiated region
     br.getEventTracker().setInitialized();
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/BucketRegionTest.java b/geode-core/src/test/java/org/apache/geode/internal/cache/BucketRegionTest.java
index 4708071..b8e392d 100644
--- a/geode-core/src/test/java/org/apache/geode/internal/cache/BucketRegionTest.java
+++ b/geode-core/src/test/java/org/apache/geode/internal/cache/BucketRegionTest.java
@@ -14,6 +14,7 @@
  */
 package org.apache.geode.internal.cache;
 
+import static org.apache.geode.internal.statistics.StatisticsClockFactory.disabledClock;
 import static org.assertj.core.api.Assertions.assertThat;
 import static org.mockito.ArgumentMatchers.eq;
 import static org.mockito.Mockito.RETURNS_DEEP_STUBS;
@@ -50,6 +51,7 @@ import org.apache.geode.internal.cache.tier.sockets.CacheClientProxy;
 import org.apache.geode.internal.cache.tier.sockets.CacheServerStats;
 import org.apache.geode.internal.cache.tier.sockets.ClientProxyMembershipID;
 import org.apache.geode.internal.cache.tier.sockets.ConnectionListener;
+import org.apache.geode.internal.statistics.StatisticsClock;
 import org.apache.geode.test.fake.Fakes;
 
 public class BucketRegionTest {
@@ -113,7 +115,7 @@ public class BucketRegionTest {
   public void waitUntilLockedThrowsIfFoundLockAndPartitionedRegionIsClosing() {
     BucketRegion bucketRegion =
         spy(new BucketRegion(regionName, regionAttributes, partitionedRegion,
-            cache, internalRegionArgs));
+            cache, internalRegionArgs, disabledClock()));
     Integer[] keys = {1};
     doReturn(mock(LockObject.class)).when(bucketRegion).searchAndLock(keys);
     doThrow(regionDestroyedException).when(partitionedRegion)
@@ -126,7 +128,7 @@ public class BucketRegionTest {
   public void waitUntilLockedReturnsTrueIfNoOtherThreadLockedKeys() {
     BucketRegion bucketRegion =
         spy(new BucketRegion(regionName, regionAttributes, partitionedRegion,
-            cache, internalRegionArgs));
+            cache, internalRegionArgs, disabledClock()));
     Integer[] keys = {1};
     doReturn(null).when(bucketRegion).searchAndLock(keys);
 
@@ -137,7 +139,7 @@ public class BucketRegionTest {
   public void basicPutEntryDoesNotReleaseLockIfKeysAndPrimaryNotLocked() {
     BucketRegion bucketRegion =
         spy(new BucketRegion(regionName, regionAttributes, partitionedRegion,
-            cache, internalRegionArgs));
+            cache, internalRegionArgs, disabledClock()));
     doThrow(regionDestroyedException).when(bucketRegion).lockKeysAndPrimary(event);
 
     bucketRegion.basicPutEntry(event, 1);
@@ -149,7 +151,7 @@ public class BucketRegionTest {
   public void basicPutEntryReleaseLockIfKeysAndPrimaryLocked() {
     BucketRegion bucketRegion =
         spy(new BucketRegion(regionName, regionAttributes, partitionedRegion,
-            cache, internalRegionArgs));
+            cache, internalRegionArgs, disabledClock()));
     doReturn(true).when(bucketRegion).lockKeysAndPrimary(event);
     doReturn(mock(AbstractRegionMap.class)).when(bucketRegion).getRegionMap();
 
@@ -162,7 +164,7 @@ public class BucketRegionTest {
   public void virtualPutDoesNotReleaseLockIfKeysAndPrimaryNotLocked() {
     BucketRegion bucketRegion =
         spy(new BucketRegion(regionName, regionAttributes, partitionedRegion,
-            cache, internalRegionArgs));
+            cache, internalRegionArgs, disabledClock()));
     doThrow(regionDestroyedException).when(bucketRegion).lockKeysAndPrimary(event);
 
     bucketRegion.virtualPut(event, false, true, null, false, 1, true);
@@ -174,7 +176,7 @@ public class BucketRegionTest {
   public void virtualPutReleaseLockIfKeysAndPrimaryLocked() {
     BucketRegion bucketRegion =
         spy(new BucketRegion(regionName, regionAttributes, partitionedRegion,
-            cache, internalRegionArgs));
+            cache, internalRegionArgs, disabledClock()));
     doReturn(true).when(bucketRegion).lockKeysAndPrimary(event);
     doReturn(true).when(bucketRegion).hasSeenEvent(event);
 
@@ -187,7 +189,7 @@ public class BucketRegionTest {
   public void basicDestroyDoesNotReleaseLockIfKeysAndPrimaryNotLocked() {
     BucketRegion bucketRegion =
         spy(new BucketRegion(regionName, regionAttributes, partitionedRegion,
-            cache, internalRegionArgs));
+            cache, internalRegionArgs, disabledClock()));
     doThrow(regionDestroyedException).when(bucketRegion).lockKeysAndPrimary(event);
 
     bucketRegion.basicDestroy(event, false, null);
@@ -199,7 +201,7 @@ public class BucketRegionTest {
   public void basicDestroyReleaseLockIfKeysAndPrimaryLocked() {
     BucketRegion bucketRegion =
         spy(new BucketRegion(regionName, regionAttributes, partitionedRegion,
-            cache, internalRegionArgs));
+            cache, internalRegionArgs, disabledClock()));
     doReturn(true).when(bucketRegion).lockKeysAndPrimary(event);
     doReturn(true).when(bucketRegion).hasSeenEvent(event);
 
@@ -212,7 +214,7 @@ public class BucketRegionTest {
   public void basicUpdateEntryVersionDoesNotReleaseLockIfKeysAndPrimaryNotLocked() {
     BucketRegion bucketRegion =
         spy(new BucketRegion(regionName, regionAttributes, partitionedRegion,
-            cache, internalRegionArgs));
+            cache, internalRegionArgs, disabledClock()));
     doThrow(regionDestroyedException).when(bucketRegion).lockKeysAndPrimary(event);
     when(event.getRegion()).thenReturn(bucketRegion);
     doReturn(true).when(bucketRegion).hasSeenEvent(event);
@@ -227,7 +229,7 @@ public class BucketRegionTest {
   public void basicUpdateEntryVersionReleaseLockIfKeysAndPrimaryLocked() {
     BucketRegion bucketRegion =
         spy(new BucketRegion(regionName, regionAttributes, partitionedRegion,
-            cache, internalRegionArgs));
+            cache, internalRegionArgs, disabledClock()));
     doReturn(true).when(bucketRegion).lockKeysAndPrimary(event);
     when(event.getRegion()).thenReturn(bucketRegion);
     doReturn(true).when(bucketRegion).hasSeenEvent(event);
@@ -242,7 +244,7 @@ public class BucketRegionTest {
   public void basicInvalidateDoesNotReleaseLockIfKeysAndPrimaryNotLocked() {
     BucketRegion bucketRegion =
         spy(new BucketRegion(regionName, regionAttributes, partitionedRegion,
-            cache, internalRegionArgs));
+            cache, internalRegionArgs, disabledClock()));
     doThrow(regionDestroyedException).when(bucketRegion).lockKeysAndPrimary(event);
 
     bucketRegion.basicInvalidate(event, false, false);
@@ -254,7 +256,7 @@ public class BucketRegionTest {
   public void basicInvalidateReleaseLockIfKeysAndPrimaryLocked() {
     BucketRegion bucketRegion =
         spy(new BucketRegion(regionName, regionAttributes, partitionedRegion,
-            cache, internalRegionArgs));
+            cache, internalRegionArgs, disabledClock()));
     doReturn(true).when(bucketRegion).lockKeysAndPrimary(event);
     doReturn(true).when(bucketRegion).hasSeenEvent(event);
 
@@ -267,7 +269,7 @@ public class BucketRegionTest {
   public void lockKeysAndPrimaryReturnFalseIfDoesNotNeedWriteLock() {
     BucketRegion bucketRegion =
         spy(new BucketRegion(regionName, regionAttributes, partitionedRegion,
-            cache, internalRegionArgs));
+            cache, internalRegionArgs, disabledClock()));
     doReturn(false).when(bucketRegion).needWriteLock(event);
 
     assertThat(bucketRegion.lockKeysAndPrimary(event)).isFalse();
@@ -277,7 +279,7 @@ public class BucketRegionTest {
   public void lockKeysAndPrimaryThrowsIfWaitUntilLockedThrows() {
     BucketRegion bucketRegion =
         spy(new BucketRegion(regionName, regionAttributes, partitionedRegion,
-            cache, internalRegionArgs));
+            cache, internalRegionArgs, disabledClock()));
     doReturn(keys).when(bucketRegion).getKeysToBeLocked(event);
     doThrow(regionDestroyedException).when(bucketRegion).waitUntilLocked(keys);
 
@@ -288,7 +290,7 @@ public class BucketRegionTest {
   public void lockKeysAndPrimaryReleaseLockHeldIfDoLockForPrimaryThrows() {
     BucketRegion bucketRegion =
         spy(new BucketRegion(regionName, regionAttributes, partitionedRegion,
-            cache, internalRegionArgs));
+            cache, internalRegionArgs, disabledClock()));
     doReturn(keys).when(bucketRegion).getKeysToBeLocked(event);
     doReturn(true).when(bucketRegion).waitUntilLocked(keys);
     doThrow(new PrimaryBucketException()).when(bucketRegion).doLockForPrimary(false);
@@ -302,7 +304,7 @@ public class BucketRegionTest {
   public void lockKeysAndPrimaryReleaseLockHeldIfDoesNotLockForPrimary() {
     BucketRegion bucketRegion =
         spy(new BucketRegion(regionName, regionAttributes, partitionedRegion,
-            cache, internalRegionArgs));
+            cache, internalRegionArgs, disabledClock()));
     doReturn(keys).when(bucketRegion).getKeysToBeLocked(event);
     doReturn(true).when(bucketRegion).waitUntilLocked(keys);
     doReturn(true).when(bucketRegion).doLockForPrimary(false);
@@ -317,7 +319,7 @@ public class BucketRegionTest {
 
     BucketRegion bucketRegion =
         spy(new BucketRegion(regionName, regionAttributes, partitionedRegion,
-            cache, internalRegionArgs));
+            cache, internalRegionArgs, disabledClock()));
 
     Map regionGCVersions = new HashMap();
     Set keysRemoved = new HashSet();
@@ -333,7 +335,7 @@ public class BucketRegionTest {
   public void testDoNotNotifyClientsOfTombstoneGCNoProxy() {
     BucketRegion bucketRegion =
         spy(new BucketRegion(regionName, regionAttributes, partitionedRegion,
-            cache, internalRegionArgs));
+            cache, internalRegionArgs, disabledClock()));
 
     Map regionGCVersions = new HashMap();
     Set keysRemoved = new HashSet();
@@ -342,8 +344,8 @@ public class BucketRegionTest {
     doReturn(mock(SystemTimer.class)).when(cache).getCCPTimer();
 
     CacheClientNotifier ccn =
-        CacheClientNotifier.getInstance(cache, mock(CacheServerStats.class), 10,
-            10, mock(ConnectionListener.class), null, true);
+        CacheClientNotifier.getInstance(cache, mock(StatisticsClock.class),
+            mock(CacheServerStats.class), 10, 10, mock(ConnectionListener.class), null, true);
 
     bucketRegion.notifyClientsOfTombstoneGC(regionGCVersions, keysRemoved, eventID, routing);
     verify(bucketRegion, never()).getFilterProfile();
@@ -356,7 +358,7 @@ public class BucketRegionTest {
   public void testNotifyClientsOfTombstoneGC() {
     BucketRegion bucketRegion =
         spy(new BucketRegion(regionName, regionAttributes, partitionedRegion,
-            cache, internalRegionArgs));
+            cache, internalRegionArgs, disabledClock()));
 
     Map regionGCVersions = new HashMap();
     Set keysRemoved = new HashSet();
@@ -365,8 +367,8 @@ public class BucketRegionTest {
     doReturn(mock(SystemTimer.class)).when(cache).getCCPTimer();
 
     CacheClientNotifier ccn =
-        CacheClientNotifier.getInstance(cache, mock(CacheServerStats.class), 10,
-            10, mock(ConnectionListener.class), null, true);
+        CacheClientNotifier.getInstance(cache, mock(StatisticsClock.class),
+            mock(CacheServerStats.class), 10, 10, mock(ConnectionListener.class), null, true);
 
     doReturn(mock(ClientProxyMembershipID.class)).when(proxy).getProxyID();
     ccn.addClientProxyToMap(proxy);
@@ -384,7 +386,7 @@ public class BucketRegionTest {
   public void invokeTXCallbacksDoesNotInvokeCallbacksIfEventIsNotGenerateCallbacks() {
     BucketRegion bucketRegion =
         spy(new BucketRegion(regionName, regionAttributes, partitionedRegion,
-            cache, internalRegionArgs));
+            cache, internalRegionArgs, disabledClock()));
     doReturn(false).when(bucketRegion).isInitialized();
     doReturn(false).when(event).isGenerateCallbacks();
 
@@ -398,7 +400,7 @@ public class BucketRegionTest {
   public void invokeTXCallbacksDoesNotInvokeCallbacksIfPartitionedRegionIsNotInitialized() {
     BucketRegion bucketRegion =
         spy(new BucketRegion(regionName, regionAttributes, partitionedRegion,
-            cache, internalRegionArgs));
+            cache, internalRegionArgs, disabledClock()));
     doReturn(false).when(bucketRegion).isInitialized();
     doReturn(true).when(event).isGenerateCallbacks();
     doReturn(false).when(partitionedRegion).isInitialized();
@@ -414,7 +416,7 @@ public class BucketRegionTest {
   public void invokeTXCallbacksIsInvoked() {
     BucketRegion bucketRegion =
         spy(new BucketRegion(regionName, regionAttributes, partitionedRegion,
-            cache, internalRegionArgs));
+            cache, internalRegionArgs, disabledClock()));
     doReturn(false).when(bucketRegion).isInitialized();
     doReturn(true).when(event).isGenerateCallbacks();
     doReturn(true).when(partitionedRegion).isInitialized();
@@ -431,7 +433,7 @@ public class BucketRegionTest {
   public void invokeDestroyCallbacksDoesNotInvokeCallbacksIfEventIsNotGenerateCallbacks() {
     BucketRegion bucketRegion =
         spy(new BucketRegion(regionName, regionAttributes, partitionedRegion,
-            cache, internalRegionArgs));
+            cache, internalRegionArgs, disabledClock()));
     doReturn(false).when(bucketRegion).isInitialized();
     doReturn(false).when(event).isGenerateCallbacks();
 
@@ -445,7 +447,7 @@ public class BucketRegionTest {
   public void invokeDestroyCallbacksDoesNotInvokeCallbacksIfPartitionedRegionIsNotInitialized() {
     BucketRegion bucketRegion =
         spy(new BucketRegion(regionName, regionAttributes, partitionedRegion,
-            cache, internalRegionArgs));
+            cache, internalRegionArgs, disabledClock()));
     doReturn(false).when(bucketRegion).isInitialized();
     doReturn(true).when(event).isGenerateCallbacks();
     doReturn(false).when(partitionedRegion).isInitialized();
@@ -461,7 +463,7 @@ public class BucketRegionTest {
   public void invokeDestroyCallbacksIsInvoked() {
     BucketRegion bucketRegion =
         spy(new BucketRegion(regionName, regionAttributes, partitionedRegion,
-            cache, internalRegionArgs));
+            cache, internalRegionArgs, disabledClock()));
     doReturn(false).when(bucketRegion).isInitialized();
     doReturn(true).when(event).isGenerateCallbacks();
     doReturn(true).when(partitionedRegion).isInitialized();
@@ -478,7 +480,7 @@ public class BucketRegionTest {
   public void invokeInvalidateCallbacksDoesNotInvokeCallbacksIfEventIsNotGenerateCallbacks() {
     BucketRegion bucketRegion =
         spy(new BucketRegion(regionName, regionAttributes, partitionedRegion,
-            cache, internalRegionArgs));
+            cache, internalRegionArgs, disabledClock()));
     doReturn(false).when(bucketRegion).isInitialized();
     doReturn(false).when(event).isGenerateCallbacks();
 
@@ -492,7 +494,7 @@ public class BucketRegionTest {
   public void invokeInvalidateCallbacksDoesNotInvokeCallbacksIfPartitionedRegionIsNotInitialized() {
     BucketRegion bucketRegion =
         spy(new BucketRegion(regionName, regionAttributes, partitionedRegion,
-            cache, internalRegionArgs));
+            cache, internalRegionArgs, disabledClock()));
     doReturn(false).when(bucketRegion).isInitialized();
     doReturn(true).when(event).isGenerateCallbacks();
     doReturn(false).when(partitionedRegion).isInitialized();
@@ -508,7 +510,7 @@ public class BucketRegionTest {
   public void invokeInvalidateCallbacksIsInvoked() {
     BucketRegion bucketRegion =
         spy(new BucketRegion(regionName, regionAttributes, partitionedRegion,
-            cache, internalRegionArgs));
+            cache, internalRegionArgs, disabledClock()));
     doReturn(false).when(bucketRegion).isInitialized();
     doReturn(true).when(event).isGenerateCallbacks();
     doReturn(true).when(partitionedRegion).isInitialized();
@@ -525,7 +527,7 @@ public class BucketRegionTest {
   public void invokePutCallbacksDoesNotInvokeCallbacksIfEventIsNotGenerateCallbacks() {
     BucketRegion bucketRegion =
         spy(new BucketRegion(regionName, regionAttributes, partitionedRegion,
-            cache, internalRegionArgs));
+            cache, internalRegionArgs, disabledClock()));
     doReturn(false).when(bucketRegion).isInitialized();
     doReturn(false).when(event).isGenerateCallbacks();
 
@@ -539,7 +541,7 @@ public class BucketRegionTest {
   public void invokePutCallbacksDoesNotInvokeCallbacksIfPartitionedRegionIsNotInitialized() {
     BucketRegion bucketRegion =
         spy(new BucketRegion(regionName, regionAttributes, partitionedRegion,
-            cache, internalRegionArgs));
+            cache, internalRegionArgs, disabledClock()));
     doReturn(false).when(bucketRegion).isInitialized();
     doReturn(true).when(event).isGenerateCallbacks();
     doReturn(false).when(partitionedRegion).isInitialized();
@@ -555,7 +557,7 @@ public class BucketRegionTest {
   public void invokePutCallbacksIsInvoked() {
     BucketRegion bucketRegion =
         spy(new BucketRegion(regionName, regionAttributes, partitionedRegion,
-            cache, internalRegionArgs));
+            cache, internalRegionArgs, disabledClock()));
     doReturn(false).when(bucketRegion).isInitialized();
     doReturn(true).when(event).isGenerateCallbacks();
     doReturn(true).when(partitionedRegion).isInitialized();
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/CachePerfStatsTest.java b/geode-core/src/test/java/org/apache/geode/internal/cache/CachePerfStatsTest.java
index 3ae5ee2..ef96053 100644
--- a/geode-core/src/test/java/org/apache/geode/internal/cache/CachePerfStatsTest.java
+++ b/geode-core/src/test/java/org/apache/geode/internal/cache/CachePerfStatsTest.java
@@ -56,7 +56,6 @@ import static org.mockito.Mockito.eq;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
 
-import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
 
@@ -89,13 +88,7 @@ public class CachePerfStatsTest {
     when(statisticsFactory.createAtomicStatistics(eq(statisticsType), eq(TEXT_ID)))
         .thenReturn(statistics);
 
-    CachePerfStats.enableClockStats = true;
-    cachePerfStats = new CachePerfStats(statisticsFactory, () -> CLOCK_TIME);
-  }
-
-  @After
-  public void tearDown() {
-    CachePerfStats.enableClockStats = false;
+    cachePerfStats = new CachePerfStats(statisticsFactory, TEXT_ID, () -> CLOCK_TIME);
   }
 
   @Test
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/CacheServerImplTest.java b/geode-core/src/test/java/org/apache/geode/internal/cache/CacheServerImplTest.java
index b8d5362..b0d365a 100644
--- a/geode-core/src/test/java/org/apache/geode/internal/cache/CacheServerImplTest.java
+++ b/geode-core/src/test/java/org/apache/geode/internal/cache/CacheServerImplTest.java
@@ -43,6 +43,7 @@ import org.apache.geode.internal.cache.tier.sockets.CacheClientNotifier;
 import org.apache.geode.internal.cache.tier.sockets.ClientHealthMonitor;
 import org.apache.geode.internal.net.SocketCreator;
 import org.apache.geode.internal.security.SecurityService;
+import org.apache.geode.internal.statistics.StatisticsClockFactory;
 import org.apache.geode.internal.statistics.StatisticsManager;
 import org.apache.geode.test.junit.categories.ClientServerTest;
 
@@ -88,8 +89,9 @@ public class CacheServerImplTest {
   @Test
   public void createdAcceptorIsGatewayEndpoint() throws IOException {
     OverflowAttributes overflowAttributes = mock(OverflowAttributes.class);
-    InternalCacheServer server = new CacheServerImpl(cache, securityService, new AcceptorBuilder(),
-        true, true, () -> socketCreator, (a, b, c, d, e, f, g) -> cacheClientNotifier,
+    InternalCacheServer server = new CacheServerImpl(cache, securityService,
+        StatisticsClockFactory.disabledClock(), new AcceptorBuilder(),
+        true, true, () -> socketCreator, (a, b, c, d, e, f, g, h) -> cacheClientNotifier,
         (a, b, c) -> clientHealthMonitor, a -> advisor);
 
     Acceptor acceptor = server.createAcceptor(overflowAttributes);
@@ -99,8 +101,9 @@ public class CacheServerImplTest {
 
   @Test
   public void getGroups_returnsSpecifiedGroup() {
-    InternalCacheServer server = new CacheServerImpl(cache, securityService, new AcceptorBuilder(),
-        true, true, () -> socketCreator, (a, b, c, d, e, f, g) -> cacheClientNotifier,
+    InternalCacheServer server = new CacheServerImpl(cache, securityService,
+        StatisticsClockFactory.disabledClock(), new AcceptorBuilder(),
+        true, true, () -> socketCreator, (a, b, c, d, e, f, g, h) -> cacheClientNotifier,
         (a, b, c) -> clientHealthMonitor, a -> advisor);
     String specifiedGroup = "group0";
 
@@ -112,8 +115,9 @@ public class CacheServerImplTest {
 
   @Test
   public void getGroups_returnsMultipleSpecifiedGroups() {
-    InternalCacheServer server = new CacheServerImpl(cache, securityService, new AcceptorBuilder(),
-        true, true, () -> socketCreator, (a, b, c, d, e, f, g) -> cacheClientNotifier,
+    InternalCacheServer server = new CacheServerImpl(cache, securityService,
+        StatisticsClockFactory.disabledClock(), new AcceptorBuilder(),
+        true, true, () -> socketCreator, (a, b, c, d, e, f, g, h) -> cacheClientNotifier,
         (a, b, c) -> clientHealthMonitor, a -> advisor);
     String specifiedGroup1 = "group1";
     String specifiedGroup2 = "group2";
@@ -129,8 +133,9 @@ public class CacheServerImplTest {
   public void getCombinedGroups_includesMembershipGroup() {
     String membershipGroup = "group-m0";
     when(config.getGroups()).thenReturn(membershipGroup);
-    InternalCacheServer server = new CacheServerImpl(cache, securityService, new AcceptorBuilder(),
-        true, true, () -> socketCreator, (a, b, c, d, e, f, g) -> cacheClientNotifier,
+    InternalCacheServer server = new CacheServerImpl(cache, securityService,
+        StatisticsClockFactory.disabledClock(), new AcceptorBuilder(),
+        true, true, () -> socketCreator, (a, b, c, d, e, f, g, h) -> cacheClientNotifier,
         (a, b, c) -> clientHealthMonitor, a -> advisor);
 
     assertThat(server.getCombinedGroups())
@@ -144,8 +149,9 @@ public class CacheServerImplTest {
     String membershipGroup3 = "group-m3";
     when(config.getGroups())
         .thenReturn(membershipGroup1 + "," + membershipGroup2 + "," + membershipGroup3);
-    InternalCacheServer server = new CacheServerImpl(cache, securityService, new AcceptorBuilder(),
-        true, true, () -> socketCreator, (a, b, c, d, e, f, g) -> cacheClientNotifier,
+    InternalCacheServer server = new CacheServerImpl(cache, securityService,
+        StatisticsClockFactory.disabledClock(), new AcceptorBuilder(),
+        true, true, () -> socketCreator, (a, b, c, d, e, f, g, h) -> cacheClientNotifier,
         (a, b, c) -> clientHealthMonitor, a -> advisor);
 
     assertThat(server.getCombinedGroups())
@@ -159,8 +165,9 @@ public class CacheServerImplTest {
     String membershipGroup3 = "group-m3";
     when(config.getGroups())
         .thenReturn(membershipGroup1 + "," + membershipGroup2 + "," + membershipGroup3);
-    InternalCacheServer server = new CacheServerImpl(cache, securityService, new AcceptorBuilder(),
-        true, true, () -> socketCreator, (a, b, c, d, e, f, g) -> cacheClientNotifier,
+    InternalCacheServer server = new CacheServerImpl(cache, securityService,
+        StatisticsClockFactory.disabledClock(), new AcceptorBuilder(),
+        true, true, () -> socketCreator, (a, b, c, d, e, f, g, h) -> cacheClientNotifier,
         (a, b, c) -> clientHealthMonitor, a -> advisor);
     String specifiedGroup1 = "group1";
     String specifiedGroup2 = "group2";
@@ -175,8 +182,9 @@ public class CacheServerImplTest {
 
   @Test
   public void startNotifiesResourceEventCacheServerStart() throws IOException {
-    InternalCacheServer server = new CacheServerImpl(cache, securityService, new AcceptorBuilder(),
-        true, true, () -> socketCreator, (a, b, c, d, e, f, g) -> cacheClientNotifier,
+    InternalCacheServer server = new CacheServerImpl(cache, securityService,
+        StatisticsClockFactory.disabledClock(), new AcceptorBuilder(),
+        true, true, () -> socketCreator, (a, b, c, d, e, f, g, h) -> cacheClientNotifier,
         (a, b, c) -> clientHealthMonitor, a -> advisor);
 
     server.start();
@@ -186,8 +194,9 @@ public class CacheServerImplTest {
 
   @Test
   public void stopNotifiesResourceEventCacheServerStart() throws IOException {
-    InternalCacheServer server = new CacheServerImpl(cache, securityService, new AcceptorBuilder(),
-        true, true, () -> socketCreator, (a, b, c, d, e, f, g) -> cacheClientNotifier,
+    InternalCacheServer server = new CacheServerImpl(cache, securityService,
+        StatisticsClockFactory.disabledClock(), new AcceptorBuilder(),
+        true, true, () -> socketCreator, (a, b, c, d, e, f, g, h) -> cacheClientNotifier,
         (a, b, c) -> clientHealthMonitor, a -> advisor);
     server.start();
 
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/DistributedRegionJUnitTest.java b/geode-core/src/test/java/org/apache/geode/internal/cache/DistributedRegionJUnitTest.java
index 25ebdfa..9fbd8fc 100755
--- a/geode-core/src/test/java/org/apache/geode/internal/cache/DistributedRegionJUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/internal/cache/DistributedRegionJUnitTest.java
@@ -14,6 +14,7 @@
  */
 package org.apache.geode.internal.cache;
 
+import static org.apache.geode.internal.statistics.StatisticsClockFactory.disabledClock;
 import static org.assertj.core.api.Assertions.assertThat;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
@@ -44,6 +45,7 @@ import org.apache.geode.internal.cache.ha.ThreadIdentifier;
 import org.apache.geode.internal.cache.tier.sockets.ClientProxyMembershipID;
 import org.apache.geode.internal.cache.versions.VMVersionTag;
 import org.apache.geode.internal.cache.versions.VersionTag;
+import org.apache.geode.internal.statistics.StatisticsClock;
 
 public class DistributedRegionJUnitTest
     extends AbstractDistributedRegionJUnitTest {
@@ -52,11 +54,11 @@ public class DistributedRegionJUnitTest
   protected void setInternalRegionArguments(InternalRegionArguments ira) {}
 
   @Override
-  protected DistributedRegion createAndDefineRegion(
-      boolean isConcurrencyChecksEnabled,
-      RegionAttributes ra, InternalRegionArguments ira, GemFireCacheImpl cache) {
+  protected DistributedRegion createAndDefineRegion(boolean isConcurrencyChecksEnabled,
+      RegionAttributes ra, InternalRegionArguments ira, GemFireCacheImpl cache,
+      StatisticsClock statisticsClock) {
     DistributedRegion region = new DistributedRegion("testRegion", ra, null,
-        cache, ira);
+        cache, ira, disabledClock());
     if (isConcurrencyChecksEnabled) {
       region.enableConcurrencyChecks();
     }
@@ -222,9 +224,7 @@ public class DistributedRegionJUnitTest
     byte[] memId = {1, 2, 3};
     long threadId = 1;
     long retrySeqId = 1;
-    ThreadIdentifier tid = new ThreadIdentifier(memId, threadId);
     EventID retryEventID = new EventID(memId, threadId, retrySeqId);
-    boolean skipCallbacks = true;
 
     final EventIDHolder clientEvent = new EventIDHolder(retryEventID);
     clientEvent.setOperation(Operation.UPDATE);
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/DistributedRegionSearchLoadJUnitTest.java b/geode-core/src/test/java/org/apache/geode/internal/cache/DistributedRegionSearchLoadJUnitTest.java
index 7c6b0c8..32de9f3 100755
--- a/geode-core/src/test/java/org/apache/geode/internal/cache/DistributedRegionSearchLoadJUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/internal/cache/DistributedRegionSearchLoadJUnitTest.java
@@ -14,6 +14,7 @@
  */
 package org.apache.geode.internal.cache;
 
+import static org.apache.geode.internal.statistics.StatisticsClockFactory.disabledClock;
 import static org.junit.Assert.assertNotNull;
 import static org.mockito.Matchers.any;
 import static org.mockito.Matchers.anyLong;
@@ -54,7 +55,8 @@ public class DistributedRegionSearchLoadJUnitTest {
 
   protected DistributedRegion createAndDefineRegion(boolean isConcurrencyChecksEnabled,
       RegionAttributes ra, InternalRegionArguments ira, GemFireCacheImpl cache) {
-    DistributedRegion region = new DistributedRegion("testRegion", ra, null, cache, ira);
+    DistributedRegion region =
+        new DistributedRegion("testRegion", ra, null, cache, ira, disabledClock());
     if (isConcurrencyChecksEnabled) {
       region.enableConcurrencyChecks();
     }
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/LocalRegionPartialMockTest.java b/geode-core/src/test/java/org/apache/geode/internal/cache/LocalRegionPartialMockTest.java
index c345d6b..c15eb20 100644
--- a/geode-core/src/test/java/org/apache/geode/internal/cache/LocalRegionPartialMockTest.java
+++ b/geode-core/src/test/java/org/apache/geode/internal/cache/LocalRegionPartialMockTest.java
@@ -17,6 +17,7 @@ package org.apache.geode.internal.cache;
 import static org.apache.geode.internal.cache.LocalRegion.InitializationLevel.AFTER_INITIAL_IMAGE;
 import static org.apache.geode.internal.cache.LocalRegion.InitializationLevel.ANY_INIT;
 import static org.apache.geode.internal.cache.LocalRegion.InitializationLevel.BEFORE_INITIAL_IMAGE;
+import static org.apache.geode.internal.statistics.StatisticsClockFactory.disabledClock;
 import static org.assertj.core.api.Assertions.assertThat;
 import static org.mockito.Mockito.doCallRealMethod;
 import static org.mockito.Mockito.mock;
@@ -226,7 +227,7 @@ public class LocalRegionPartialMockTest {
     when(cache.getCCPTimer()).thenReturn(mock(SystemTimer.class));
 
     CacheClientNotifier ccn =
-        CacheClientNotifier.getInstance(cache, mock(CacheServerStats.class), 10,
+        CacheClientNotifier.getInstance(cache, disabledClock(), mock(CacheServerStats.class), 10,
             10, mock(ConnectionListener.class), null, true);
 
     doCallRealMethod().when(region).notifyClientsOfTombstoneGC(regionGCVersions, keysRemoved,
@@ -248,7 +249,7 @@ public class LocalRegionPartialMockTest {
     when(cache.getCCPTimer()).thenReturn(mock(SystemTimer.class));
 
     CacheClientNotifier ccn =
-        CacheClientNotifier.getInstance(cache, mock(CacheServerStats.class), 10,
+        CacheClientNotifier.getInstance(cache, disabledClock(), mock(CacheServerStats.class), 10,
             10, mock(ConnectionListener.class), null, true);
 
     when(proxy.getProxyID()).thenReturn(mock(ClientProxyMembershipID.class));
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/LocalRegionTest.java b/geode-core/src/test/java/org/apache/geode/internal/cache/LocalRegionTest.java
index 22adf2b..22ac4c9 100644
--- a/geode-core/src/test/java/org/apache/geode/internal/cache/LocalRegionTest.java
+++ b/geode-core/src/test/java/org/apache/geode/internal/cache/LocalRegionTest.java
@@ -15,6 +15,7 @@
 
 package org.apache.geode.internal.cache;
 
+import static org.apache.geode.internal.statistics.StatisticsClockFactory.disabledClock;
 import static org.assertj.core.api.Assertions.assertThatCode;
 import static org.mockito.ArgumentMatchers.any;
 import static org.mockito.Mockito.mock;
@@ -84,9 +85,10 @@ public class LocalRegionTest {
       return mock(RegionPerfStats.class);
     };
 
-    assertThatCode(() -> new LocalRegion("region", regionAttributes, null, cache,
-        internalRegionArguments, internalDataView, regionMapConstructor,
-        serverRegionProxyConstructor, entryEventFactory, poolFinder, regionPerfStatsFactory))
-            .doesNotThrowAnyException();
+    assertThatCode(
+        () -> new LocalRegion("region", regionAttributes, null, cache, internalRegionArguments,
+            internalDataView, regionMapConstructor, serverRegionProxyConstructor, entryEventFactory,
+            poolFinder, regionPerfStatsFactory, disabledClock()))
+                .doesNotThrowAnyException();
   }
 }
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/PartitionedRegionTest.java b/geode-core/src/test/java/org/apache/geode/internal/cache/PartitionedRegionTest.java
index 8c48a94..ee9825a 100644
--- a/geode-core/src/test/java/org/apache/geode/internal/cache/PartitionedRegionTest.java
+++ b/geode-core/src/test/java/org/apache/geode/internal/cache/PartitionedRegionTest.java
@@ -15,6 +15,7 @@
 package org.apache.geode.internal.cache;
 
 import static org.apache.geode.cache.asyncqueue.internal.AsyncEventQueueImpl.getSenderIdFromAsyncEventQueueId;
+import static org.apache.geode.internal.statistics.StatisticsClockFactory.disabledClock;
 import static org.assertj.core.api.Assertions.assertThat;
 import static org.mockito.ArgumentMatchers.any;
 import static org.mockito.ArgumentMatchers.anyInt;
@@ -73,7 +74,7 @@ public class PartitionedRegionTest {
     attributesFactory.setPartitionAttributes(
         new PartitionAttributesFactory().setTotalNumBuckets(1).setRedundantCopies(1).create());
     partitionedRegion = new PartitionedRegion("prTestRegion", attributesFactory.create(), null,
-        internalCache, mock(InternalRegionArguments.class));
+        internalCache, mock(InternalRegionArguments.class), disabledClock());
     DistributedSystem mockDistributedSystem = mock(DistributedSystem.class);
     when(internalCache.getDistributedSystem()).thenReturn(mockDistributedSystem);
     when(mockDistributedSystem.getProperties()).thenReturn(gemfireProperties);
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/RegionPerfStatsTest.java b/geode-core/src/test/java/org/apache/geode/internal/cache/RegionPerfStatsTest.java
index e79210f..4c9e29b 100644
--- a/geode-core/src/test/java/org/apache/geode/internal/cache/RegionPerfStatsTest.java
+++ b/geode-core/src/test/java/org/apache/geode/internal/cache/RegionPerfStatsTest.java
@@ -14,6 +14,7 @@
  */
 package org.apache.geode.internal.cache;
 
+import static org.apache.geode.internal.statistics.StatisticsClockFactory.disabledClock;
 import static org.assertj.core.api.Assertions.assertThat;
 import static org.mockito.ArgumentMatchers.any;
 import static org.mockito.ArgumentMatchers.eq;
@@ -71,8 +72,8 @@ public class RegionPerfStatsTest {
     when(statisticsFactory.createAtomicStatistics(any(), any())).thenReturn(statistics);
 
     regionPerfStats =
-        new RegionPerfStats(statisticsFactory, TEXT_ID, cachePerfStats,
-            region, meterRegistry);
+        new RegionPerfStats(statisticsFactory, TEXT_ID, disabledClock(), cachePerfStats, region,
+            meterRegistry);
   }
 
   @After
@@ -90,8 +91,8 @@ public class RegionPerfStatsTest {
     StatisticsFactory statisticsFactory = mock(StatisticsFactory.class);
     when(statisticsFactory.createAtomicStatistics(any(), any())).thenReturn(mock(Statistics.class));
 
-    new RegionPerfStats(statisticsFactory, TEXT_ID, cachePerfStats,
-        region, meterRegistry);
+    new RegionPerfStats(statisticsFactory, TEXT_ID, disabledClock(), cachePerfStats, region,
+        meterRegistry);
 
     verify(statisticsFactory).createAtomicStatistics(any(), eq(TEXT_ID));
   }
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/ServerBuilderTest.java b/geode-core/src/test/java/org/apache/geode/internal/cache/ServerBuilderTest.java
index 096dfe9..eb0bbdf 100644
--- a/geode-core/src/test/java/org/apache/geode/internal/cache/ServerBuilderTest.java
+++ b/geode-core/src/test/java/org/apache/geode/internal/cache/ServerBuilderTest.java
@@ -37,6 +37,7 @@ import org.apache.geode.distributed.internal.DistributionAdvisee;
 import org.apache.geode.internal.cache.tier.sockets.CacheClientNotifier.CacheClientNotifierProvider;
 import org.apache.geode.internal.cache.tier.sockets.ClientHealthMonitor.ClientHealthMonitorProvider;
 import org.apache.geode.internal.security.SecurityService;
+import org.apache.geode.internal.statistics.StatisticsClock;
 import org.apache.geode.test.junit.categories.ClientServerTest;
 
 @Category(ClientServerTest.class)
@@ -47,30 +48,32 @@ public class ServerBuilderTest {
 
   private InternalCache cache;
   private SecurityService securityService;
+  private StatisticsClock statisticsClock;
 
   @Before
   public void setUp() {
     cache = mock(InternalCache.class);
     securityService = mock(SecurityService.class);
+    statisticsClock = mock(StatisticsClock.class);
   }
 
   @Test
   public void sendResourceEventsIsTrueByDefault() {
-    ServerBuilder builder = new ServerBuilder(cache, securityService);
+    ServerBuilder builder = new ServerBuilder(cache, securityService, statisticsClock);
 
     assertThat(builder.isSendResourceEvents()).isTrue();
   }
 
   @Test
   public void includeMemberGroupsIsTrueByDefault() {
-    ServerBuilder builder = new ServerBuilder(cache, securityService);
+    ServerBuilder builder = new ServerBuilder(cache, securityService, statisticsClock);
 
     assertThat(builder.isIncludeMemberGroups()).isTrue();
   }
 
   @Test
   public void socketCreatorIsForServerByDefault() {
-    ServerBuilder builder = new ServerBuilder(cache, securityService);
+    ServerBuilder builder = new ServerBuilder(cache, securityService, statisticsClock);
 
     assertThat(builder.getSocketCreatorSupplier()).isSameAs(SERVER.getSupplier());
   }
@@ -80,7 +83,7 @@ public class ServerBuilderTest {
     GatewayReceiver gatewayReceiver = mock(GatewayReceiver.class);
     when(gatewayReceiver.getGatewayTransportFilters())
         .thenReturn(singletonList(mock(GatewayTransportFilter.class)));
-    ServerBuilder builder = new ServerBuilder(cache, securityService);
+    ServerBuilder builder = new ServerBuilder(cache, securityService, statisticsClock);
 
     builder.forGatewayReceiver(gatewayReceiver);
 
@@ -92,7 +95,7 @@ public class ServerBuilderTest {
     GatewayReceiver gatewayReceiver = mock(GatewayReceiver.class);
     when(gatewayReceiver.getGatewayTransportFilters())
         .thenReturn(singletonList(mock(GatewayTransportFilter.class)));
-    ServerBuilder builder = new ServerBuilder(cache, securityService);
+    ServerBuilder builder = new ServerBuilder(cache, securityService, statisticsClock);
 
     builder.forGatewayReceiver(gatewayReceiver);
 
@@ -104,7 +107,7 @@ public class ServerBuilderTest {
     GatewayReceiver gatewayReceiver = mock(GatewayReceiver.class);
     when(gatewayReceiver.getGatewayTransportFilters())
         .thenReturn(singletonList(mock(GatewayTransportFilter.class)));
-    ServerBuilder builder = new ServerBuilder(cache, securityService);
+    ServerBuilder builder = new ServerBuilder(cache, securityService, statisticsClock);
 
     builder.forGatewayReceiver(gatewayReceiver);
 
@@ -115,7 +118,7 @@ public class ServerBuilderTest {
   public void setCacheClientNotifierProviderReplacesCacheClientNotifierProvider() {
     CacheClientNotifierProvider cacheClientNotifierProvider =
         mock(CacheClientNotifierProvider.class);
-    ServerBuilder builder = new ServerBuilder(cache, securityService);
+    ServerBuilder builder = new ServerBuilder(cache, securityService, statisticsClock);
 
     builder.setCacheClientNotifierProvider(cacheClientNotifierProvider);
 
@@ -126,7 +129,7 @@ public class ServerBuilderTest {
   public void setClientHealthMonitorProviderReplacesClientHealthMonitorProvider() {
     ClientHealthMonitorProvider clientHealthMonitorProvider =
         mock(ClientHealthMonitorProvider.class);
-    ServerBuilder builder = new ServerBuilder(cache, securityService);
+    ServerBuilder builder = new ServerBuilder(cache, securityService, statisticsClock);
 
     builder.setClientHealthMonitorProvider(clientHealthMonitorProvider);
 
@@ -137,7 +140,7 @@ public class ServerBuilderTest {
   public void setCacheServerAdvisorProviderReplacesCacheServerAdvisorProvider() {
     Function<DistributionAdvisee, CacheServerAdvisor> cacheServerAdvisorProvider =
         a -> mock(CacheServerAdvisor.class);
-    ServerBuilder builder = new ServerBuilder(cache, securityService);
+    ServerBuilder builder = new ServerBuilder(cache, securityService, statisticsClock);
 
     builder.setCacheServerAdvisorProvider(cacheServerAdvisorProvider);
 
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/TXManagerImplTest.java b/geode-core/src/test/java/org/apache/geode/internal/cache/TXManagerImplTest.java
index 21ce34e..345872c 100644
--- a/geode-core/src/test/java/org/apache/geode/internal/cache/TXManagerImplTest.java
+++ b/geode-core/src/test/java/org/apache/geode/internal/cache/TXManagerImplTest.java
@@ -15,6 +15,7 @@
 
 package org.apache.geode.internal.cache;
 
+import static org.apache.geode.internal.statistics.StatisticsClockFactory.disabledClock;
 import static org.apache.geode.test.awaitility.GeodeAwaitility.await;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
@@ -76,7 +77,7 @@ public class TXManagerImplTest {
   public void setUp() {
     cache = Fakes.cache();
     dm = mock(ClusterDistributionManager.class);
-    txMgr = new TXManagerImpl(mock(CachePerfStats.class), cache);
+    txMgr = new TXManagerImpl(mock(CachePerfStats.class), cache, disabledClock());
     txid = new TXId(null, 0);
     msg = mock(DestroyMessage.class);
     txCommitMsg = mock(TXCommitMessage.class);
@@ -95,7 +96,7 @@ public class TXManagerImplTest {
     doReturn(distributedSystem).when(spyCache).getDistributedSystem();
     when(distributedSystem.getDistributionManager()).thenReturn(dm);
     when(distributedSystem.getDistributedMember()).thenReturn(member);
-    spyTxMgr = spy(new TXManagerImpl(mock(CachePerfStats.class), spyCache));
+    spyTxMgr = spy(new TXManagerImpl(mock(CachePerfStats.class), spyCache, disabledClock()));
     timer = mock(SystemTimer.class);
     doReturn(timer).when(spyCache).getCCPTimer();
   }
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/TXStateProxyImplTest.java b/geode-core/src/test/java/org/apache/geode/internal/cache/TXStateProxyImplTest.java
index 2256a1a..42fe93f 100644
--- a/geode-core/src/test/java/org/apache/geode/internal/cache/TXStateProxyImplTest.java
+++ b/geode-core/src/test/java/org/apache/geode/internal/cache/TXStateProxyImplTest.java
@@ -14,6 +14,7 @@
  */
 package org.apache.geode.internal.cache;
 
+import static org.apache.geode.internal.statistics.StatisticsClockFactory.disabledClock;
 import static org.assertj.core.api.Assertions.assertThat;
 import static org.mockito.Mockito.doReturn;
 import static org.mockito.Mockito.mock;
@@ -33,9 +34,8 @@ public class TXStateProxyImplTest {
 
   private InternalCache cache;
   private LocalRegion region;
-  String key = "testkey";
-  TXStateProxyImpl tx;
-  LocalRegionDataView view;
+  private String key = "testkey";
+  private LocalRegionDataView view;
   private TXId txId;
   private TXManagerImpl txManager;
   private DistributedSystem system;
@@ -74,7 +74,7 @@ public class TXStateProxyImplTest {
     when(view.getKeyForIterator(regionEntryKeyInfo, region, rememberReads, allowTombstones))
         .thenCallRealMethod();
 
-    TXStateProxyImpl tx = new TXStateProxyImpl(cache, txManager, txId, false);
+    TXStateProxyImpl tx = new TXStateProxyImpl(cache, txManager, txId, false, disabledClock());
 
     Object key1 = tx.getKeyForIterator(regionEntryKeyInfo, region, rememberReads, allowTombstones);
     assertThat(key1.equals(key)).isTrue();
@@ -85,13 +85,13 @@ public class TXStateProxyImplTest {
 
   @Test
   public void getCacheReturnsInjectedCache() {
-    TXStateProxyImpl tx = new TXStateProxyImpl(cache, txManager, txId, false);
+    TXStateProxyImpl tx = new TXStateProxyImpl(cache, txManager, txId, false, disabledClock());
     assertThat(tx.getCache()).isSameAs(cache);
   }
 
   @Test
   public void isOverTransactionTimeoutLimitReturnsTrueIfHavingRecentOperation() {
-    TXStateProxyImpl tx = spy(new TXStateProxyImpl(cache, txManager, txId, false));
+    TXStateProxyImpl tx = spy(new TXStateProxyImpl(cache, txManager, txId, false, disabledClock()));
     doReturn(0L).when(tx).getLastOperationTimeFromClient();
     doReturn(1001L).when(tx).getCurrentTime();
     when(txManager.getTransactionTimeToLive()).thenReturn(1);
@@ -101,7 +101,7 @@ public class TXStateProxyImplTest {
 
   @Test
   public void isOverTransactionTimeoutLimitReturnsFalseIfNotHavingRecentOperation() {
-    TXStateProxyImpl tx = spy(new TXStateProxyImpl(cache, txManager, txId, false));
+    TXStateProxyImpl tx = spy(new TXStateProxyImpl(cache, txManager, txId, false, disabledClock()));
     doReturn(0L).when(tx).getLastOperationTimeFromClient();
     doReturn(1000L).when(tx).getCurrentTime();
     when(txManager.getTransactionTimeToLive()).thenReturn(1);
@@ -111,7 +111,7 @@ public class TXStateProxyImplTest {
 
   @Test
   public void setTargetWillSetTargetToItselfAndSetTXStateIfRealDealIsNull() {
-    TXStateProxyImpl tx = spy(new TXStateProxyImpl(cache, txManager, txId, false));
+    TXStateProxyImpl tx = spy(new TXStateProxyImpl(cache, txManager, txId, false, disabledClock()));
     assertThat(tx.hasRealDeal()).isFalse();
     assertThat(tx.getTarget()).isNull();
 
@@ -123,7 +123,7 @@ public class TXStateProxyImplTest {
 
   @Test
   public void setTargetWillSetTXStateStubIfTargetIsDifferentFromLocalMember() {
-    TXStateProxyImpl tx = spy(new TXStateProxyImpl(cache, txManager, txId, false));
+    TXStateProxyImpl tx = spy(new TXStateProxyImpl(cache, txManager, txId, false, disabledClock()));
     assertThat(tx.hasRealDeal()).isFalse();
     assertThat(tx.getTarget()).isNull();
     DistributedMember remoteMember = mock(InternalDistributedMember.class);
@@ -137,8 +137,8 @@ public class TXStateProxyImplTest {
 
   @Test
   public void setTargetToItSelfIfRealDealIsTXStateAndTargetIsSameAsLocalMember() {
-    TXStateProxyImpl tx = spy(new TXStateProxyImpl(cache, txManager, txId, false));
-    tx.setLocalTXState(new TXState(tx, true));
+    TXStateProxyImpl tx = spy(new TXStateProxyImpl(cache, txManager, txId, false, disabledClock()));
+    tx.setLocalTXState(new TXState(tx, true, disabledClock()));
     assertThat(tx.isRealDealLocal()).isTrue();
     assertThat(tx.getTarget()).isNull();
 
@@ -150,8 +150,8 @@ public class TXStateProxyImplTest {
 
   @Test(expected = AssertionError.class)
   public void setTargetThrowsIfIfRealDealIsTXStateAndTargetIsDifferentFromLocalMember() {
-    TXStateProxyImpl tx = spy(new TXStateProxyImpl(cache, txManager, txId, false));
-    tx.setLocalTXState(new TXState(tx, true));
+    TXStateProxyImpl tx = spy(new TXStateProxyImpl(cache, txManager, txId, false, disabledClock()));
+    tx.setLocalTXState(new TXState(tx, true, disabledClock()));
     assertThat(tx.getTarget()).isNull();
     DistributedMember remoteMember = mock(InternalDistributedMember.class);
 
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/TXStateTest.java b/geode-core/src/test/java/org/apache/geode/internal/cache/TXStateTest.java
index e49f423..362ee5f 100644
--- a/geode-core/src/test/java/org/apache/geode/internal/cache/TXStateTest.java
+++ b/geode-core/src/test/java/org/apache/geode/internal/cache/TXStateTest.java
@@ -15,6 +15,7 @@
 package org.apache.geode.internal.cache;
 
 
+import static org.apache.geode.internal.statistics.StatisticsClockFactory.disabledClock;
 import static org.assertj.core.api.Assertions.assertThat;
 import static org.assertj.core.api.Assertions.assertThatThrownBy;
 import static org.assertj.core.api.Assertions.catchThrowable;
@@ -65,7 +66,7 @@ public class TXStateTest {
 
   @Test
   public void doBeforeCompletionThrowsIfReserveAndCheckFails() {
-    TXState txState = spy(new TXState(txStateProxy, true));
+    TXState txState = spy(new TXState(txStateProxy, true, disabledClock()));
     doThrow(exception).when(txState).reserveAndCheck();
 
     assertThatThrownBy(() -> txState.doBeforeCompletion())
@@ -74,7 +75,7 @@ public class TXStateTest {
 
   @Test
   public void doAfterCompletionThrowsIfCommitFails() {
-    TXState txState = spy(new TXState(txStateProxy, true));
+    TXState txState = spy(new TXState(txStateProxy, true, disabledClock()));
     txState.reserveAndCheck();
     doThrow(transactionDataNodeHasDepartedException).when(txState).commit();
 
@@ -84,7 +85,7 @@ public class TXStateTest {
 
   @Test
   public void doAfterCompletionCanCommitJTA() {
-    TXState txState = spy(new TXState(txStateProxy, false));
+    TXState txState = spy(new TXState(txStateProxy, false, disabledClock()));
     txState.reserveAndCheck();
     txState.closed = true;
     txState.doAfterCompletionCommit();
@@ -95,13 +96,13 @@ public class TXStateTest {
 
   @Test(expected = FailedSynchronizationException.class)
   public void afterCompletionThrowsExceptionIfBeforeCompletionNotCalled() {
-    TXState txState = new TXState(txStateProxy, true);
+    TXState txState = new TXState(txStateProxy, true, disabledClock());
     txState.afterCompletion(Status.STATUS_COMMITTED);
   }
 
   @Test
   public void afterCompletionInvokesExecuteAfterCompletionCommitIfBeforeCompletionCalled() {
-    TXState txState = spy(new TXState(txStateProxy, true, executor));
+    TXState txState = spy(new TXState(txStateProxy, true, executor, disabledClock()));
     doReturn(true).when(txState).wasBeforeCompletionCalled();
 
     txState.afterCompletion(Status.STATUS_COMMITTED);
@@ -111,7 +112,7 @@ public class TXStateTest {
 
   @Test
   public void afterCompletionThrowsWithUnexpectedStatusIfBeforeCompletionCalled() {
-    TXState txState = spy(new TXState(txStateProxy, true, executor));
+    TXState txState = spy(new TXState(txStateProxy, true, executor, disabledClock()));
     doReturn(true).when(txState).wasBeforeCompletionCalled();
 
     Throwable thrown = catchThrowable(() -> txState.afterCompletion(Status.STATUS_NO_TRANSACTION));
@@ -121,7 +122,7 @@ public class TXStateTest {
 
   @Test
   public void afterCompletionInvokesExecuteAfterCompletionRollbackIfBeforeCompletionCalled() {
-    TXState txState = spy(new TXState(txStateProxy, true, executor));
+    TXState txState = spy(new TXState(txStateProxy, true, executor, disabledClock()));
     doReturn(true).when(txState).wasBeforeCompletionCalled();
 
     txState.afterCompletion(Status.STATUS_ROLLEDBACK);
@@ -131,7 +132,7 @@ public class TXStateTest {
 
   @Test
   public void afterCompletionCanRollbackJTA() {
-    TXState txState = spy(new TXState(txStateProxy, true));
+    TXState txState = spy(new TXState(txStateProxy, true, disabledClock()));
     txState.afterCompletion(Status.STATUS_ROLLEDBACK);
 
     verify(txState, times(1)).rollback();
@@ -140,7 +141,7 @@ public class TXStateTest {
 
   @Test
   public void closeWillCleanupIfLocksObtained() {
-    TXState txState = spy(new TXState(txStateProxy, false));
+    TXState txState = spy(new TXState(txStateProxy, false, disabledClock()));
     txState.closed = false;
     txState.locks = mock(TXLockRequest.class);
     TXRegionState regionState1 = mock(TXRegionState.class);
@@ -161,7 +162,7 @@ public class TXStateTest {
 
   @Test
   public void closeWillCloseTXRegionStatesIfLocksNotObtained() {
-    TXState txState = spy(new TXState(txStateProxy, false));
+    TXState txState = spy(new TXState(txStateProxy, false, disabledClock()));
     txState.closed = false;
     // txState.locks = mock(TXLockRequest.class);
     TXRegionState regionState1 = mock(TXRegionState.class);
@@ -182,14 +183,14 @@ public class TXStateTest {
 
   @Test
   public void getOriginatingMemberReturnsNullIfNotOriginatedFromClient() {
-    TXState txState = spy(new TXState(txStateProxy, false));
+    TXState txState = spy(new TXState(txStateProxy, false, disabledClock()));
 
     assertThat(txState.getOriginatingMember()).isSameAs(txStateProxy.getOnBehalfOfClientMember());
   }
 
   @Test
   public void txReadEntryDoesNotCleanupTXEntriesIfRegionCreateReadEntryReturnsNull() {
-    TXState txState = spy(new TXState(txStateProxy, true));
+    TXState txState = spy(new TXState(txStateProxy, true, disabledClock()));
     KeyInfo keyInfo = mock(KeyInfo.class);
     Object key = new Object();
     InternalRegion internalRegion = mock(InternalRegion.class);
@@ -207,7 +208,7 @@ public class TXStateTest {
 
   @Test
   public void txReadEntryDoesNotCleanupTXEntriesIfEntryNotFound() {
-    TXState txState = spy(new TXState(txStateProxy, true));
+    TXState txState = spy(new TXState(txStateProxy, true, disabledClock()));
     KeyInfo keyInfo = mock(KeyInfo.class);
     Object key = new Object();
     Object expectedValue = new Object();
@@ -231,7 +232,7 @@ public class TXStateTest {
 
   @Test
   public void doCleanupContinuesWhenReleasingLockGotIllegalArgumentExceptionIfCacheIsClosing() {
-    TXState txState = spy(new TXState(txStateProxy, false));
+    TXState txState = spy(new TXState(txStateProxy, false, disabledClock()));
     txState.locks = mock(TXLockRequest.class);
     doReturn(cache).when(txStateProxy).getCache();
     doThrow(new IllegalArgumentException()).when(txState.locks).cleanup(internalDistributedSystem);
@@ -247,7 +248,7 @@ public class TXStateTest {
 
   @Test
   public void doCleanupContinuesWhenReleasingLockGotIllegalMonitorStateExceptionIfCacheIsClosing() {
-    TXState txState = spy(new TXState(txStateProxy, false));
+    TXState txState = spy(new TXState(txStateProxy, false, disabledClock()));
     txState.locks = mock(TXLockRequest.class);
     doReturn(cache).when(txStateProxy).getCache();
     doThrow(new IllegalMonitorStateException()).when(txState.locks)
@@ -264,7 +265,7 @@ public class TXStateTest {
 
   @Test
   public void doCleanupThrowsWhenReleasingLockGotIllegalArgumentExceptionIfCacheIsNotClosing() {
-    TXState txState = spy(new TXState(txStateProxy, false));
+    TXState txState = spy(new TXState(txStateProxy, false, disabledClock()));
     txState.locks = mock(TXLockRequest.class);
     doReturn(cache).when(txStateProxy).getCache();
     doThrow(new IllegalArgumentException()).when(txState.locks).cleanup(internalDistributedSystem);
@@ -281,7 +282,7 @@ public class TXStateTest {
 
   @Test
   public void doCleanupThrowsWhenReleasingLockGotIllegalMonitorStateExceptionIfCacheIsNotClosing() {
-    TXState txState = spy(new TXState(txStateProxy, false));
+    TXState txState = spy(new TXState(txStateProxy, false, disabledClock()));
     txState.locks = mock(TXLockRequest.class);
     doReturn(cache).when(txStateProxy).getCache();
     doThrow(new IllegalMonitorStateException()).when(txState.locks)
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/UpdateOperationJUnitTest.java b/geode-core/src/test/java/org/apache/geode/internal/cache/UpdateOperationJUnitTest.java
index 2e26b70..ac3520a 100644
--- a/geode-core/src/test/java/org/apache/geode/internal/cache/UpdateOperationJUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/internal/cache/UpdateOperationJUnitTest.java
@@ -29,9 +29,10 @@ import org.apache.geode.cache.Operation;
 import org.apache.geode.cache.RegionAttributes;
 
 public class UpdateOperationJUnitTest {
-  EntryEventImpl event;
-  UpdateOperation.UpdateMessage message;
-  DistributedRegion region;
+
+  private EntryEventImpl event;
+  private UpdateOperation.UpdateMessage message;
+  private DistributedRegion region;
 
   @Before
   public void setup() {
@@ -39,6 +40,8 @@ public class UpdateOperationJUnitTest {
     region = mock(DistributedRegion.class);
     RegionAttributes attr = mock(RegionAttributes.class);
     CachePerfStats stats = mock(CachePerfStats.class);
+    InternalCache cache = mock(InternalCache.class);
+
     when(event.isOriginRemote()).thenReturn(false);
     when(stats.endPut(anyLong(), eq(false))).thenReturn(0L);
 
@@ -50,6 +53,8 @@ public class UpdateOperationJUnitTest {
     when(region.getDataPolicy()).thenReturn(DataPolicy.REPLICATE);
     when(region.getConcurrencyChecksEnabled()).thenReturn(true);
     when(region.getCachePerfStats()).thenReturn(stats);
+    when(region.getCache()).thenReturn(cache);
+    // when(cache.getStatisticsClock()).thenReturn(disabledClock());
     when(attr.getDataPolicy()).thenReturn(DataPolicy.REPLICATE);
     when(event.getOperation()).thenReturn(Operation.CREATE);
   }
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/ha/HARegionQueueTest.java b/geode-core/src/test/java/org/apache/geode/internal/cache/ha/HARegionQueueTest.java
index 081a3e5..cd33f8d 100644
--- a/geode-core/src/test/java/org/apache/geode/internal/cache/ha/HARegionQueueTest.java
+++ b/geode-core/src/test/java/org/apache/geode/internal/cache/ha/HARegionQueueTest.java
@@ -31,6 +31,7 @@ import org.apache.geode.CancelCriterion;
 import org.apache.geode.internal.cache.EventID;
 import org.apache.geode.internal.cache.HARegion;
 import org.apache.geode.internal.cache.InternalCache;
+import org.apache.geode.internal.statistics.StatisticsClock;
 import org.apache.geode.internal.util.concurrent.StoppableReentrantReadWriteLock;
 import org.apache.geode.test.junit.categories.ClientSubscriptionTest;
 
@@ -64,7 +65,8 @@ public class HARegionQueueTest {
     when(haRegion.getGemFireCache()).thenReturn(internalCache);
     haRegionQueue = new HARegionQueue("haRegion", haRegion, internalCache,
         new HAContainerMap(new ConcurrentHashMap()), null, (byte) 1, true,
-        mock(HARegionQueueStats.class), giiLock, rwLock, mock(CancelCriterion.class), false);
+        mock(HARegionQueueStats.class), giiLock, rwLock, mock(CancelCriterion.class), false,
+        mock(StatisticsClock.class));
   }
 
   @Test
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/partitioned/FetchKeysMessageTest.java b/geode-core/src/test/java/org/apache/geode/internal/cache/partitioned/FetchKeysMessageTest.java
index 547144a..3728718 100644
--- a/geode-core/src/test/java/org/apache/geode/internal/cache/partitioned/FetchKeysMessageTest.java
+++ b/geode-core/src/test/java/org/apache/geode/internal/cache/partitioned/FetchKeysMessageTest.java
@@ -14,6 +14,7 @@
  */
 package org.apache.geode.internal.cache.partitioned;
 
+import static org.apache.geode.internal.statistics.StatisticsClockFactory.disabledClock;
 import static org.assertj.core.api.Assertions.assertThat;
 import static org.mockito.Answers.RETURNS_DEEP_STUBS;
 import static org.mockito.ArgumentMatchers.any;
@@ -75,7 +76,7 @@ public class FetchKeysMessageTest {
 
     originalTxManager = TXManagerImpl.getCurrentInstanceForTest();
     // The constructor sets the new tx manager as currentInstance
-    txManager = spy(new TXManagerImpl(mock(CachePerfStats.class), cache));
+    txManager = spy(new TXManagerImpl(mock(CachePerfStats.class), cache, disabledClock()));
     txManager.setTXState(txStateProxy);
     txManager.setDistributed(false);
 
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/partitioned/PartitionMessageTest.java b/geode-core/src/test/java/org/apache/geode/internal/cache/partitioned/PartitionMessageTest.java
index 704e47b..352cbb2 100644
--- a/geode-core/src/test/java/org/apache/geode/internal/cache/partitioned/PartitionMessageTest.java
+++ b/geode-core/src/test/java/org/apache/geode/internal/cache/partitioned/PartitionMessageTest.java
@@ -14,6 +14,7 @@
  */
 package org.apache.geode.internal.cache.partitioned;
 
+import static org.apache.geode.internal.statistics.StatisticsClockFactory.disabledClock;
 import static org.assertj.core.api.Assertions.assertThat;
 import static org.mockito.ArgumentMatchers.argThat;
 import static org.mockito.ArgumentMatchers.eq;
@@ -127,7 +128,7 @@ public class PartitionMessageTest {
 
   @Test
   public void noNewTxProcessingAfterTXManagerImplClosed() throws Exception {
-    txMgr = new TXManagerImpl(null, cache);
+    txMgr = new TXManagerImpl(null, cache, disabledClock());
     when(msg.getPartitionedRegion()).thenReturn(pr);
     when(msg.getStartPartitionMessageProcessingTime(pr)).thenReturn(startTime);
     when(msg.getTXManagerImpl(cache)).thenReturn(txMgr);
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/tier/sockets/AcceptorBuilderTest.java b/geode-core/src/test/java/org/apache/geode/internal/cache/tier/sockets/AcceptorBuilderTest.java
index c7f37e7..14f83db 100644
--- a/geode-core/src/test/java/org/apache/geode/internal/cache/tier/sockets/AcceptorBuilderTest.java
+++ b/geode-core/src/test/java/org/apache/geode/internal/cache/tier/sockets/AcceptorBuilderTest.java
@@ -36,6 +36,7 @@ import org.apache.geode.internal.cache.tier.sockets.CacheClientNotifier.CacheCli
 import org.apache.geode.internal.cache.tier.sockets.ClientHealthMonitor.ClientHealthMonitorProvider;
 import org.apache.geode.internal.net.SocketCreator;
 import org.apache.geode.internal.security.SecurityService;
+import org.apache.geode.internal.statistics.StatisticsClock;
 import org.apache.geode.test.junit.categories.ClientServerTest;
 
 @Category(ClientServerTest.class)
@@ -288,6 +289,18 @@ public class AcceptorBuilderTest {
   }
 
   @Test
+  public void forServerSetsStatisticsClockFromServer() {
+    InternalCacheServer server = mock(InternalCacheServer.class);
+    StatisticsClock statisticsClock = mock(StatisticsClock.class);
+    when(server.getStatisticsClock()).thenReturn(statisticsClock);
+    AcceptorBuilder builder = new AcceptorBuilder();
+
+    builder.forServer(server);
+
+    assertThat(builder.getStatisticsClock()).isEqualTo(statisticsClock);
+  }
+
+  @Test
   public void setCacheReplacesCacheFromServer() {
     InternalCacheServer server = mock(InternalCacheServer.class);
     InternalCache cacheFromServer = mock(InternalCache.class, "fromServer");
@@ -376,4 +389,18 @@ public class AcceptorBuilderTest {
     assertThat(builder.getClientHealthMonitorProvider())
         .isEqualTo(clientHealthMonitorProviderFromSetter);
   }
+
+  @Test
+  public void setStatisticsClockReplacesStatisticsClockFromServer() {
+    InternalCacheServer server = mock(InternalCacheServer.class);
+    StatisticsClock statisticsClockFromServer = mock(StatisticsClock.class, "fromServer");
+    when(server.getStatisticsClock()).thenReturn(statisticsClockFromServer);
+    AcceptorBuilder builder = new AcceptorBuilder();
+    builder.forServer(server);
+    StatisticsClock statisticsClockFromSetter = mock(StatisticsClock.class, "fromSetter");
+
+    builder.setStatisticsClock(statisticsClockFromSetter);
+
+    assertThat(builder.getStatisticsClock()).isEqualTo(statisticsClockFromSetter);
+  }
 }
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/tier/sockets/AcceptorImplTest.java b/geode-core/src/test/java/org/apache/geode/internal/cache/tier/sockets/AcceptorImplTest.java
index a67b2d6..7ed8eef 100644
--- a/geode-core/src/test/java/org/apache/geode/internal/cache/tier/sockets/AcceptorImplTest.java
+++ b/geode-core/src/test/java/org/apache/geode/internal/cache/tier/sockets/AcceptorImplTest.java
@@ -14,10 +14,12 @@
  */
 package org.apache.geode.internal.cache.tier.sockets;
 
+import static java.util.Collections.emptyList;
 import static org.apache.geode.cache.server.CacheServer.DEFAULT_MAXIMUM_TIME_BETWEEN_PINGS;
 import static org.apache.geode.cache.server.CacheServer.DEFAULT_SOCKET_BUFFER_SIZE;
 import static org.apache.geode.cache.server.CacheServer.DEFAULT_TCP_NO_DELAY;
 import static org.apache.geode.internal.cache.tier.sockets.AcceptorImpl.MINIMUM_MAX_CONNECTIONS;
+import static org.apache.geode.internal.statistics.StatisticsClockFactory.disabledClock;
 import static org.assertj.core.api.Assertions.assertThat;
 import static org.mockito.ArgumentMatchers.any;
 import static org.mockito.ArgumentMatchers.anyInt;
@@ -29,7 +31,6 @@ import static org.mockito.quality.Strictness.STRICT_STUBS;
 
 import java.net.ServerSocket;
 import java.net.SocketAddress;
-import java.util.Collections;
 import java.util.Properties;
 
 import io.micrometer.core.instrument.simple.SimpleMeterRegistry;
@@ -96,8 +97,8 @@ public class AcceptorImplTest {
         DEFAULT_MAXIMUM_TIME_BETWEEN_PINGS, cache, MINIMUM_MAX_CONNECTIONS, 0,
         CacheServer.DEFAULT_MAXIMUM_MESSAGE_COUNT, CacheServer.DEFAULT_MESSAGE_TIME_TO_LIVE, null,
         null, DEFAULT_TCP_NO_DELAY, serverConnectionFactory, 1000, securityService,
-        () -> socketCreator, (a, b, c, d, e, f, g) -> cacheClientNotifier,
-        (a, b, c) -> clientHealthMonitor, false, Collections.emptyList());
+        () -> socketCreator, (a, b, c, d, e, f, g, h) -> cacheClientNotifier,
+        (a, b, c) -> clientHealthMonitor, false, emptyList(), disabledClock());
 
     assertThat(acceptor.isGatewayReceiver()).isFalse();
   }
@@ -114,8 +115,8 @@ public class AcceptorImplTest {
         DEFAULT_MAXIMUM_TIME_BETWEEN_PINGS, cache, MINIMUM_MAX_CONNECTIONS, 0,
         CacheServer.DEFAULT_MAXIMUM_MESSAGE_COUNT, CacheServer.DEFAULT_MESSAGE_TIME_TO_LIVE, null,
         null, DEFAULT_TCP_NO_DELAY, serverConnectionFactory, 1000, securityService,
-        () -> socketCreator, (a, b, c, d, e, f, g) -> cacheClientNotifier,
-        (a, b, c) -> clientHealthMonitor, true, Collections.emptyList());
+        () -> socketCreator, (a, b, c, d, e, f, g, h) -> cacheClientNotifier,
+        (a, b, c) -> clientHealthMonitor, true, emptyList(), disabledClock());
 
     assertThat(acceptor.isGatewayReceiver()).isTrue();
   }
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/tier/sockets/CacheClientNotifierTest.java b/geode-core/src/test/java/org/apache/geode/internal/cache/tier/sockets/CacheClientNotifierTest.java
index 31f059b..2c1c57c 100644
--- a/geode-core/src/test/java/org/apache/geode/internal/cache/tier/sockets/CacheClientNotifierTest.java
+++ b/geode-core/src/test/java/org/apache/geode/internal/cache/tier/sockets/CacheClientNotifierTest.java
@@ -12,7 +12,6 @@
  * or implied. See the License for the specific language governing permissions and limitations under
  * the License.
  */
-
 package org.apache.geode.internal.cache.tier.sockets;
 
 import static org.junit.Assert.assertFalse;
@@ -48,9 +47,11 @@ import org.apache.geode.internal.cache.FilterProfile;
 import org.apache.geode.internal.cache.FilterRoutingInfo;
 import org.apache.geode.internal.cache.InternalCache;
 import org.apache.geode.internal.cache.InternalCacheEvent;
+import org.apache.geode.internal.statistics.StatisticsClock;
 import org.apache.geode.test.fake.Fakes;
 
 public class CacheClientNotifierTest {
+
   @Test
   public void eventsInClientRegistrationQueueAreSentToClientAfterRegistrationIsComplete()
       throws IOException, ClassNotFoundException, NoSuchMethodException, IllegalAccessException,
@@ -63,11 +64,13 @@ public class CacheClientNotifierTest {
     CacheClientProxy cacheClientProxy = mock(CacheClientProxy.class);
     ClientUpdateMessageImpl clientUpdateMessage = mock(ClientUpdateMessageImpl.class);
     ClientRegistrationMetadata clientRegistrationMetadata = mock(ClientRegistrationMetadata.class);
+    StatisticsClock statisticsClock = mock(StatisticsClock.class);
+
     when(clientRegistrationMetadata.getClientProxyMembershipID()).thenReturn(
         clientProxyMembershipID);
 
     CacheClientNotifier cacheClientNotifier = CacheClientNotifier.getInstance(internalCache,
-        cacheServerStats, 0, 0, connectionListener, null, false);
+        statisticsClock, cacheServerStats, 0, 0, connectionListener, null, false);
     final CacheClientNotifier cacheClientNotifierSpy = spy(cacheClientNotifier);
 
     CountDownLatch waitForEventDispatchCountdownLatch = new CountDownLatch(1);
@@ -188,8 +191,8 @@ public class CacheClientNotifierTest {
     InternalCache internalCache = Fakes.cache();
 
     CacheClientNotifier ccn =
-        CacheClientNotifier.getInstance(internalCache, mock(CacheServerStats.class), 10,
-            10, mock(ConnectionListener.class), null, true);
+        CacheClientNotifier.getInstance(internalCache, mock(StatisticsClock.class),
+            mock(CacheServerStats.class), 10, 10, mock(ConnectionListener.class), null, true);
 
     assertFalse(CacheClientNotifier.singletonHasClientProxies());
     ccn.shutdown(111);
@@ -202,8 +205,8 @@ public class CacheClientNotifierTest {
     CacheClientProxy proxy = mock(CacheClientProxy.class);
 
     CacheClientNotifier ccn =
-        CacheClientNotifier.getInstance(internalCache, mock(CacheServerStats.class), 10,
-            10, mock(ConnectionListener.class), null, true);
+        CacheClientNotifier.getInstance(internalCache, mock(StatisticsClock.class),
+            mock(CacheServerStats.class), 10, 10, mock(ConnectionListener.class), null, true);
 
     when(proxy.getProxyID()).thenReturn(mock(ClientProxyMembershipID.class));
     ccn.addClientProxy(proxy);
@@ -221,8 +224,8 @@ public class CacheClientNotifierTest {
     CacheClientProxy proxy = mock(CacheClientProxy.class);
 
     CacheClientNotifier ccn =
-        CacheClientNotifier.getInstance(internalCache, mock(CacheServerStats.class), 10,
-            10, mock(ConnectionListener.class), null, true);
+        CacheClientNotifier.getInstance(internalCache, mock(StatisticsClock.class),
+            mock(CacheServerStats.class), 10, 10, mock(ConnectionListener.class), null, true);
 
     when(proxy.getProxyID()).thenReturn(mock(ClientProxyMembershipID.class));
     ccn.addClientInitProxy(proxy);
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/tier/sockets/command/TXFailoverCommandTest.java b/geode-core/src/test/java/org/apache/geode/internal/cache/tier/sockets/command/TXFailoverCommandTest.java
index e8105db..6b969dd 100644
--- a/geode-core/src/test/java/org/apache/geode/internal/cache/tier/sockets/command/TXFailoverCommandTest.java
+++ b/geode-core/src/test/java/org/apache/geode/internal/cache/tier/sockets/command/TXFailoverCommandTest.java
@@ -14,6 +14,7 @@
  */
 package org.apache.geode.internal.cache.tier.sockets.command;
 
+import static org.apache.geode.internal.statistics.StatisticsClockFactory.disabledClock;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotNull;
 import static org.mockito.Mockito.doReturn;
@@ -54,7 +55,7 @@ public class TXFailoverCommandTest {
 
     int uniqueId = 1;
     TXId txId = new TXId(client, uniqueId);
-    TXStateProxyImpl proxy = new TXStateProxyImpl(cache, txManager, txId, null);
+    TXStateProxyImpl proxy = new TXStateProxyImpl(cache, txManager, txId, null, disabledClock());
 
     when(cache.getCacheTransactionManager()).thenReturn(txManager);
     when(cache.getCancelCriterion()).thenReturn(mock(CancelCriterion.class));
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/wan/parallel/ParallelGatewaySenderHelper.java b/geode-core/src/test/java/org/apache/geode/internal/cache/wan/parallel/ParallelGatewaySenderHelper.java
index b0ace84..81bc77e 100644
--- a/geode-core/src/test/java/org/apache/geode/internal/cache/wan/parallel/ParallelGatewaySenderHelper.java
+++ b/geode-core/src/test/java/org/apache/geode/internal/cache/wan/parallel/ParallelGatewaySenderHelper.java
@@ -14,6 +14,7 @@
  */
 package org.apache.geode.internal.cache.wan.parallel;
 
+import static org.apache.geode.internal.statistics.StatisticsClockFactory.disabledClock;
 import static org.mockito.ArgumentMatchers.eq;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
@@ -149,7 +150,7 @@ public class ParallelGatewaySenderHelper {
 
     // Create BucketRegionQueue
     return new BucketRegionQueue(
-        queueRegion.getBucketName(bucketId), attributes, parentRegion, cache, ira);
+        queueRegion.getBucketName(bucketId), attributes, parentRegion, cache, ira, disabledClock());
   }
 
   public static String getRegionQueueName(String gatewaySenderId) {
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/wan/parallel/ParallelQueueRemovalMessageJUnitTest.java b/geode-core/src/test/java/org/apache/geode/internal/cache/wan/parallel/ParallelQueueRemovalMessageJUnitTest.java
index cbb6551..c48b8fc 100644
--- a/geode-core/src/test/java/org/apache/geode/internal/cache/wan/parallel/ParallelQueueRemovalMessageJUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/internal/cache/wan/parallel/ParallelQueueRemovalMessageJUnitTest.java
@@ -14,6 +14,7 @@
  */
 package org.apache.geode.internal.cache.wan.parallel;
 
+import static org.apache.geode.internal.statistics.StatisticsClockFactory.disabledClock;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.mockito.Mockito.any;
@@ -97,7 +98,8 @@ public class ParallelQueueRemovalMessageJUnitTest {
     when(this.queueRegion.getParallelGatewaySender()).thenReturn(this.sender);
     when(this.sender.getQueues()).thenReturn(null);
     when(this.sender.getDispatcherThreads()).thenReturn(1);
-    stats = new GatewaySenderStats(new DummyStatisticsFactory(), "ln");
+    stats = new GatewaySenderStats(new DummyStatisticsFactory(), "gatewaySenderStats-", "ln",
+        disabledClock());
     when(this.sender.getStatistics()).thenReturn(stats);
   }
 
diff --git a/geode-core/src/test/java/org/apache/geode/internal/statistics/StatisticsClockFactoryTest.java b/geode-core/src/test/java/org/apache/geode/internal/statistics/StatisticsClockFactoryTest.java
new file mode 100644
index 0000000..f84cff4
--- /dev/null
+++ b/geode-core/src/test/java/org/apache/geode/internal/statistics/StatisticsClockFactoryTest.java
@@ -0,0 +1,127 @@
+/*
+ * 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.geode.internal.statistics;
+
+import static org.apache.geode.internal.statistics.StatisticsClockFactory.ENABLE_CLOCK_STATS_PROPERTY;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.verify;
+
+import java.util.function.BooleanSupplier;
+import java.util.function.LongSupplier;
+
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.contrib.java.lang.system.RestoreSystemProperties;
+
+public class StatisticsClockFactoryTest {
+
+  @Rule
+  public RestoreSystemProperties restoreSystemProperties = new RestoreSystemProperties();
+
+  @Test
+  public void clock_createsEnabledClockIfPropertyIsTrue() {
+    System.setProperty(ENABLE_CLOCK_STATS_PROPERTY, "true");
+
+    StatisticsClock clock = StatisticsClockFactory.clock();
+
+    assertThat(clock.isEnabled()).isTrue();
+  }
+
+  @Test
+  public void clock_createsDisabledClockIfPropertyIsFalse() {
+    System.setProperty(ENABLE_CLOCK_STATS_PROPERTY, "false");
+
+    StatisticsClock clock = StatisticsClockFactory.clock();
+
+    assertThat(clock.isEnabled()).isFalse();
+  }
+
+  @Test
+  public void clock_boolean_createsEnabledClockIfParameterIsTrue() {
+    StatisticsClock clock = StatisticsClockFactory.clock(true);
+
+    assertThat(clock.isEnabled()).isTrue();
+  }
+
+  @Test
+  public void clock_boolean_createsEnabledClockIfParameterIsFalse() {
+    StatisticsClock clock = StatisticsClockFactory.clock(false);
+
+    assertThat(clock.isEnabled()).isFalse();
+  }
+
+  @Test
+  public void enabledClock_usesProvidedLongSupplierForGetTime() {
+    StatisticsClock clock = StatisticsClockFactory.enabledClock(() -> 42);
+
+    assertThat(clock.getTime()).isEqualTo(42);
+  }
+
+  @Test
+  public void enabledClock_createsEnabledClock() {
+    StatisticsClock clock = StatisticsClockFactory.enabledClock(() -> 24);
+
+    assertThat(clock.isEnabled()).isTrue();
+  }
+
+  @Test
+  public void disabledClock_usesZeroForGetTime() {
+    StatisticsClock clock = StatisticsClockFactory.disabledClock();
+
+    assertThat(clock.getTime()).isZero();
+  }
+
+  @Test
+  public void disabledClock_createsDisabledClock() {
+    StatisticsClock clock = StatisticsClockFactory.disabledClock();
+
+    assertThat(clock.isEnabled()).isFalse();
+  }
+
+  @Test
+  public void clock_usesProvidedLongSupplierForGetTime() {
+    StatisticsClock clock = StatisticsClockFactory.clock(() -> 100, () -> true);
+
+    assertThat(clock.getTime()).isEqualTo(100);
+  }
+
+  @Test
+  public void clock_usesProvidedBooleanSupplierForIsEnabled() {
+    StatisticsClock clock = StatisticsClockFactory.clock(() -> 100, () -> true);
+
+    assertThat(clock.isEnabled()).isTrue();
+  }
+
+  @Test
+  public void clock_getTime_delegatesToLongSupplier() {
+    LongSupplier time = mock(LongSupplier.class);
+    StatisticsClock clock = StatisticsClockFactory.clock(time, mock(BooleanSupplier.class));
+
+    clock.getTime();
+
+    verify(time).getAsLong();
+  }
+
+  @Test
+  public void clock_isEnabled_delegatesToBooleanSupplier() {
+    BooleanSupplier isEnabled = mock(BooleanSupplier.class);
+    StatisticsClock clock = StatisticsClockFactory.clock(mock(LongSupplier.class), isEnabled);
+
+    clock.isEnabled();
+
+    verify(isEnabled).getAsBoolean();
+  }
+}
diff --git a/geode-core/src/test/java/org/apache/geode/management/bean/stats/MemberLevelStatsTest.java b/geode-core/src/test/java/org/apache/geode/management/bean/stats/MemberLevelStatsTest.java
index 21e166d..d450e87 100644
--- a/geode-core/src/test/java/org/apache/geode/management/bean/stats/MemberLevelStatsTest.java
+++ b/geode-core/src/test/java/org/apache/geode/management/bean/stats/MemberLevelStatsTest.java
@@ -14,6 +14,7 @@
  */
 package org.apache.geode.management.bean.stats;
 
+import static org.apache.geode.internal.statistics.StatisticsClockFactory.disabledClock;
 import static org.apache.geode.internal.statistics.SuppliableStatistics.toSuppliableStatistics;
 import static org.assertj.core.api.Assertions.assertThat;
 import static org.mockito.Mockito.mock;
@@ -69,7 +70,6 @@ public class MemberLevelStatsTest {
   @Before
   public void setUp() throws Exception {
     DistributionStats.enableClockStats = true;
-    CachePerfStats.enableClockStats = true;
 
     StatisticsManager statisticsManager = new StatisticsRegistry("TestStatisticsRegistry", 1);
     InternalDistributedSystem system = mock(InternalDistributedSystem.class);
@@ -116,7 +116,7 @@ public class MemberLevelStatsTest {
     partitionedRegionStatsArray = new PartitionedRegionStats[4];
     for (int i = 0; i < 4; i++) {
       PartitionedRegionStats stats = new PartitionedRegionStats(
-          statisticsManager, name.getMethodName() + i);
+          statisticsManager, name.getMethodName() + i, disabledClock());
       partitionedRegionStatsArray[i] = stats;
       memberMBeanBridge.addPartitionedRegionStats(stats);
     }
@@ -128,7 +128,6 @@ public class MemberLevelStatsTest {
   @After
   public void tearDown() throws Exception {
     DistributionStats.enableClockStats = true;
-    CachePerfStats.enableClockStats = false;
     statSampler.stop();
   }
 
diff --git a/geode-junit/src/main/java/org/apache/geode/internal/cache/ha/TestBlockingHARegionQueue.java b/geode-junit/src/main/java/org/apache/geode/internal/cache/ha/TestBlockingHARegionQueue.java
index 1f66234..27905f8 100755
--- a/geode-junit/src/main/java/org/apache/geode/internal/cache/ha/TestBlockingHARegionQueue.java
+++ b/geode-junit/src/main/java/org/apache/geode/internal/cache/ha/TestBlockingHARegionQueue.java
@@ -21,6 +21,7 @@ import org.apache.logging.log4j.Logger;
 import org.apache.geode.cache.CacheException;
 import org.apache.geode.internal.cache.InternalCache;
 import org.apache.geode.internal.logging.LogService;
+import org.apache.geode.internal.statistics.StatisticsClock;
 
 /**
  * Test class for Blocking HA region queue functionalities
@@ -31,7 +32,7 @@ import org.apache.geode.internal.logging.LogService;
 
 // TODO:Asif: Modify the test to allow working with the new class containing
 // ReadWrite lock functionality
-public class TestBlockingHARegionQueue extends HARegionQueue.TestOnlyHARegionQueue {
+public class TestBlockingHARegionQueue extends TestOnlyHARegionQueue {
   private static final Logger logger = LogService.getLogger();
 
   /**
@@ -43,9 +44,10 @@ public class TestBlockingHARegionQueue extends HARegionQueue.TestOnlyHARegionQue
 
   boolean takeWhenPeekInProgress = false;
 
-  public TestBlockingHARegionQueue(String regionName, InternalCache cache)
+  public TestBlockingHARegionQueue(String regionName, InternalCache cache,
+      StatisticsClock statisticsClock)
       throws IOException, ClassNotFoundException, CacheException, InterruptedException {
-    super(regionName, cache);
+    super(regionName, cache, statisticsClock);
   }
 
   /**
diff --git a/geode-junit/src/main/java/org/apache/geode/internal/cache/ha/TestOnlyHARegionQueue.java b/geode-junit/src/main/java/org/apache/geode/internal/cache/ha/TestOnlyHARegionQueue.java
new file mode 100644
index 0000000..7be78bf
--- /dev/null
+++ b/geode-junit/src/main/java/org/apache/geode/internal/cache/ha/TestOnlyHARegionQueue.java
@@ -0,0 +1,65 @@
+/*
+ * 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.geode.internal.cache.ha;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.geode.cache.CacheException;
+import org.apache.geode.cache.ExpirationAction;
+import org.apache.geode.cache.ExpirationAttributes;
+import org.apache.geode.internal.cache.InternalCache;
+import org.apache.geode.internal.cache.tier.sockets.Handshake;
+import org.apache.geode.internal.statistics.StatisticsClock;
+
+/**
+ * A static class which is created only for for testing purposes as some existing tests extend the
+ * HARegionQueue. Since the constructors of HARegionQueue are private , this class can act as a
+ * bridge between the user defined HARegionQueue class & the actual class. This class object will
+ * be buggy as it will tend to publish the Object o QRM thread & the expiry thread before the
+ * complete creation of the HARegionQueue instance
+ */
+class TestOnlyHARegionQueue extends HARegionQueue {
+
+  TestOnlyHARegionQueue(String regionName, InternalCache cache, StatisticsClock statisticsClock)
+      throws IOException, ClassNotFoundException, CacheException, InterruptedException {
+    this(regionName, cache, HARegionQueueAttributes.DEFAULT_HARQ_ATTRIBUTES, new HashMap(),
+        Handshake.CONFLATION_DEFAULT, false, statisticsClock);
+  }
+
+  TestOnlyHARegionQueue(String regionName, InternalCache cache, HARegionQueueAttributes hrqa,
+      Map haContainer, byte clientConflation, boolean isPrimary, StatisticsClock statisticsClock)
+      throws IOException, ClassNotFoundException, CacheException, InterruptedException {
+    super(regionName, cache, haContainer, null, clientConflation, isPrimary, statisticsClock);
+    ExpirationAttributes expirationAttributes =
+        new ExpirationAttributes(hrqa.getExpiryTime(), ExpirationAction.LOCAL_INVALIDATE);
+    region.setOwner(this);
+    region.getAttributesMutator().setEntryTimeToLive(expirationAttributes);
+    initialized.set(true);
+  }
+
+  /**
+   * Overloaded constructor to pass an {@code HashMap} instance as a haContainer.
+   *
+   * @since GemFire 5.7
+   */
+  TestOnlyHARegionQueue(String regionName, InternalCache cache, HARegionQueueAttributes hrqa,
+      StatisticsClock statisticsClock)
+      throws IOException, ClassNotFoundException, CacheException, InterruptedException {
+    this(regionName, cache, hrqa, new HashMap(), Handshake.CONFLATION_DEFAULT, false,
+        statisticsClock);
+  }
+}
diff --git a/geode-junit/src/main/java/org/apache/geode/test/fake/Fakes.java b/geode-junit/src/main/java/org/apache/geode/test/fake/Fakes.java
index c1bd3f9..d83feac 100644
--- a/geode-junit/src/main/java/org/apache/geode/test/fake/Fakes.java
+++ b/geode-junit/src/main/java/org/apache/geode/test/fake/Fakes.java
@@ -45,6 +45,7 @@ import org.apache.geode.internal.cache.TXManagerImpl;
 import org.apache.geode.internal.logging.InternalLogWriter;
 import org.apache.geode.internal.security.SecurableCommunicationChannel;
 import org.apache.geode.internal.security.SecurityService;
+import org.apache.geode.internal.statistics.StatisticsClock;
 import org.apache.geode.internal.statistics.StatisticsManager;
 import org.apache.geode.pdx.PdxInstanceFactory;
 import org.apache.geode.pdx.internal.TypeRegistry;
@@ -108,6 +109,8 @@ public class Fakes {
     when(cache.getQueryMonitor()).thenReturn(queryMonitor);
     when(cache.getMeterRegistry()).thenReturn(new SimpleMeterRegistry());
     when(cache.getCCPTimer()).thenReturn(mock(SystemTimer.class));
+    when(cache.getStatisticsClock()).thenReturn(mock(StatisticsClock.class));
+
     when(system.getDistributedMember()).thenReturn(member);
     when(system.getConfig()).thenReturn(config);
     when(system.getDistributionManager()).thenReturn(distributionManager);
diff --git a/geode-wan/src/main/java/org/apache/geode/cache/client/internal/locator/wan/WANFactoryImpl.java b/geode-wan/src/main/java/org/apache/geode/cache/client/internal/locator/wan/WANFactoryImpl.java
index 43d3871..1687c5d 100644
--- a/geode-wan/src/main/java/org/apache/geode/cache/client/internal/locator/wan/WANFactoryImpl.java
+++ b/geode-wan/src/main/java/org/apache/geode/cache/client/internal/locator/wan/WANFactoryImpl.java
@@ -46,7 +46,7 @@ public class WANFactoryImpl implements WANFactory {
 
   @Override
   public GatewaySenderFactory createGatewaySenderFactory(InternalCache cache) {
-    return new GatewaySenderFactoryImpl(cache);
+    return new GatewaySenderFactoryImpl(cache, cache.getStatisticsClock());
   }
 
   @Override
diff --git a/geode-wan/src/main/java/org/apache/geode/internal/cache/wan/AbstractRemoteGatewaySender.java b/geode-wan/src/main/java/org/apache/geode/internal/cache/wan/AbstractRemoteGatewaySender.java
index 47f0b07..53d574f 100644
--- a/geode-wan/src/main/java/org/apache/geode/internal/cache/wan/AbstractRemoteGatewaySender.java
+++ b/geode-wan/src/main/java/org/apache/geode/internal/cache/wan/AbstractRemoteGatewaySender.java
@@ -32,6 +32,7 @@ import org.apache.geode.internal.admin.remote.DistributionLocatorId;
 import org.apache.geode.internal.cache.InternalCache;
 import org.apache.geode.internal.cache.PoolFactoryImpl;
 import org.apache.geode.internal.logging.LogService;
+import org.apache.geode.internal.statistics.StatisticsClock;
 
 public abstract class AbstractRemoteGatewaySender extends AbstractGatewaySender {
   private static final Logger logger = LogService.getLogger();
@@ -39,8 +40,9 @@ public abstract class AbstractRemoteGatewaySender extends AbstractGatewaySender
   /** used to reduce warning logs in case remote locator is down (#47634) */
   protected int proxyFailureTries = 0;
 
-  public AbstractRemoteGatewaySender(InternalCache cache, GatewaySenderAttributes attrs) {
-    super(cache, attrs);
+  public AbstractRemoteGatewaySender(InternalCache cache, StatisticsClock statisticsClock,
+      GatewaySenderAttributes attrs) {
+    super(cache, statisticsClock, attrs);
   }
 
   @Override
diff --git a/geode-wan/src/main/java/org/apache/geode/internal/cache/wan/GatewaySenderFactoryImpl.java b/geode-wan/src/main/java/org/apache/geode/internal/cache/wan/GatewaySenderFactoryImpl.java
index 1f996c2..a56ad85 100644
--- a/geode-wan/src/main/java/org/apache/geode/internal/cache/wan/GatewaySenderFactoryImpl.java
+++ b/geode-wan/src/main/java/org/apache/geode/internal/cache/wan/GatewaySenderFactoryImpl.java
@@ -35,6 +35,7 @@ import org.apache.geode.internal.cache.xmlcache.CacheCreation;
 import org.apache.geode.internal.cache.xmlcache.ParallelGatewaySenderCreation;
 import org.apache.geode.internal.cache.xmlcache.SerialGatewaySenderCreation;
 import org.apache.geode.internal.logging.LogService;
+import org.apache.geode.internal.statistics.StatisticsClock;
 
 /**
  * @since GemFire 7.0
@@ -43,19 +44,22 @@ public class GatewaySenderFactoryImpl implements InternalGatewaySenderFactory {
 
   private static final Logger logger = LogService.getLogger();
 
+  private static final AtomicBoolean GATEWAY_CONNECTION_READ_TIMEOUT_PROPERTY_CHECKED =
+      new AtomicBoolean(false);
+
   /**
    * Used internally to pass the attributes from this factory to the real GatewaySender it is
    * creating.
    */
-  private GatewaySenderAttributes attrs = new GatewaySenderAttributes();
+  private final GatewaySenderAttributes attrs = new GatewaySenderAttributes();
 
-  private InternalCache cache;
+  private final InternalCache cache;
 
-  private static final AtomicBoolean GATEWAY_CONNECTION_READ_TIMEOUT_PROPERTY_CHECKED =
-      new AtomicBoolean(false);
+  private final StatisticsClock statisticsClock;
 
-  public GatewaySenderFactoryImpl(InternalCache cache) {
+  public GatewaySenderFactoryImpl(InternalCache cache, StatisticsClock statisticsClock) {
     this.cache = cache;
+    this.statisticsClock = statisticsClock;
   }
 
   @Override
@@ -246,7 +250,7 @@ public class GatewaySenderFactoryImpl implements InternalGatewaySenderFactory {
                 id, this.attrs.getOrderPolicy()));
       }
       if (this.cache instanceof GemFireCacheImpl) {
-        sender = new ParallelGatewaySenderImpl(this.cache, this.attrs);
+        sender = new ParallelGatewaySenderImpl(cache, statisticsClock, attrs);
         this.cache.addGatewaySender(sender);
 
         if (!this.attrs.isManualStart()) {
@@ -260,14 +264,14 @@ public class GatewaySenderFactoryImpl implements InternalGatewaySenderFactory {
       if (this.attrs.getAsyncEventListeners().size() > 0) {
         throw new GatewaySenderException(
             String.format(
-                "SerialGatewaySener %s cannot define a remote site because at least AsyncEventListener is already added. Both listeners and remote site cannot be defined for the same gateway sender.",
+                "SerialGatewaySender %s cannot define a remote site because at least AsyncEventListener is already added. Both listeners and remote site cannot be defined for the same gateway sender.",
                 id));
       }
       if (this.attrs.getOrderPolicy() == null && this.attrs.getDispatcherThreads() > 1) {
         this.attrs.policy = GatewaySender.DEFAULT_ORDER_POLICY;
       }
       if (this.cache instanceof GemFireCacheImpl) {
-        sender = new SerialGatewaySenderImpl(this.cache, this.attrs);
+        sender = new SerialGatewaySenderImpl(cache, statisticsClock, attrs);
         this.cache.addGatewaySender(sender);
 
         if (!this.attrs.isManualStart()) {
@@ -302,7 +306,7 @@ public class GatewaySenderFactoryImpl implements InternalGatewaySenderFactory {
       }
 
       if (this.cache instanceof GemFireCacheImpl) {
-        sender = new ParallelGatewaySenderImpl(this.cache, this.attrs);
+        sender = new ParallelGatewaySenderImpl(cache, statisticsClock, attrs);
         this.cache.addGatewaySender(sender);
         if (!this.attrs.isManualStart()) {
           sender.start();
@@ -316,7 +320,7 @@ public class GatewaySenderFactoryImpl implements InternalGatewaySenderFactory {
         this.attrs.policy = GatewaySender.DEFAULT_ORDER_POLICY;
       }
       if (this.cache instanceof GemFireCacheImpl) {
-        sender = new SerialGatewaySenderImpl(this.cache, this.attrs);
+        sender = new SerialGatewaySenderImpl(cache, statisticsClock, attrs);
         this.cache.addGatewaySender(sender);
         if (!this.attrs.isManualStart()) {
           sender.start();
diff --git a/geode-wan/src/main/java/org/apache/geode/internal/cache/wan/parallel/ParallelGatewaySenderImpl.java b/geode-wan/src/main/java/org/apache/geode/internal/cache/wan/parallel/ParallelGatewaySenderImpl.java
index a5c19d2..565a9f1 100644
--- a/geode-wan/src/main/java/org/apache/geode/internal/cache/wan/parallel/ParallelGatewaySenderImpl.java
+++ b/geode-wan/src/main/java/org/apache/geode/internal/cache/wan/parallel/ParallelGatewaySenderImpl.java
@@ -36,6 +36,7 @@ import org.apache.geode.internal.cache.wan.GatewaySenderAdvisor.GatewaySenderPro
 import org.apache.geode.internal.cache.wan.GatewaySenderAttributes;
 import org.apache.geode.internal.logging.LogService;
 import org.apache.geode.internal.monitoring.ThreadsMonitoring;
+import org.apache.geode.internal.statistics.StatisticsClock;
 
 /**
  * @since GemFire 7.0
@@ -44,8 +45,9 @@ public class ParallelGatewaySenderImpl extends AbstractRemoteGatewaySender {
 
   private static final Logger logger = LogService.getLogger();
 
-  public ParallelGatewaySenderImpl(InternalCache cache, GatewaySenderAttributes attrs) {
-    super(cache, attrs);
+  public ParallelGatewaySenderImpl(InternalCache cache, StatisticsClock statisticsClock,
+      GatewaySenderAttributes attrs) {
+    super(cache, statisticsClock, attrs);
   }
 
   @Override
diff --git a/geode-wan/src/main/java/org/apache/geode/internal/cache/wan/serial/SerialGatewaySenderImpl.java b/geode-wan/src/main/java/org/apache/geode/internal/cache/wan/serial/SerialGatewaySenderImpl.java
index d5cfe31..6033036 100644
--- a/geode-wan/src/main/java/org/apache/geode/internal/cache/wan/serial/SerialGatewaySenderImpl.java
+++ b/geode-wan/src/main/java/org/apache/geode/internal/cache/wan/serial/SerialGatewaySenderImpl.java
@@ -37,6 +37,7 @@ import org.apache.geode.internal.cache.wan.GatewaySenderAttributes;
 import org.apache.geode.internal.cache.wan.GatewaySenderConfigurationException;
 import org.apache.geode.internal.logging.LogService;
 import org.apache.geode.internal.monitoring.ThreadsMonitoring;
+import org.apache.geode.internal.statistics.StatisticsClock;
 
 /**
  * @since GemFire 7.0
@@ -45,8 +46,9 @@ public class SerialGatewaySenderImpl extends AbstractRemoteGatewaySender {
 
   private static final Logger logger = LogService.getLogger();
 
-  public SerialGatewaySenderImpl(InternalCache cache, GatewaySenderAttributes attrs) {
-    super(cache, attrs);
+  public SerialGatewaySenderImpl(InternalCache cache, StatisticsClock statisticsClock,
+      GatewaySenderAttributes attrs) {
+    super(cache, statisticsClock, attrs);
   }
 
   @Override