You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@geode.apache.org by ud...@apache.org on 2018/12/04 17:57:15 UTC

[geode] 01/03: Refactored stats to have public interfaces in own package Refactored DistributedSystem.java and InternalDistributedSystem.java to now contain a InternalDistributedSystemStats.java instead of being one Added Micrometer stats implementations Added StatsFactory, which currently only provides Micrometer implementations Moving geode-stats-common packages to base package of org.apache.geode.stats.common Adding `geode.metrics.prometheus.address` and `geode.experimental.stats.micrometer` properties

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

udo pushed a commit to branch feature/Micrometer
in repository https://gitbox.apache.org/repos/asf/geode.git

commit c805a9889602dac7d3b8de3dc92e9fc7d15aade4
Author: Udo Kohlmeyer <uk...@pivotal.io>
AuthorDate: Wed Jul 11 17:05:17 2018 -0700

    Refactored stats to have public interfaces in own package
    Refactored DistributedSystem.java and InternalDistributedSystem.java to
    now contain a InternalDistributedSystemStats.java instead of being one
    Added Micrometer stats implementations
    Added StatsFactory, which currently only provides Micrometer
    implementations
    Moving geode-stats-common packages to base package of org.apache.geode.stats.common
    Adding `geode.metrics.prometheus.address` and
    `geode.experimental.stats.micrometer` properties
---
 build.gradle                                       |    5 +
 .../internal/common/AbstractSessionCache.java      |    5 +-
 .../session/catalina/AbstractSessionCache.java     |    5 +-
 .../catalina/internal/DeltaSessionStatistics.java  |   32 +-
 .../geode/modules/util/ModuleStatistics.java       |   44 +-
 geode-assembly/build.gradle                        |   10 +
 geode-connectors/build.gradle                      |    9 +-
 geode-core/build.gradle                            |   28 +
 .../geode/cache/ConnectionPoolDUnitTest.java       |   12 +-
 .../management/MemoryThresholdsDUnitTest.java      |    5 +-
 .../CacheRegionsReliablityStatsCheckDUnitTest.java |    2 +-
 .../geode/cache30/ClientMembershipDUnitTest.java   |    8 +-
 .../geode/cache30/ClientServerCCEDUnitTest.java    |    6 +-
 .../DistributedMulticastRegionDUnitTest.java       |    2 +-
 ...tedMulticastRegionWithUDPSecurityDUnitTest.java |    2 +-
 .../org/apache/geode/cache30/ProxyDUnitTest.java   |    2 +-
 .../apache/geode/cache30/QueueMsgDUnitTest.java    |    2 +-
 .../org/apache/geode/cache30/SlowRecDUnitTest.java |    2 +-
 .../cache/DeltaPropagationStatsDUnitTest.java      |    5 +-
 .../geode/internal/cache/GIIDeltaDUnitTest.java    |    1 +
 .../internal/cache/GIIFlowControlDUnitTest.java    |    2 +-
 .../cache/P2PDeltaPropagationDUnitTest.java        |    1 +
 ...itionedRegionHAFailureAndRecoveryDUnitTest.java |    1 +
 .../cache/PartitionedRegionStatsDUnitTest.java     |    4 +-
 .../cache/control/RebalanceOperationDUnitTest.java |    1 +
 .../execute/FunctionServiceStatsDUnitTest.java     |  126 +-
 .../HARegionQueueThreadIdExpiryRegressionTest.java |    1 +
 ...eueRemovalMessageProcessingDistributedTest.java |    2 +-
 .../PersistentRecoveryOrderDUnitTest.java          |    2 +-
 .../ClientProxyWithDeltaDistributedTest.java       |    2 +-
 .../tier/sockets/DurableClientStatsDUnitTest.java  |    4 +-
 .../compression/CompressionStatsDUnitTest.java     |    7 +-
 .../statistics/StatisticsDistributedTest.java      |   45 +-
 .../geode/management/DiskManagementDUnitTest.java  |    2 +-
 .../management/MemberMBeanAttributesDUnitTest.java |    2 +-
 .../java/org/apache/geode/TXJUnitTest.java         |    2 +-
 .../internal/CacheHealthEvaluatorJUnitTest.java    |    2 +-
 .../internal/MemberHealthEvaluatorJUnitTest.java   |    2 +-
 .../org/apache/geode/cache/ProxyJUnitTest.java     |    2 +-
 .../AutoConnectionSourceImplJUnitTest.java         |   16 +-
 .../client/internal/QueueManagerJUnitTest.java     |   37 +-
 .../pooling/ConnectionManagerJUnitTest.java        |   19 +-
 .../apache/geode/cache/query/QueryJUnitTest.java   |    1 +
 .../IteratorTypeDefaultTypesJUnitTest.java         |    6 +-
 .../geode/cache30/DiskRegionIntegrationTest.java   |    2 +-
 .../internal/membership/MembershipJUnitTest.java   |    2 +-
 .../gms/fd/GMSHealthMonitorJUnitTest.java          |   17 +-
 .../gms/locator/GMSLocatorRecoveryJUnitTest.java   |   13 +-
 .../gms/messenger/JGroupsMessengerJUnitTest.java   |   15 +-
 .../gms/mgr/GMSMembershipManagerJUnitTest.java     |    2 +-
 .../tcpserver/TCPClientSSLIntegrationTest.java     |    2 +-
 .../internal/tcpserver/TCPServerSSLJUnitTest.java  |    2 +-
 .../internal/tcpserver/TcpServerJUnitTest.java     |    2 +-
 .../internal/cache/ComplexDiskRegionJUnitTest.java |   15 +-
 .../internal/cache/DiskInitFileJUnitTest.java      |   11 +-
 .../DiskRegOplogSwtchingAndRollerJUnitTest.java    |    1 +
 .../geode/internal/cache/DiskRegionJUnitTest.java  |    2 +
 .../cache/DiskStoreImplIntegrationTest.java        |    1 +
 .../LIFOEvictionAlgoEnabledRegionJUnitTest.java    |    1 +
 ...FOEvictionAlgoMemoryEnabledRegionJUnitTest.java |    1 +
 .../geode/internal/cache/OplogJUnitTest.java       |   40 +-
 .../geode/internal/cache/OplogRVVJUnitTest.java    |   13 +-
 .../cache/PartitionedRegionStatsJUnitTest.java     |    3 +-
 .../internal/cache/SimpleDiskRegionJUnitTest.java  |    6 +-
 .../control/MemoryMonitorOffHeapJUnitTest.java     |    1 +
 .../cache/ha/HARegionQueueIntegrationTest.java     |   11 +-
 .../cache/ha/HARegionQueueStatsJUnitTest.java      |    1 +
 .../CacheClientNotifierIntegrationTest.java        |    9 +-
 .../CacheServerMaxConnectionsJUnitTest.java        |    8 +-
 .../ClientHealthMonitorIntegrationTest.java        |   15 +-
 .../sockets/ServerConnectionIntegrationTest.java   |    8 +-
 .../MemoryAllocatorFillPatternIntegrationTest.java |    2 +-
 .../statistics/DiskSpaceLimitIntegrationTest.java  |   10 +-
 ...DistributedSystemStatisticsIntegrationTest.java |   10 +-
 ...ributedSystemStatisticsTypeIntegrationTest.java |    8 +-
 .../statistics/FileSizeLimitIntegrationTest.java   |   10 +-
 .../GemFireStatSamplerIntegrationTest.java         |   82 +-
 .../internal/statistics/LinuxSystemStatsTest.java  |    8 +-
 .../SimpleStatSamplerIntegrationTest.java          |    6 +-
 ...rchiveWithConsecutiveResourceInstGenerator.java |    8 +-
 ...chiveWithMissingResourceTypeRegressionTest.java |    4 +-
 .../StatArchiveWriterReaderIntegrationTest.java    |   22 +-
 .../statistics/StatSamplerIntegrationTest.java     |    7 +-
 .../StatTypesAreRolledOverRegressionTest.java      |   10 +-
 .../statistics/ValueMonitorIntegrationTest.java    |   48 +-
 .../internal/stats50/AtomicStatsJUnitTest.java     |   17 +-
 .../bean/stats/AsyncEventQueueStatsJUnitTest.java  |    6 +-
 .../bean/stats/CacheServerStatsJUnitTest.java      |    8 +-
 .../management/bean/stats/DiskStatsJUnitTest.java  |    6 +-
 .../stats/DistributedSystemStatsJUnitTest.java     |   12 +-
 .../bean/stats/GatewayMBeanBridgeJUnitTest.java    |    5 +-
 .../bean/stats/GatewayReceiverStatsJUnitTest.java  |    9 +-
 .../bean/stats/MemberLevelStatsJUnitTest.java      |   43 +-
 .../bean/stats/RegionStatsJUnitTest.java           |   18 +-
 .../internal/beans/ManagementAdapterTest.java      |    7 +-
 .../geode/pdx/OffHeapByteSourceJUnitTest.java      |    4 +-
 .../geode/admin/internal/CacheHealthEvaluator.java |    2 +-
 .../admin/internal/MemberHealthEvaluator.java      |    6 +-
 .../asyncqueue/internal/AsyncEventQueueStats.java  |  119 +-
 .../internal/ParallelAsyncEventQueueImpl.java      |    5 +-
 .../internal/SerialAsyncEventQueueImpl.java        |    5 +-
 .../geode/cache/client/internal/AbstractOp.java    |    1 +
 .../geode/cache/client/internal/AddPDXEnumOp.java  |    1 +
 .../geode/cache/client/internal/AddPDXTypeOp.java  |    1 +
 .../cache/client/internal/AuthenticateUserOp.java  |    1 +
 .../geode/cache/client/internal/ClearOp.java       |    1 +
 .../cache/client/internal/CloseConnectionOp.java   |    1 +
 .../geode/cache/client/internal/CommitOp.java      |    1 +
 .../geode/cache/client/internal/Connection.java    |    1 +
 .../cache/client/internal/ConnectionImpl.java      |    1 +
 .../cache/client/internal/ConnectionStats.java     | 3787 --------------------
 .../cache/client/internal/ConnectionStatsImpl.java | 3289 +++++++++++++++++
 .../geode/cache/client/internal/ContainsKeyOp.java |    1 +
 .../geode/cache/client/internal/DestroyOp.java     |    1 +
 .../cache/client/internal/DestroyRegionOp.java     |    1 +
 .../geode/cache/client/internal/Endpoint.java      |    1 +
 .../cache/client/internal/EndpointManager.java     |    1 +
 .../cache/client/internal/EndpointManagerImpl.java |   26 +-
 .../client/internal/ExecuteFunctionNoAckOp.java    |    1 +
 .../cache/client/internal/ExecuteFunctionOp.java   |    7 +-
 .../internal/ExecuteRegionFunctionNoAckOp.java     |    1 +
 .../client/internal/ExecuteRegionFunctionOp.java   |    7 +-
 .../internal/ExecuteRegionFunctionSingleHopOp.java |    7 +-
 .../geode/cache/client/internal/GetAllOp.java      |    1 +
 .../client/internal/GetClientPRMetaDataOp.java     |    1 +
 .../internal/GetClientPartitionAttributesOp.java   |    1 +
 .../geode/cache/client/internal/GetEntryOp.java    |    1 +
 .../cache/client/internal/GetEventValueOp.java     |    1 +
 .../client/internal/GetFunctionAttributeOp.java    |    1 +
 .../apache/geode/cache/client/internal/GetOp.java  |    1 +
 .../cache/client/internal/GetPDXEnumByIdOp.java    |    1 +
 .../geode/cache/client/internal/GetPDXEnumsOp.java |    1 +
 .../cache/client/internal/GetPDXIdForEnumOp.java   |    1 +
 .../cache/client/internal/GetPDXIdForTypeOp.java   |    1 +
 .../cache/client/internal/GetPDXTypeByIdOp.java    |    1 +
 .../geode/cache/client/internal/GetPDXTypesOp.java |    1 +
 .../cache/client/internal/InternalClientCache.java |    2 +-
 .../geode/cache/client/internal/InternalPool.java  |    2 +-
 .../geode/cache/client/internal/InvalidateOp.java  |    1 +
 .../geode/cache/client/internal/KeySetOp.java      |    1 +
 .../geode/cache/client/internal/MakePrimaryOp.java |    1 +
 .../apache/geode/cache/client/internal/PingOp.java |    1 +
 .../geode/cache/client/internal/PoolImpl.java      |   73 +-
 .../geode/cache/client/internal/PrimaryAckOp.java  |    1 +
 .../cache/client/internal/ProxyCacheCloseOp.java   |    1 +
 .../geode/cache/client/internal/PutAllOp.java      |    1 +
 .../apache/geode/cache/client/internal/PutOp.java  |    1 +
 .../geode/cache/client/internal/QueryOp.java       |    1 +
 .../cache/client/internal/QueueConnectionImpl.java |    1 +
 .../cache/client/internal/ReadyForEventsOp.java    |    1 +
 .../client/internal/RegisterDataSerializersOp.java |    1 +
 .../client/internal/RegisterInstantiatorsOp.java   |    1 +
 .../cache/client/internal/RegisterInterestOp.java  |    1 +
 .../geode/cache/client/internal/RemoveAllOp.java   |    1 +
 .../geode/cache/client/internal/RollbackOp.java    |    1 +
 .../apache/geode/cache/client/internal/SizeOp.java |    1 +
 .../geode/cache/client/internal/TXFailoverOp.java  |    1 +
 .../cache/client/internal/TXSynchronizationOp.java |    1 +
 .../client/internal/UnregisterInterestListOp.java  |    1 +
 .../client/internal/UnregisterInterestOp.java      |    1 +
 .../internal/pooling/ConnectionManagerImpl.java    |    2 +-
 .../client/internal/pooling/PooledConnection.java  |    2 +-
 .../java/org/apache/geode/cache/query/CqQuery.java |    4 +-
 .../java/org/apache/geode/cache/query/Query.java   |    1 +
 .../org/apache/geode/cache/query/QueryService.java |    1 +
 .../org/apache/geode/cache/query/internal/Bag.java |    2 +-
 ...QueryVsdStats.java => CqQueryVsdStatsImpl.java} |  111 +-
 .../geode/cache/query/internal/DefaultQuery.java   |    4 +-
 .../cache/query/internal/DefaultQueryService.java  |   16 +-
 .../cache/query/internal/ExecutionContext.java     |    2 +-
 .../cache/query/internal/ProxyQueryService.java    |    2 +-
 .../geode/cache/query/internal/QueryUtils.java     |    2 +-
 .../geode/cache/query/internal/ResultsBag.java     |    2 +-
 .../cache/query/internal/SortedResultsBag.java     |    2 +-
 .../cache/query/internal/SortedStructBag.java      |    2 +-
 .../geode/cache/query/internal/StructBag.java      |    2 +-
 .../geode/cache/query/internal/cq/CqService.java   |   39 +-
 .../cache/query/internal/cq/InternalCqQuery.java   |    2 +-
 .../cache/query/internal/cq/MissingCqService.java  |   48 +-
 .../internal/cq/MissingCqServiceStatistics.java    |    2 +-
 .../query/internal/index/AbstractMapIndex.java     |    3 +-
 .../query/internal/index/CompactRangeIndex.java    |    3 +-
 .../cache/query/internal/index/HashIndex.java      |    3 +-
 .../cache/query/internal/index/HashIndexSet.java   |    4 +-
 .../cache/query/internal/index/IndexManager.java   |    2 +-
 .../cache/query/internal/index/IndexStats.java     |   76 +-
 .../query/internal/index/PartitionedIndex.java     |    3 +-
 .../cache/query/internal/index/RangeIndex.java     |    3 +-
 .../geode/cache/server/internal/LoadMonitor.java   |    8 +-
 .../geode/distributed/DistributedSystem.java       |    7 +-
 .../internal/ClusterDistributionManager.java       |  257 +-
 .../distributed/internal/DistributionManager.java  |    1 +
 .../distributed/internal/DistributionMessage.java  |   12 +-
 ...butionStats.java => DistributionStatsImpl.java} | 1171 +++---
 .../internal/FunctionExecutionPooledExecutor.java  |    1 +
 .../internal/InternalDistributedSystem.java        |  415 +--
 .../distributed/internal/InternalLocator.java      |   41 +-
 .../{LocatorStats.java => LocatorStatsImpl.java}   |   93 +-
 .../internal/LonerDistributionManager.java         |    5 +-
 .../internal/OverflowQueueWithDMStats.java         |    2 +
 .../internal/PooledExecutorWithDMStats.java        |    1 +
 .../geode/distributed/internal/ReplyMessage.java   |    4 +-
 .../internal/RuntimeDistributionConfigImpl.java    |    2 +-
 .../internal/SerialQueuedExecutorWithDMStats.java  |    1 +
 .../geode/distributed/internal/ServerLocator.java  |    1 +
 .../ThrottlingMemLinkedQueueWithDMStats.java       |    8 +-
 .../distributed/internal/direct/DirectChannel.java |    2 +-
 .../distributed/internal/locks/DLockGrantor.java   |    3 +-
 .../internal/locks/DLockRequestProcessor.java      |    3 +-
 .../distributed/internal/locks/DLockService.java   |   19 +-
 .../locks/{DLockStats.java => DLockStatsImpl.java} |  428 +--
 .../internal/locks/DummyDLockStats.java            |   18 +-
 .../internal/membership/MemberFactory.java         |    4 +-
 .../internal/membership/MemberServices.java        |    4 +-
 .../internal/membership/MembershipManager.java     |    2 +-
 .../internal/membership/gms/GMSMemberFactory.java  |    4 +-
 .../internal/membership/gms/Services.java          |    2 +-
 .../membership/gms/fd/GMSHealthMonitor.java        |    2 +-
 .../membership/gms/locator/GMSLocator.java         |    2 +-
 .../membership/gms/messenger/JGroupsMessenger.java |    7 +-
 .../membership/gms/messenger/StatRecorder.java     |    2 +-
 .../membership/gms/mgr/GMSMembershipManager.java   |    2 +-
 .../distributed/internal/tcpserver/TcpServer.java  |    9 +-
 .../org/apache/geode/internal/DSFIDFactory.java    |    6 +-
 .../org/apache/geode/internal/HistogramStats.java  |    8 +-
 .../geode/internal/InternalDataSerializer.java     |    2 +-
 .../geode/internal/admin/ClientStatsManager.java   |   18 +-
 .../geode/internal/admin/StatAlertDefinition.java  |    2 +-
 .../geode/internal/admin/StatAlertsManager.java    |    9 +-
 .../admin/remote/AddStatListenerResponse.java      |    2 +-
 .../remote/AdminConsoleDisconnectMessage.java      |    2 +-
 .../admin/remote/CancelStatListenerResponse.java   |    2 +-
 .../remote/FetchResourceAttributesResponse.java    |   39 +-
 .../internal/admin/remote/FetchStatsResponse.java  |   28 +-
 .../geode/internal/admin/remote/RemoteStat.java    |    4 +-
 .../internal/admin/remote/RemoteStatResource.java  |    2 +-
 .../admin/statalerts/BaseDecoratorImpl.java        |    2 +-
 .../admin/statalerts/DummyStatisticInfoImpl.java   |    4 +-
 .../admin/statalerts/FunctionDecoratorImpl.java    |    2 +-
 .../statalerts/GaugeThresholdDecoratorImpl.java    |    2 +-
 .../admin/statalerts/MultiAttrDefinitionImpl.java  |    6 +-
 .../statalerts/NumberThresholdDecoratorImpl.java   |    2 +-
 .../admin/statalerts/SingleAttrDefinitionImpl.java |    6 +-
 .../internal/admin/statalerts/StatisticInfo.java   |    6 +-
 .../admin/statalerts/StatisticInfoImpl.java        |    8 +-
 .../internal/cache/AbstractBucketRegionQueue.java  |    3 +-
 .../geode/internal/cache/AbstractDiskRegion.java   |    3 +-
 .../geode/internal/cache/AbstractRegion.java       |    4 +-
 .../geode/internal/cache/AbstractRegionMap.java    |    1 +
 .../internal/cache/AbstractUpdateOperation.java    |    1 +
 .../apache/geode/internal/cache/BucketAdvisor.java |    1 +
 .../internal/cache/BucketPersistenceAdvisor.java   |    1 +
 .../apache/geode/internal/cache/BucketRegion.java  |    4 +-
 .../geode/internal/cache/CachePerfStats.java       | 1537 --------
 .../geode/internal/cache/CachePerfStatsImpl.java   | 1785 +++++++++
 .../geode/internal/cache/DirectoryHolder.java      |   11 +-
 ...ctoryStats.java => DiskDirectoryStatsImpl.java} |   62 +-
 .../apache/geode/internal/cache/DiskInitFile.java  |    1 +
 .../apache/geode/internal/cache/DiskRegion.java    |    1 +
 ...skRegionStats.java => DiskRegionStatsImpl.java} |  110 +-
 .../apache/geode/internal/cache/DiskStoreImpl.java |   12 +-
 ...DiskStoreStats.java => DiskStoreStatsImpl.java} |  311 +-
 .../internal/cache/DistTXRollbackMessage.java      |    8 +-
 .../geode/internal/cache/DistributedRegion.java    |    7 +-
 .../geode/internal/cache/DummyCachePerfStats.java  |   12 +-
 .../geode/internal/cache/EntryEventImpl.java       |    1 +
 .../geode/internal/cache/EvictableRegion.java      |    2 +-
 .../geode/internal/cache/GemFireCacheImpl.java     |  157 +-
 .../org/apache/geode/internal/cache/HARegion.java  |    1 +
 .../geode/internal/cache/HasCachePerfStats.java    |    2 +
 ...ava~d2263ebc2... Create HasDiskRegion interface |   22 -
 .../internal/cache/InitialImageOperation.java      |    1 +
 .../apache/geode/internal/cache/InternalCache.java |    1 +
 .../cache/InternalCacheForClientAccess.java        |    1 +
 .../geode/internal/cache/InternalRegion.java       |    3 +-
 .../apache/geode/internal/cache/LocalRegion.java   | 1003 ++----
 .../cache/MemberFunctionStreamingMessage.java      |    5 +-
 .../cache/OfflineCompactionDiskRegion.java         |    4 +-
 .../org/apache/geode/internal/cache/Oplog.java     |    1 +
 .../apache/geode/internal/cache/OverflowOplog.java |    1 +
 .../internal/cache/PRHARedundancyProvider.java     |    3 +-
 .../geode/internal/cache/PartitionedRegion.java    |  344 +-
 .../internal/cache/PartitionedRegionDataStore.java |  220 +-
 .../internal/cache/PartitionedRegionHelper.java    |    9 +-
 .../cache/PartitionedRegionRedundancyTracker.java  |    1 +
 ...nStats.java => PartitionedRegionStatsImpl.java} |  606 ++--
 .../internal/cache/PartitionedRegionStatus.java    |    2 +
 .../cache/{PoolStats.java => PoolStatsImpl.java}   |  214 +-
 .../geode/internal/cache/ProxyBucketRegion.java    |    1 +
 .../geode/internal/cache/RegionPerfStatsImpl.java  |  582 +++
 .../cache/SearchLoadAndWriteProcessor.java         |    1 +
 .../apache/geode/internal/cache/TXManagerImpl.java |    1 +
 .../internal/cache/TXRemoteCommitMessage.java      |    8 +-
 .../org/apache/geode/internal/cache/TXState.java   |    5 +-
 .../geode/internal/cache/TombstoneService.java     |    1 +
 .../geode/internal/cache/ValidatingDiskRegion.java |   38 +-
 .../client/protocol/ClientProtocolService.java     |    2 +-
 .../internal/cache/control/HeapMemoryMonitor.java  |   21 +-
 .../cache/control/InternalResourceManager.java     |    5 +-
 .../cache/control/OffHeapMemoryMonitor.java        |    1 +
 .../cache/control/RebalanceOperationImpl.java      |    1 +
 ...gerStats.java => ResourceManagerStatsImpl.java} |  269 +-
 .../cache/eviction/AbstractEvictionController.java |   21 +-
 ...Statistics.java => CountLRUStatisticsImpl.java} |   55 +-
 .../cache/eviction/DisabledEvictionCounters.java   |    2 +-
 .../internal/cache/eviction/EvictionCounters.java  |    2 +-
 .../cache/eviction/EvictionCountersImpl.java       |    3 +-
 ...UStatistics.java => HeapLRUStatisticsImpl.java} |   58 +-
 ...tatistics.java => MemoryLRUStatisticsImpl.java} |   53 +-
 .../internal/cache/eviction/RegionEvictorTask.java |    2 +-
 .../internal/cache/execute/AbstractExecution.java  |    5 +-
 .../DistributedRegionFunctionResultSender.java     |   20 +-
 ...iceStats.java => FunctionServiceStatsImpl.java} |   95 +-
 .../{FunctionStats.java => FunctionStatsImpl.java} |  214 +-
 .../cache/execute/MemberFunctionResultSender.java  |   16 +-
 .../PartitionedRegionFunctionResultSender.java     |   22 +-
 .../cache/execute/ServerFunctionExecutor.java      |    9 +-
 .../execute/ServerRegionFunctionExecutor.java      |   11 +-
 .../ServerToClientFunctionResultSender.java        |    8 +-
 .../ServerToClientFunctionResultSender65.java      |    8 +-
 .../geode/internal/cache/ha/HARegionQueue.java     |   10 +-
 ...QueueStats.java => HARegionQueueStatsImpl.java} |  105 +-
 .../cache/partitioned/BucketSizeMessage.java       |    3 +-
 .../cache/partitioned/ContainsKeyValueMessage.java |    3 +-
 .../cache/partitioned/CreateBucketMessage.java     |    3 +-
 .../partitioned/CreateMissingBucketsTask.java      |    3 +-
 .../internal/cache/partitioned/DumpB2NRegion.java  |    3 +-
 .../cache/partitioned/FetchBulkEntriesMessage.java |    3 +-
 .../cache/partitioned/FetchEntriesMessage.java     |    3 +-
 .../cache/partitioned/FetchKeysMessage.java        |    3 +-
 .../internal/cache/partitioned/GetMessage.java     |   45 +-
 .../cache/partitioned/IdentityRequestMessage.java  |    3 +-
 .../cache/partitioned/InterestEventMessage.java    |    3 +-
 .../partitioned/ManageBackupBucketMessage.java     |    3 +-
 .../cache/partitioned/ManageBucketMessage.java     |    3 +-
 .../partitioned/PartitionedRegionRebalanceOp.java  |    2 +-
 .../internal/cache/partitioned/RegionAdvisor.java  |    2 +-
 .../internal/cache/partitioned/SizeMessage.java    |    3 +-
 .../rebalance/BucketOperatorWrapper.java           |    2 +-
 .../cache/persistence/PersistenceAdvisorImpl.java  |    2 +-
 .../cache/snapshot/RegionSnapshotServiceImpl.java  |    2 +-
 .../internal/cache/tier/sockets/AcceptorImpl.java  |  430 +--
 .../internal/cache/tier/sockets/BaseCommand.java   |    4 +-
 .../cache/tier/sockets/BaseCommandQuery.java       |    8 +-
 .../cache/tier/sockets/CacheClientNotifier.java    |  109 +-
 ...tats.java => CacheClientNotifierStatsImpl.java} |  135 +-
 .../cache/tier/sockets/CacheClientProxy.java       |    6 +-
 .../cache/tier/sockets/CacheClientProxyStats.java  |   75 +-
 .../cache/tier/sockets/CacheClientUpdater.java     |   46 +-
 ...eServerStats.java => CacheServerStatsImpl.java} |  124 +-
 .../cache/tier/sockets/ClientHealthMonitor.java    |    8 +-
 .../geode/internal/cache/tier/sockets/Message.java |    5 +-
 .../tier/sockets/OriginalServerConnection.java     |    1 +
 .../tier/sockets/ProtobufServerConnection.java     |    1 +
 .../cache/tier/sockets/ServerConnection.java       |    2 +
 .../tier/sockets/ServerConnectionFactory.java      |    6 +-
 .../cache/tier/sockets/TcpServerFactory.java       |    2 +-
 .../cache/tier/sockets/command/ClearRegion.java    |    9 +-
 .../cache/tier/sockets/command/ClientReady.java    |    9 +-
 .../tier/sockets/command/CloseConnection.java      |    7 +-
 .../cache/tier/sockets/command/ContainsKey.java    |    9 +-
 .../cache/tier/sockets/command/ContainsKey66.java  |    9 +-
 .../cache/tier/sockets/command/Destroy.java        |    9 +-
 .../cache/tier/sockets/command/Destroy65.java      |    9 +-
 .../cache/tier/sockets/command/DestroyRegion.java  |    9 +-
 .../tier/sockets/command/ExecuteFunction.java      |    5 +-
 .../tier/sockets/command/ExecuteFunction65.java    |    5 +-
 .../tier/sockets/command/ExecuteFunction66.java    |   29 +-
 .../sockets/command/GatewayReceiverCommand.java    |    9 +-
 .../internal/cache/tier/sockets/command/Get70.java |    9 +-
 .../cache/tier/sockets/command/Invalidate.java     |    9 +-
 .../internal/cache/tier/sockets/command/Ping.java  |    3 +-
 .../internal/cache/tier/sockets/command/Put.java   |    9 +-
 .../internal/cache/tier/sockets/command/Put61.java |    9 +-
 .../internal/cache/tier/sockets/command/Put65.java |    9 +-
 .../cache/tier/sockets/command/PutAll.java         |    9 +-
 .../cache/tier/sockets/command/PutAll70.java       |    9 +-
 .../cache/tier/sockets/command/PutAll80.java       |    9 +-
 .../cache/tier/sockets/command/RemoveAll.java      |    9 +-
 .../cache/tier/sockets/command/Request.java        |   11 +-
 .../internal/cache/tier/sockets/command/Size.java  |    7 +-
 .../sockets/command/UpdateClientNotification.java  |    7 +-
 .../internal/cache/tx/PartitionedTXRegionStub.java |    5 +-
 .../internal/cache/tx/RemoteClearMessage.java      |    3 +-
 .../cache/tx/RemoteContainsKeyValueMessage.java    |    3 +-
 .../internal/cache/tx/RemoteFetchKeysMessage.java  |    3 +-
 .../geode/internal/cache/tx/RemoteGetMessage.java  |   10 +-
 .../geode/internal/cache/tx/RemoteSizeMessage.java |    3 +-
 .../internal/cache/wan/AbstractGatewaySender.java  |   11 +-
 .../wan/AbstractGatewaySenderEventProcessor.java   |    1 +
 ...verStats.java => GatewayReceiverStatsImpl.java} |   70 +-
 .../wan/GatewaySenderEventCallbackDispatcher.java  |    1 +
 ...enderStats.java => GatewaySenderStatsImpl.java} |  422 ++-
 .../internal/cache/wan/InternalGatewaySender.java  |    1 +
 .../wan/parallel/ParallelGatewaySenderQueue.java   |    4 +-
 .../serial/SerialGatewaySenderEventProcessor.java  |    2 +-
 .../cache/wan/serial/SerialGatewaySenderQueue.java |    2 +-
 .../internal/cache/xmlcache/CacheCreation.java     |    4 +-
 .../apache/geode/internal/concurrent/Atomics.java  |    7 +-
 .../monitoring/ThreadsMonitoringProcess.java       |    2 +-
 .../apache/geode/internal/net/SocketCloser.java    |    5 +-
 .../geode/internal/offheap/FreeListManager.java    |    5 +-
 .../geode/internal/offheap/MemoryAllocator.java    |    4 +-
 .../internal/offheap/MemoryAllocatorImpl.java      |   17 +-
 .../geode/internal/offheap/OffHeapStorage.java     |  280 +-
 .../internal/offheap/OffHeapStorageStatsImpl.java  |  280 ++
 .../statistics/AbstractStatisticsFactory.java      |   81 +-
 .../geode/internal/statistics/CallbackSampler.java |    6 +-
 .../statistics/DummyStatisticsFactory.java         |   68 +-
 .../internal/statistics/DummyStatisticsImpl.java   |    6 +-
 ...sFactory.java => GFSStatisticsFactoryImpl.java} |  333 +-
 .../internal/statistics/GemFireStatSampler.java    |  173 +-
 .../geode/internal/statistics/HostStatHelper.java  |   37 +-
 .../geode/internal/statistics/HostStatSampler.java |   14 +-
 .../statistics/InternalDistributedSystemStats.java |  165 +
 .../statistics/LocalStatisticsFactory.java         |   11 +-
 .../internal/statistics/LocalStatisticsImpl.java   |    6 +-
 .../statistics/MapBasedStatisticsNotification.java |    6 +-
 .../internal/statistics/ResourceInstance.java      |    4 +-
 .../geode/internal/statistics/ResourceType.java    |    4 +-
 .../geode/internal/statistics/SampleCollector.java |    6 +-
 .../internal/statistics/SimpleStatSampler.java     |   28 +-
 .../internal/statistics/SimpleStatisticId.java     |    4 +-
 .../internal/statistics/StatArchiveWriter.java     |    2 +-
 ...SamplerStats.java => StatSamplerStatsImpl.java} |   90 +-
 .../statistics/StatisticDescriptorImpl.java        |   30 +-
 .../geode/internal/statistics/StatisticId.java     |    4 +-
 .../geode/internal/statistics/StatisticsImpl.java  |   10 +-
 .../statistics/StatisticsNotification.java         |    6 +-
 .../internal/statistics/StatisticsSampler.java     |    2 +-
 .../statistics/StatisticsTypeFactoryImpl.java      |   31 +-
 .../internal/statistics/StatisticsTypeImpl.java    |   11 +-
 .../internal/statistics/StatisticsTypeXml.java     |    6 +-
 .../statistics/VMStatsContractFactory.java         |   17 +-
 .../statistics/{VMStats.java => VMStatsImpl.java}  |   42 +-
 .../geode/internal/statistics/ValueMonitor.java    |    6 +-
 .../statistics/platform/LinuxProcessStats.java     |   33 +-
 .../statistics/platform/LinuxSystemStats.java      |  127 +-
 .../statistics/platform/OSXProcessStats.java       |   42 +-
 .../statistics/platform/OSXSystemStats.java        |  218 +-
 .../statistics/platform/OsStatisticsFactory.java   |    6 +-
 .../internal/statistics/platform/ProcessStats.java |    2 +-
 .../statistics/platform/SolarisProcessStats.java   |   91 +-
 .../statistics/platform/SolarisSystemStats.java    |  182 +-
 .../statistics/platform/WindowsProcessStats.java   |  118 +-
 .../statistics/platform/WindowsSystemStats.java    |  103 +-
 .../internal/stats50/Atomic50StatisticsImpl.java   |    7 +-
 .../apache/geode/internal/stats50/VMStats50.java   |  279 +-
 .../org/apache/geode/internal/tcp/Buffers.java     |    2 +-
 .../org/apache/geode/internal/tcp/Connection.java  |   32 +-
 .../geode/internal/tcp/DirectReplySender.java      |    2 +-
 .../apache/geode/internal/tcp/MsgDestreamer.java   |    2 +-
 .../org/apache/geode/internal/tcp/MsgReader.java   |    2 +-
 .../org/apache/geode/internal/tcp/MsgStreamer.java |    2 +-
 .../org/apache/geode/internal/tcp/TCPConduit.java  |    2 +-
 .../geode/internal/tcp/VersionedMsgStreamer.java   |    2 +-
 .../org/apache/geode/internal/util/BlobHelper.java |    2 +-
 .../management/internal/FederatingManager.java     |   20 +-
 .../geode/management/internal/LocalManager.java    |   23 +-
 .../internal/beans/AsyncEventQueueMBeanBridge.java |    2 +-
 .../internal/beans/DiskRegionBridge.java           |    4 +-
 .../internal/beans/DiskStoreMBeanBridge.java       |    2 +-
 .../internal/beans/GatewayReceiverMBeanBridge.java |    2 +-
 .../internal/beans/GatewaySenderMBeanBridge.java   |    4 +-
 .../internal/beans/MemberMBeanBridge.java          |   88 +-
 .../internal/beans/PartitionedRegionBridge.java    |    2 +-
 .../internal/beans/RegionMBeanBridge.java          |    6 +-
 .../management/internal/beans/ServerBridge.java    |    2 +-
 .../beans/stats/AggregateRegionStatsMonitor.java   |    5 +-
 .../internal/beans/stats/GCStatsMonitor.java       |    2 +-
 .../beans/stats/GatewaySenderOverflowMonitor.java  |    4 +-
 .../internal/beans/stats/MBeanStatsMonitor.java    |    6 +-
 .../beans/stats/MemberLevelDiskMonitor.java        |    5 +-
 .../internal/beans/stats/VMStatsMonitor.java       |    2 +-
 .../GetSubscriptionQueueSizeFunction.java          |    2 +-
 .../src/main/java/org/apache/geode/package.html    |   32 +-
 .../apache/geode/pdx/internal/PdxInstanceImpl.java |    2 +-
 geode-core/src/main/kotlin/Client.kt               |   61 +
 .../QueryStatistics.java => kotlin/Locator.kt}     |   34 +-
 geode-core/src/main/kotlin/Server.kt               |   59 +
 ...geode.stats.common.statistics.StatisticsFactory |    1 +
 .../client/internal/OpExecutorImplJUnitTest.java   |    1 +
 .../org/apache/geode/cache/query/PerfQuery.java    |    1 +
 .../QueryObjectSerializationJUnitTest.java         |    2 +-
 .../org/apache/geode/cache30/TestDiskRegion.java   |    2 +-
 .../internal/cache/AbstractRegionMapTest.java      |    1 +
 .../cache/AbstractRegionMapTxApplyDestroyTest.java |    1 +
 .../cache/BucketPersistenceAdvisorTest.java        |    1 +
 .../geode/internal/cache/DiskRegionTest.java       |    1 +
 .../geode/internal/cache/FlusherThreadTest.java    |    1 +
 .../cache/OffHeapValueWrapperJUnitTest.java        |    4 +-
 .../internal/cache/OldValueImporterTestBase.java   |   10 +-
 .../org/apache/geode/internal/cache/OplogTest.java |    2 +
 .../cache/PartitionedRegionQueryEvaluatorTest.java |    1 +
 .../PartitionedRegionRedundancyTrackerTest.java    |    2 +
 .../cache/SearchLoadAndWriteProcessorTest.java     |    1 +
 .../geode/internal/cache/TXManagerImplTest.java    |    1 +
 .../cache/entries/AbstractRegionEntryTest.java     |    4 +-
 .../geode/internal/cache/ha/HARegionQueueTest.java |    1 +
 .../cache/map/AbstractRegionMapPutTest.java        |    2 +-
 .../internal/cache/map/RegionMapCommitPutTest.java |    2 +-
 .../internal/cache/map/RegionMapDestroyTest.java   |    2 +-
 .../geode/internal/cache/map/RegionMapPutTest.java |    2 +-
 .../cache/partitioned/FetchKeysMessageTest.java    |    2 +-
 .../ManageBackupBucketReplyMessageTest.java        |    2 +-
 .../rebalance/BucketOperatorWrapperTest.java       |    2 +-
 .../internal/cache/tier/sockets/CCUStatsTest.java  |    4 +-
 .../tier/sockets/ClientHealthMonitorJUnitTest.java |    2 +-
 .../cache/tier/sockets/MessageJUnitTest.java       |    1 +
 .../tier/sockets/ServerConnectionFactoryTest.java  |    3 +-
 .../cache/tier/sockets/ServerConnectionTest.java   |    1 +
 .../tier/sockets/command/ContainsKey66Test.java    |    4 +-
 .../tier/sockets/command/ContainsKeyTest.java      |    4 +-
 .../tier/sockets/command/CreateRegionTest.java     |    4 +-
 .../cache/tier/sockets/command/Destroy65Test.java  |    4 +-
 .../tier/sockets/command/DestroyRegionTest.java    |    4 +-
 .../cache/tier/sockets/command/DestroyTest.java    |    4 +-
 .../sockets/command/ExecuteFunction66Test.java     |   11 +-
 .../cache/tier/sockets/command/Get70Test.java      |    4 +-
 .../cache/tier/sockets/command/GetAll651Test.java  |    4 +-
 .../cache/tier/sockets/command/GetAll70Test.java   |    4 +-
 .../cache/tier/sockets/command/GetAllTest.java     |    4 +-
 .../cache/tier/sockets/command/InvalidateTest.java |    4 +-
 .../cache/tier/sockets/command/KeySetTest.java     |    4 +-
 .../cache/tier/sockets/command/Put61Test.java      |    4 +-
 .../cache/tier/sockets/command/Put65Test.java      |    4 +-
 .../cache/tier/sockets/command/PutTest.java        |    4 +-
 .../cache/tier/sockets/command/RemoveAllTest.java  |    4 +-
 .../cache/tier/sockets/command/RequestTest.java    |    4 +-
 .../sockets/command/UnregisterInterestTest.java    |    4 +-
 ...rallelGatewaySenderEventProcessorJUnitTest.java |    2 +-
 .../ParallelGatewaySenderQueueJUnitTest.java       |    2 +-
 .../ParallelQueueRemovalMessageJUnitTest.java      |   10 +-
 ...SerialGatewaySenderEventProcessorJUnitTest.java |    2 +-
 .../internal/offheap/FreeListManagerTest.java      |    3 +-
 .../offheap/LifecycleListenerJUnitTest.java        |   18 +-
 .../MemoryAllocatorFillPatternJUnitTest.java       |    2 +-
 .../internal/offheap/MemoryAllocatorJUnitTest.java |   32 +-
 .../internal/offheap/MemoryBlockNodeJUnitTest.java |    5 +-
 ...moryStats.java => NullOffHeapStorageStats.java} |   22 +-
 .../internal/offheap/OffHeapHelperJUnitTest.java   |    3 +-
 .../offheap/OffHeapRegionEntryHelperJUnitTest.java |    5 +-
 .../internal/offheap/OffHeapStorageJUnitTest.java  |   35 +-
 .../OffHeapStoredObjectAddressStackJUnitTest.java  |   18 +-
 .../offheap/OffHeapStoredObjectJUnitTest.java      |    5 +-
 .../OffHeapWriteObjectAsByteArrayJUnitTest.java    |    2 +-
 .../internal/offheap/TinyMemoryBlockJUnitTest.java |    5 +-
 .../offheap/TinyStoredObjectJUnitTest.java         |    2 +-
 .../internal/statistics/CallbackSamplerTest.java   |    2 +-
 .../internal/statistics/SampleCollectorTest.java   |    6 +-
 .../internal/statistics/StatSamplerTestCase.java   |    4 +-
 .../internal/statistics/TestStatisticsManager.java |    6 +-
 .../internal/statistics/TestStatisticsSampler.java |    2 +-
 .../geode/internal/tcp/ConnectionTableTest.java    |    2 +-
 .../management/bean/stats/MBeanStatsTestCase.java  |   10 +-
 .../management/bean/stats/StatsRateJUnitTest.java  |    2 +-
 ...ServerBridgeClientMembershipRegressionTest.java |    4 +-
 .../beans/stats/MBeanStatsMonitorTest.java         |    6 +-
 .../cache/query/cq/dunit/CqStatsDUnitTest.java     |   34 +-
 .../query/cq/dunit/CqStatsUsingPoolDUnitTest.java  |   34 +-
 .../tier/sockets/DurableClientSimpleDUnitTest.java |    4 +-
 .../cache/tier/sockets/DurableClientTestCase.java  |    3 +-
 .../geode/cache/client/internal/CloseCQOp.java     |    1 +
 .../geode/cache/client/internal/CreateCQOp.java    |    1 +
 .../cache/client/internal/CreateCQWithIROp.java    |    1 +
 .../cache/client/internal/GetDurableCQsOp.java     |    1 +
 .../geode/cache/client/internal/StopCQOp.java      |    1 +
 .../cache/query/internal/cq/ClientCQImpl.java      |   32 +-
 .../geode/cache/query/internal/cq/CqQueryImpl.java |   73 +-
 .../cache/query/internal/cq/CqServiceImpl.java     |  170 +-
 .../query/internal/cq/CqServiceStatisticsImpl.java |    2 +-
 ...iceVsdStats.java => CqServiceVsdStatsImpl.java} |  218 +-
 .../cache/query/internal/cq/CqStatisticsImpl.java  |   18 +-
 .../cache/query/internal/cq/ServerCQImpl.java      |   31 +-
 .../cache/tier/sockets/command/CloseCQ.java        |    7 +-
 .../cache/tier/sockets/command/ExecuteCQ.java      |   11 +-
 .../cache/tier/sockets/command/ExecuteCQ61.java    |   11 +-
 .../cache/tier/sockets/command/GetCQStats.java     |    5 +-
 .../cache/tier/sockets/command/GetDurableCQs.java  |    2 +-
 .../cache/tier/sockets/command/StopCQ.java         |    7 +-
 .../geode/test/dunit/rules/CQUnitTestRule.java     |    4 +-
 .../geode/cache30/MultiVMRegionTestCase.java       |    2 +-
 .../apache/geode/internal/cache/ha/HAHelper.java   |    1 +
 .../main/java/org/apache/geode/DeltaTestImpl.java  |    2 +-
 .../java/org/apache/geode/test/fake/Fakes.java     |   18 +-
 .../internal/LuceneIndexForPartitionedRegion.java  |    3 +-
 .../cache/lucene/internal/LuceneIndexImpl.java     |    3 +-
 .../cache/lucene/internal/LuceneIndexStats.java    |  106 +-
 .../internal/filesystem/FileSystemStats.java       |   57 +-
 .../IndexRepositoryImplPerformanceTest.java        |    9 +-
 .../lucene/internal/LuceneIndexStatsJUnitTest.java |    8 +-
 .../internal/LuceneServiceImplJUnitTest.java       |    4 +-
 .../filesystem/FileSystemStatsJUnitTest.java       |    8 +-
 geode-micrometer-stats/build.gradle.kts            |   47 +
 .../statistics/MicrometerStatSamplerStatsImpl.kt   |   90 +
 .../statistics/cache/MicrometerCCUStatsImpl.kt     |   67 +
 .../cache/MicrometerCacheClientNotifierStats.kt    |  126 +
 .../cache/MicrometerCacheClientProxyStats.kt       |  123 +
 .../statistics/cache/MicrometerCachePerfStats.kt   |  981 +++++
 .../statistics/cache/MicrometerCacheServerStats.kt |  670 ++++
 .../client/connection/MicrometerClientSendStats.kt |  616 ++++
 .../client/connection/MicrometerClientStats.kt     |  755 ++++
 .../client/connection/MicrometerConnectionStats.kt |  718 ++++
 .../client/connection/MicrometerPoolStats.kt       |  246 ++
 .../disk/MicrometerDiskDirectoryStats.kt           |   73 +
 .../statistics/disk/MicrometerDiskRegionStats.kt   |  159 +
 .../statistics/disk/MicrometerDiskStoreStats.kt    |  324 ++
 .../distributed/MicrometerDistributionStats.kt     | 1489 ++++++++
 .../geode/statistics/dlock/MicrometerDLockStats.kt |  654 ++++
 .../eviction/MicrometerCountLRUStatistics.kt       |   45 +
 .../eviction/MicrometerEvictionStatsImpl.kt        |   66 +
 .../eviction/MicrometerHeapLRUStatistics.kt        |   42 +
 .../eviction/MicrometerMemoryLRUStatistics.kt      |   44 +
 .../function/MicrometerFunctionServiceStats.kt     |   94 +
 .../statistics/function/MicrometerFunctionStats.kt |  165 +
 .../internal/micrometer/StatisticsManager.kt       |   10 +-
 .../internal/micrometer/StatisticsMeter.kt         |   37 +-
 .../internal/micrometer/StatisticsMeterGroup.kt    |    7 +-
 .../micrometer/impl/MicrometerMeterGroup.kt        |   62 +
 .../micrometer/impl/MicrometerStatisticMeter.kt    |  232 ++
 .../micrometer/impl/MicrometerStatisticsManager.kt |  147 +
 .../statistics/locator/MicrometerLocatorStats.kt   |   97 +
 .../micrometer/MicrometerStatisticsFactoryImpl.kt  |  119 +
 .../micrometer/MicrometerStatisticsImpl.kt         |  179 +
 .../micrometer/MicrometerStatisticsType.kt         |   62 +
 .../micrometer/MicrometerStatsImplementer.kt       |    9 +-
 .../offheap/MicrometerOffHeapStorageStats.kt       |  150 +
 .../statistics/query/MicrometerCqQueryVsdStats.kt  |  118 +
 .../query/MicrometerCqServiceVsdStats.kt           |  137 +
 .../region/MicrometerHARegionQueueStats.kt         |  140 +
 .../region/MicrometerPartitionedRegionStats.kt     |  768 ++++
 .../statistics/region/MicrometerRegionPerfStats.kt |  357 ++
 .../MicrometerResourceManagerStats.kt              |  420 +++
 .../geode/statistics/util/SimpleTimeUtils.kt       |    8 +-
 .../wan/MicrometerAsyncEventQueueStats.kt          |   28 +-
 .../wan/MicrometerGatewayReceiverStats.kt          |  121 +
 .../statistics/wan/MicrometerGatewaySenderStats.kt |  382 ++
 ...geode.stats.common.statistics.StatisticsFactory |    1 +
 geode-old-versions/build.gradle                    |   16 +-
 .../v1/acceptance/LocatorConnectionDUnitTest.java  |    8 +-
 .../protobuf/v1/AuthenticationIntegrationTest.java |    7 +-
 .../v1/acceptance/CacheConnectionJUnitTest.java    |    7 +-
 .../statistics/ProtobufClientStatistics.java       |   10 +-
 .../protobuf/v1/ProtobufProtocolService.java       |    2 +-
 .../OutputCapturingServerConnectionTest.java       |    2 +-
 .../tier/sockets/ProtobufServerConnectionTest.java |    2 +-
 .../v1/ProtobufProtocolServiceJUnitTest.java       |   37 -
 .../util/AutoBalancerIntegrationJUnitTest.java     |    3 +-
 .../build.gradle                                   |   17 +-
 .../cache/execute/FunctionServiceStats.java        |   64 +
 .../geode/internal/statistics/StatSampleStats.java |   23 +
 .../internal/statistics/StatisticsManager.java     |    4 +-
 .../apache/geode/internal/statistics/VMStats.java  |    6 +-
 .../java/org/apache/geode/stats/common/Stats.java  |    6 +-
 .../cache/client/internal/ConnectionStats.java     |  285 ++
 .../common}/cache/query/CqServiceStatistics.java   |    6 +-
 .../stats/common}/cache/query/CqStatistics.java    |   14 +-
 .../stats/common}/cache/query/QueryStatistics.java |    6 +-
 .../cache/query/internal/CqQueryVsdStats.java      |   43 +-
 .../cache/query/internal/cq/CqServiceVsdStats.java |   49 +-
 .../common}/distributed/internal/DMStats.java      |    8 +-
 .../distributed/internal/DistributionStats.java    |  495 +++
 .../common/distributed/internal/LocatorStats.java  |   32 +-
 .../distributed/internal/PoolStatHelper.java       |    2 +-
 .../distributed/internal/QueueStatHelper.java      |    2 +-
 .../internal/ThrottledMemQueueStatHelper.java      |    2 +-
 .../internal/ThrottledQueueStatHelper.java         |    2 +-
 .../distributed/internal/locks/DLockStats.java     |   14 +-
 .../common/internal/cache/CachePerfStats.java      |  369 ++
 .../common/internal/cache/DiskDirectoryStats.java  |   25 +-
 .../common/internal/cache/DiskRegionStats.java     |   68 +
 .../common/internal/cache/DiskStoreStats.java      |  114 +
 .../internal/cache/PartitionedRegionStats.java     |  239 ++
 .../stats/common/internal/cache/PoolStats.java     |   96 +
 .../common/internal/cache/RegionPerfStats.java     |    6 +-
 .../cache/control/ResourceManagerStats.java        |  157 +
 .../cache/eviction/CountLRUEvictionStats.java      |    6 +-
 .../internal/cache/eviction/EvictionStats.java     |    7 +-
 .../cache/eviction/HeapLRUEvictionStats.java       |    6 +-
 .../cache/eviction/MemoryLRUEvictionStats.java     |    6 +-
 .../internal/cache/execute/FunctionStats.java      |   62 +
 .../internal/cache/ha/HARegionQueueStats.java      |   72 +
 .../internal/cache/tier/sockets/CCUStats.java      |    7 +-
 .../tier/sockets/CacheClientNotifierStats.java     |   63 +
 .../cache/tier/sockets/CacheServerStats.java       |  198 +
 .../internal/cache/tier/sockets/MessageStats.java  |    6 +-
 .../internal/cache/wan/GatewayReceiverStats.java   |   59 +
 .../internal/cache/wan/GatewaySenderStats.java     |  157 +
 .../internal/offheap/OffHeapStorageStats.java      |   15 +-
 .../common/statistics/GFSStatsImplementer.java     |   33 +-
 .../common/statistics}/StatisticDescriptor.java    |    3 +-
 .../geode/stats/common/statistics}/Statistics.java |    2 +-
 .../common/statistics}/StatisticsFactory.java      |   90 +-
 .../stats/common/statistics}/StatisticsType.java   |    7 +-
 .../common/statistics}/StatisticsTypeFactory.java  |   92 +-
 .../stats/common/statistics/StatsImplementer.java  |   11 +-
 .../common/statistics/factory/StatsFactory.java    |  219 ++
 geode-wan/build.gradle                             |    1 +
 ...ueueOverflowMBeanAttributesDistributedTest.java |    9 +-
 ...nderOverflowMBeanAttributesDistributedTest.java |    9 +-
 .../geode/internal/cache/wan/WANTestBase.java      |    4 +-
 .../KeepEventsOnGatewaySenderQueueDUnitTest.java   |    4 +-
 .../client/internal/GatewaySenderBatchOp.java      |    1 +
 .../wan/GatewaySenderEventRemoteDispatcher.java    |    3 +-
 ...currentParallelGatewaySenderEventProcessor.java |    2 +-
 .../RemoteParallelGatewaySenderEventProcessor.java |    2 +-
 gradle.properties                                  |    3 +-
 settings.gradle                                    |    6 +-
 708 files changed, 27949 insertions(+), 12787 deletions(-)

diff --git a/build.gradle b/build.gradle
index 8b0085a..774aa20 100755
--- a/build.gradle
+++ b/build.gradle
@@ -17,10 +17,14 @@
 
 buildscript {
   repositories {
+    mavenCentral()
+    maven { url "file://home/ukohlmeyer/.m2/repository"}
     maven { url "https://plugins.gradle.org/m2/" }
     maven { url "https://dl.bintray.com/palantir/releases" }
     jcenter()
     maven { url "http://geode-maven.s3-website-us-west-2.amazonaws.com" }
+    mavenLocal()
+
   }
 
   dependencies {
@@ -58,6 +62,7 @@ allprojects {
   repositories {
     mavenCentral()
     maven { url "http://repo.spring.io/release" }
+    maven { url 'file://home/ukohlmeyer/.m2/repository' }
   }
 
   group = "org.apache.geode"
diff --git a/extensions/geode-modules-session-internal/src/main/java/org/apache/geode/modules/session/internal/common/AbstractSessionCache.java b/extensions/geode-modules-session-internal/src/main/java/org/apache/geode/modules/session/internal/common/AbstractSessionCache.java
index 7d65961..afc73da 100644
--- a/extensions/geode-modules-session-internal/src/main/java/org/apache/geode/modules/session/internal/common/AbstractSessionCache.java
+++ b/extensions/geode-modules-session-internal/src/main/java/org/apache/geode/modules/session/internal/common/AbstractSessionCache.java
@@ -70,8 +70,9 @@ public abstract class AbstractSessionCache implements SessionCache {
   }
 
   protected void createStatistics() {
-    this.statistics = new DeltaSessionStatistics(getCache().getDistributedSystem(),
-        (String) properties.get(CacheProperty.STATISTICS_NAME));
+    this.statistics =
+        new DeltaSessionStatistics(getCache().getDistributedSystem().getStatisticsFactory(),
+            (String) properties.get(CacheProperty.STATISTICS_NAME));
   }
 
   /**
diff --git a/extensions/geode-modules/src/main/java/org/apache/geode/modules/session/catalina/AbstractSessionCache.java b/extensions/geode-modules/src/main/java/org/apache/geode/modules/session/catalina/AbstractSessionCache.java
index 1764190..10717bb 100644
--- a/extensions/geode-modules/src/main/java/org/apache/geode/modules/session/catalina/AbstractSessionCache.java
+++ b/extensions/geode-modules/src/main/java/org/apache/geode/modules/session/catalina/AbstractSessionCache.java
@@ -94,8 +94,9 @@ public abstract class AbstractSessionCache implements SessionCache {
   }
 
   protected void createStatistics() {
-    this.statistics = new DeltaSessionStatistics(getCache().getDistributedSystem(),
-        getSessionManager().getStatisticsName());
+    this.statistics =
+        new DeltaSessionStatistics(getCache().getDistributedSystem().getStatisticsFactory(),
+            getSessionManager().getStatisticsName());
   }
 
   protected RegionConfiguration createRegionConfiguration() {
diff --git a/extensions/geode-modules/src/main/java/org/apache/geode/modules/session/catalina/internal/DeltaSessionStatistics.java b/extensions/geode-modules/src/main/java/org/apache/geode/modules/session/catalina/internal/DeltaSessionStatistics.java
index 57c14b4..1514285 100644
--- a/extensions/geode-modules/src/main/java/org/apache/geode/modules/session/catalina/internal/DeltaSessionStatistics.java
+++ b/extensions/geode-modules/src/main/java/org/apache/geode/modules/session/catalina/internal/DeltaSessionStatistics.java
@@ -14,36 +14,35 @@
  */
 package org.apache.geode.modules.session.catalina.internal;
 
-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.internal.statistics.StatisticsTypeFactoryImpl;
+import org.apache.geode.stats.common.statistics.StatisticDescriptor;
+import org.apache.geode.stats.common.statistics.Statistics;
+import org.apache.geode.stats.common.statistics.StatisticsFactory;
+import org.apache.geode.stats.common.statistics.StatisticsType;
 
 public class DeltaSessionStatistics {
 
   public static final String typeName = "SessionStatistics";
 
-  private static final StatisticsType type;
+  private StatisticsType type;
 
   private static final String SESSIONS_CREATED = "sessionsCreated";
   private static final String SESSIONS_INVALIDATED = "sessionsInvalidated";
   private static final String SESSIONS_EXPIRED = "sessionsExpired";
 
-  private static final int sessionsCreatedId;
-  private static final int sessionsInvalidatedId;
-  private static final int sessionsExpiredId;
+  private int sessionsCreatedId;
+  private int sessionsInvalidatedId;
+  private int sessionsExpiredId;
 
-  static {
+  private void initializeStats(StatisticsFactory factory) {
     // Initialize type
-    StatisticsTypeFactory f = StatisticsTypeFactoryImpl.singleton();
-    type = f.createType(typeName, typeName,
+    type = factory.createType(typeName, typeName,
         new StatisticDescriptor[] {
-            f.createIntCounter(SESSIONS_CREATED, "The number of sessions created", "operations"),
-            f.createIntCounter(SESSIONS_INVALIDATED,
+            factory.createIntCounter(SESSIONS_CREATED, "The number of sessions created",
+                "operations"),
+            factory.createIntCounter(SESSIONS_INVALIDATED,
                 "The number of sessions invalidated by invoking invalidate", "operations"),
-            f.createIntCounter(SESSIONS_EXPIRED, "The number of sessions invalidated by timeout",
+            factory.createIntCounter(SESSIONS_EXPIRED,
+                "The number of sessions invalidated by timeout",
                 "operations"),});
 
     // Initialize id fields
@@ -55,6 +54,7 @@ public class DeltaSessionStatistics {
   private final Statistics stats;
 
   public DeltaSessionStatistics(StatisticsFactory factory, String applicationName) {
+    initializeStats(factory);
     this.stats = factory.createAtomicStatistics(type, typeName + "_" + applicationName);
   }
 
diff --git a/extensions/geode-modules/src/main/java/org/apache/geode/modules/util/ModuleStatistics.java b/extensions/geode-modules/src/main/java/org/apache/geode/modules/util/ModuleStatistics.java
index 6f276c4..34d8f9c 100644
--- a/extensions/geode-modules/src/main/java/org/apache/geode/modules/util/ModuleStatistics.java
+++ b/extensions/geode-modules/src/main/java/org/apache/geode/modules/util/ModuleStatistics.java
@@ -14,13 +14,11 @@
  */
 package org.apache.geode.modules.util;
 
-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.distributed.DistributedSystem;
-import org.apache.geode.internal.statistics.StatisticsTypeFactoryImpl;
+import org.apache.geode.stats.common.statistics.StatisticDescriptor;
+import org.apache.geode.stats.common.statistics.Statistics;
+import org.apache.geode.stats.common.statistics.StatisticsFactory;
+import org.apache.geode.stats.common.statistics.StatisticsType;
 
 /**
  * Statistics for modules.
@@ -28,25 +26,26 @@ import org.apache.geode.internal.statistics.StatisticsTypeFactoryImpl;
  */
 public class ModuleStatistics {
 
-  private static final StatisticsType type;
+  private StatisticsType type;
 
-  private static final int cacheHitsId;
+  private int cacheHitsId;
 
-  private static final int cacheMissesId;
+  private int cacheMissesId;
 
-  private static final int hibernateEntityDestroyJobsScheduledId;
+  private int hibernateEntityDestroyJobsScheduledId;
 
-  static {
-    StatisticsTypeFactory f = StatisticsTypeFactoryImpl.singleton();
-    type = f.createType("pluginStats", "statistics for hibernate plugin and hibernate L2 cache",
-        new StatisticDescriptor[] {
-            f.createLongCounter("cacheHits", "number of times an entity was found in L2 cache",
-                "count"),
-            f.createLongCounter("cacheMisses",
-                "number of times an entity was NOT found in l2 cache", "count"),
-            f.createLongCounter("hibernateEntityDestroyJobsScheduled",
-                "number of entities scheduled for destroy because of version conflict with a remote member",
-                "jobs")});
+  private void initializeStats(StatisticsFactory factory) {
+    type =
+        factory.createType("pluginStats", "statistics for hibernate plugin and hibernate L2 cache",
+            new StatisticDescriptor[] {
+                factory.createLongCounter("cacheHits",
+                    "number of times an entity was found in L2 cache",
+                    "count"),
+                factory.createLongCounter("cacheMisses",
+                    "number of times an entity was NOT found in l2 cache", "count"),
+                factory.createLongCounter("hibernateEntityDestroyJobsScheduled",
+                    "number of entities scheduled for destroy because of version conflict with a remote member",
+                    "jobs")});
 
     cacheHitsId = type.nameToId("cacheHits");
     cacheMissesId = type.nameToId("cacheMisses");
@@ -58,13 +57,14 @@ public class ModuleStatistics {
   private static ModuleStatistics instance;
 
   private ModuleStatistics(StatisticsFactory factory) {
+    initializeStats(factory);
     this.stats = factory.createAtomicStatistics(type, "PluginStatistics");
   }
 
   public static ModuleStatistics getInstance(DistributedSystem system) {
     synchronized (ModuleStatistics.class) {
       if (instance == null) {
-        instance = new ModuleStatistics(system);
+        instance = new ModuleStatistics(system.getStatisticsFactory());
       }
     }
     return instance;
diff --git a/geode-assembly/build.gradle b/geode-assembly/build.gradle
index cbef1c0..075467b 100755
--- a/geode-assembly/build.gradle
+++ b/geode-assembly/build.gradle
@@ -89,6 +89,7 @@ configurations {
 
 dependencies {
   archives project(':geode-common')
+  archives project(':geode-stats-common')
   archives project(':geode-json')
   archives project(':geode-core')
   archives project(':geode-connectors')
@@ -250,6 +251,8 @@ def cp = {
         it.contains('lucene-queryparser') ||
         it.contains('lucene-analyzers-phonetic') ||
 
+        it.contains('micrometer-stats') ||
+
         // dependencies from geode-protobuf
         it.contains('protobuf-java') ||
 
@@ -389,6 +392,10 @@ distributions {
       into ('lib') {
         from project(":geode-common").configurations.runtimeClasspath
         from project(":geode-common").configurations.archives.allArtifacts.files
+
+        from project(":geode-stats-common").configurations.runtime
+        from project(":geode-stats-common").configurations.archives.allArtifacts.files
+
         from project(":geode-json").configurations.runtimeClasspath
         from project(":geode-json").configurations.archives.allArtifacts.files
         from project(":geode-wan").configurations.runtimeClasspath
@@ -410,6 +417,9 @@ distributions {
         from project(":geode-rebalancer").configurations.runtimeClasspath
         from project(":geode-rebalancer").configurations.archives.allArtifacts.files
 
+        from project(":geode-micrometer-stats").configurations.runtime
+        from project(":geode-micrometer-stats").configurations.archives.allArtifacts.files
+
         from configurations.bundled
         from configurations.gfshDependencies
 
diff --git a/geode-connectors/build.gradle b/geode-connectors/build.gradle
index 0880809..a0d5d6f 100644
--- a/geode-connectors/build.gradle
+++ b/geode-connectors/build.gradle
@@ -18,10 +18,11 @@
 evaluationDependsOn(":geode-core")
 
 repositories {
-  maven {
-    // docker-compose-rule is published on bintray
-    url 'https://dl.bintray.com/palantir/releases'
-  }
+    maven {
+        url 'https://dl.bintray.com/palantir/releases'
+        // docker-compose-rule is published on bintray
+    }
+    mavenLocal()
 }
 
 dependencies {
diff --git a/geode-core/build.gradle b/geode-core/build.gradle
index 1375d4a..f41a13d 100755
--- a/geode-core/build.gradle
+++ b/geode-core/build.gradle
@@ -18,12 +18,15 @@
 
 apply plugin: 'antlr'
 apply plugin: 'me.champeau.gradle.jmh'
+apply plugin: 'kotlin'
 
 sourceSets {
   jca {
     compileClasspath += configurations.compileClasspath
     runtimeClasspath += configurations.runtimeClasspath
   }
+  main.kotlin.srcDirs += 'src/main/kotlin'
+  main.kotlin.srcDirs += 'src/main/java'
 }
 
 idea {
@@ -34,6 +37,27 @@ idea {
     testSourceDirs += project.tasks.generateUpgradeTestGrammarSource.outputs.files
   }
 }
+sourceCompatibility = 1.8
+targetCompatibility = sourceCompatibility
+compileKotlin { kotlinOptions.jvmTarget = sourceCompatibility }
+compileTestKotlin { kotlinOptions.jvmTarget = sourceCompatibility }
+
+buildscript {
+  ext.kotlin_version = '1.2.61'
+
+  repositories {
+    mavenCentral()
+    mavenLocal()
+    maven {
+      url "file://home/ukohlmeyer/.m2/repository"
+    }
+
+  }
+
+  dependencies {
+    classpath "org.jetbrains.kotlin:kotlin-gradle-plugin:$kotlin_version"
+  }
+}
 
 configurations {
   //declaring new configuration that will be used to associate with artifacts
@@ -45,6 +69,10 @@ dependencies {
   antlr 'antlr:antlr:' + project.'antlr.version'
 
   // External
+  implementation "org.jetbrains.kotlin:kotlin-stdlib-jdk8"
+  implementation project(':geode-micrometer-stats')
+  compile project(':geode-stats-common')
+
   compileOnly files("${System.getProperty('java.home')}/../lib/tools.jar")
   compile 'com.github.stephenc.findbugs:findbugs-annotations:' + project.'stephenc-findbugs.version'
   compile 'org.jgroups:jgroups:' + project.'jgroups.version'
diff --git a/geode-core/src/distributedTest/java/org/apache/geode/cache/ConnectionPoolDUnitTest.java b/geode-core/src/distributedTest/java/org/apache/geode/cache/ConnectionPoolDUnitTest.java
index b2e80c7..3a909e8 100755
--- a/geode-core/src/distributedTest/java/org/apache/geode/cache/ConnectionPoolDUnitTest.java
+++ b/geode-core/src/distributedTest/java/org/apache/geode/cache/ConnectionPoolDUnitTest.java
@@ -60,12 +60,12 @@ import org.apache.geode.internal.Assert;
 import org.apache.geode.internal.cache.CacheServerImpl;
 import org.apache.geode.internal.cache.EntryExpiryTask;
 import org.apache.geode.internal.cache.LocalRegion;
-import org.apache.geode.internal.cache.PoolStats;
 import org.apache.geode.internal.cache.tier.sockets.CacheClientNotifier;
-import org.apache.geode.internal.cache.tier.sockets.CacheClientNotifierStats;
+import org.apache.geode.internal.cache.tier.sockets.CacheClientNotifierStatsImpl;
 import org.apache.geode.internal.cache.tier.sockets.ClientProxyMembershipID;
 import org.apache.geode.internal.logging.InternalLogWriter;
 import org.apache.geode.internal.logging.LocalLogWriter;
+import org.apache.geode.stats.common.internal.cache.PoolStats;
 import org.apache.geode.test.dunit.AsyncInvocation;
 import org.apache.geode.test.dunit.Host;
 import org.apache.geode.test.dunit.Invoke;
@@ -868,7 +868,7 @@ public class ConnectionPoolDUnitTest extends JUnit4CacheTestCase {
             LocalRegion region = (LocalRegion) getRootRegion().getSubregion(name);
             PoolStats stats = ((PoolImpl) PoolManager.find(poolName)).getStats();
             int oldConnects = stats.getConnects();
-            int oldDisConnects = stats.getDisConnects();
+            int oldDisConnects = stats.getDisconnects();
             try {
               for (int i = 0; i < numberOfKeys; i++) {
                 String actual = (String) region.get("key-" + i);
@@ -881,7 +881,7 @@ public class ConnectionPoolDUnitTest extends JUnit4CacheTestCase {
               }
             }
             int newConnects = stats.getConnects();
-            int newDisConnects = stats.getDisConnects();
+            int newDisConnects = stats.getDisconnects();
             // System.out.println("#### new connects/disconnects :" + newConnects + ":" +
             // newDisConnects);
             if (newConnects != oldConnects && newDisConnects != oldDisConnects) {
@@ -3505,7 +3505,7 @@ public class ConnectionPoolDUnitTest extends JUnit4CacheTestCase {
           public void run2() throws CacheException {
             for (Iterator bi = getCache().getCacheServers().iterator(); bi.hasNext();) {
               CacheServerImpl bsi = (CacheServerImpl) bi.next();
-              final CacheClientNotifierStats ccnStats =
+              final CacheClientNotifierStatsImpl ccnStats =
                   bsi.getAcceptor().getCacheClientNotifier().getStats();
               WaitCriterion ev = new WaitCriterion() {
                 public boolean done() {
@@ -4434,7 +4434,7 @@ public class ConnectionPoolDUnitTest extends JUnit4CacheTestCase {
       srv1.invoke(new CacheSerializableRunnable("Validate Server1 update") {
         public void run2() throws CacheException {
           CacheClientNotifier ccn = CacheClientNotifier.getInstance();
-          final CacheClientNotifierStats ccnStats = ccn.getStats();
+          final CacheClientNotifierStatsImpl ccnStats = ccn.getStats();
           final int eventCount = ccnStats.getEvents();
           Region r = getRootRegion(name);
           assertNotNull(r);
diff --git a/geode-core/src/distributedTest/java/org/apache/geode/cache/management/MemoryThresholdsDUnitTest.java b/geode-core/src/distributedTest/java/org/apache/geode/cache/management/MemoryThresholdsDUnitTest.java
index 79c522e..0309e0f 100644
--- a/geode-core/src/distributedTest/java/org/apache/geode/cache/management/MemoryThresholdsDUnitTest.java
+++ b/geode-core/src/distributedTest/java/org/apache/geode/cache/management/MemoryThresholdsDUnitTest.java
@@ -1745,7 +1745,8 @@ public class MemoryThresholdsDUnitTest extends ClientServerTestCase {
     Cache cache = getCache();
     InternalDistributedSystem internalSystem =
         (InternalDistributedSystem) cache.getDistributedSystem();
-    final GemFireStatSampler sampler = internalSystem.getStatSampler();
+    final GemFireStatSampler sampler =
+        internalSystem.getInternalDistributedSystemStats().getStatSampler();
     sampler.waitForInitialization(10000); // fix: remove infinite wait
     final LocalStatListener l = new LocalStatListener() {
       public void statValueChanged(double value) {
@@ -1754,7 +1755,7 @@ public class MemoryThresholdsDUnitTest extends ClientServerTestCase {
     };
     final String tenuredPoolName = HeapMemoryMonitor.getTenuredMemoryPoolMXBean().getName();
     LogWriterUtils.getLogWriter().info("TenuredPoolName:" + tenuredPoolName);
-    final List list = internalSystem.getStatsList();
+    final List list = internalSystem.getInternalDistributedSystemStats().getStatsList();
     assertFalse(list.isEmpty());
 
     // fix: found race condition here...
diff --git a/geode-core/src/distributedTest/java/org/apache/geode/cache30/CacheRegionsReliablityStatsCheckDUnitTest.java b/geode-core/src/distributedTest/java/org/apache/geode/cache30/CacheRegionsReliablityStatsCheckDUnitTest.java
index a32f58b..4303c2a 100644
--- a/geode-core/src/distributedTest/java/org/apache/geode/cache30/CacheRegionsReliablityStatsCheckDUnitTest.java
+++ b/geode-core/src/distributedTest/java/org/apache/geode/cache30/CacheRegionsReliablityStatsCheckDUnitTest.java
@@ -31,8 +31,8 @@ import org.apache.geode.cache.MembershipAttributes;
 import org.apache.geode.cache.RegionAttributes;
 import org.apache.geode.cache.ResumptionAction;
 import org.apache.geode.cache.Scope;
-import org.apache.geode.internal.cache.CachePerfStats;
 import org.apache.geode.internal.cache.GemFireCacheImpl;
+import org.apache.geode.stats.common.internal.cache.CachePerfStats;
 import org.apache.geode.test.dunit.Host;
 import org.apache.geode.test.dunit.LogWriterUtils;
 import org.apache.geode.test.dunit.SerializableRunnable;
diff --git a/geode-core/src/distributedTest/java/org/apache/geode/cache30/ClientMembershipDUnitTest.java b/geode-core/src/distributedTest/java/org/apache/geode/cache30/ClientMembershipDUnitTest.java
index 43ca8a1..ed7ce6a 100644
--- a/geode-core/src/distributedTest/java/org/apache/geode/cache30/ClientMembershipDUnitTest.java
+++ b/geode-core/src/distributedTest/java/org/apache/geode/cache30/ClientMembershipDUnitTest.java
@@ -44,8 +44,6 @@ import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
 import org.apache.geode.InternalGemFireException;
-import org.apache.geode.Statistics;
-import org.apache.geode.StatisticsType;
 import org.apache.geode.cache.AttributesFactory;
 import org.apache.geode.cache.CacheException;
 import org.apache.geode.cache.Region;
@@ -64,6 +62,8 @@ import org.apache.geode.internal.cache.tier.sockets.ServerConnection;
 import org.apache.geode.management.membership.ClientMembership;
 import org.apache.geode.management.membership.ClientMembershipEvent;
 import org.apache.geode.management.membership.ClientMembershipListener;
+import org.apache.geode.stats.common.statistics.Statistics;
+import org.apache.geode.stats.common.statistics.StatisticsType;
 import org.apache.geode.test.dunit.Assert;
 import org.apache.geode.test.dunit.Host;
 import org.apache.geode.test.dunit.IgnoredException;
@@ -117,8 +117,8 @@ public class ClientMembershipDUnitTest extends ClientServerTestCase {
 
   protected int getAcceptsInProgress() {
     DistributedSystem distributedSystem = getCache().getDistributedSystem();
-    StatisticsType st = distributedSystem.findType("CacheServerStats");
-    Statistics[] s = distributedSystem.findStatisticsByType(st);
+    StatisticsType st = distributedSystem.getStatisticsFactory().findType("CacheServerStats");
+    Statistics[] s = distributedSystem.getStatisticsFactory().findStatisticsByType(st);
     return s[0].getInt("acceptsInProgress");
   }
 
diff --git a/geode-core/src/distributedTest/java/org/apache/geode/cache30/ClientServerCCEDUnitTest.java b/geode-core/src/distributedTest/java/org/apache/geode/cache30/ClientServerCCEDUnitTest.java
index 08a7edb..4abbd66 100644
--- a/geode-core/src/distributedTest/java/org/apache/geode/cache30/ClientServerCCEDUnitTest.java
+++ b/geode-core/src/distributedTest/java/org/apache/geode/cache30/ClientServerCCEDUnitTest.java
@@ -59,9 +59,7 @@ import org.apache.geode.distributed.internal.ClusterDistributionManager;
 import org.apache.geode.distributed.internal.DistributionMessage;
 import org.apache.geode.distributed.internal.DistributionMessageObserver;
 import org.apache.geode.distributed.internal.locks.DLockService;
-import org.apache.geode.distributed.internal.locks.DistributedLockStats;
 import org.apache.geode.internal.AvailablePortHelper;
-import org.apache.geode.internal.cache.CachePerfStats;
 import org.apache.geode.internal.cache.DistributedRegion;
 import org.apache.geode.internal.cache.DistributedTombstoneOperation.TombstoneMessage;
 import org.apache.geode.internal.cache.EntryEventImpl;
@@ -76,6 +74,8 @@ import org.apache.geode.internal.cache.partitioned.PRTombstoneMessage;
 import org.apache.geode.internal.cache.tier.sockets.CacheClientNotifier;
 import org.apache.geode.internal.cache.tier.sockets.CacheClientProxy;
 import org.apache.geode.internal.i18n.LocalizedStrings;
+import org.apache.geode.stats.common.distributed.internal.locks.DLockStats;
+import org.apache.geode.stats.common.internal.cache.CachePerfStats;
 import org.apache.geode.test.dunit.Assert;
 import org.apache.geode.test.dunit.AsyncInvocation;
 import org.apache.geode.test.dunit.Host;
@@ -169,7 +169,7 @@ public class ClientServerCCEDUnitTest extends JUnit4CacheTestCase {
     assertEquals(0, stats.getConflatedEventsCount());
 
     DLockService dLockService = (DLockService) region.getLockService();
-    DistributedLockStats distributedLockStats = dLockService.getStats();
+    DLockStats distributedLockStats = dLockService.getStats();
     assertEquals(numIterations, distributedLockStats.getLockReleasesCompleted());
   }
 
diff --git a/geode-core/src/distributedTest/java/org/apache/geode/cache30/DistributedMulticastRegionDUnitTest.java b/geode-core/src/distributedTest/java/org/apache/geode/cache30/DistributedMulticastRegionDUnitTest.java
index c1ffdc1..2364ecb 100755
--- a/geode-core/src/distributedTest/java/org/apache/geode/cache30/DistributedMulticastRegionDUnitTest.java
+++ b/geode-core/src/distributedTest/java/org/apache/geode/cache30/DistributedMulticastRegionDUnitTest.java
@@ -238,7 +238,7 @@ public class DistributedMulticastRegionDUnitTest extends JUnit4CacheTestCase {
 
   protected void validateUDPEncryptionStats() {
     long encrptTime =
-        getGemfireCache().getDistributionManager().getStats().getUDPMsgEncryptionTiime();
+        getGemfireCache().getDistributionManager().getStats().getUDPMsgEncryptionTime();
     long decryptTime =
         getGemfireCache().getDistributionManager().getStats().getUDPMsgDecryptionTime();
     assertTrue("Should have multicast writes or reads. encrptTime=  " + encrptTime
diff --git a/geode-core/src/distributedTest/java/org/apache/geode/cache30/DistributedMulticastRegionWithUDPSecurityDUnitTest.java b/geode-core/src/distributedTest/java/org/apache/geode/cache30/DistributedMulticastRegionWithUDPSecurityDUnitTest.java
index 6f0df4b..32432f3 100644
--- a/geode-core/src/distributedTest/java/org/apache/geode/cache30/DistributedMulticastRegionWithUDPSecurityDUnitTest.java
+++ b/geode-core/src/distributedTest/java/org/apache/geode/cache30/DistributedMulticastRegionWithUDPSecurityDUnitTest.java
@@ -29,7 +29,7 @@ public class DistributedMulticastRegionWithUDPSecurityDUnitTest
   @Override
   protected void validateUDPEncryptionStats() {
     long encrptTime =
-        getGemfireCache().getDistributionManager().getStats().getUDPMsgEncryptionTiime();
+        getGemfireCache().getDistributionManager().getStats().getUDPMsgEncryptionTime();
     long decryptTime =
         getGemfireCache().getDistributionManager().getStats().getUDPMsgDecryptionTime();
     assertTrue("Should have multicast writes or reads. encrptTime=  " + encrptTime
diff --git a/geode-core/src/distributedTest/java/org/apache/geode/cache30/ProxyDUnitTest.java b/geode-core/src/distributedTest/java/org/apache/geode/cache30/ProxyDUnitTest.java
index 192f433..8d12020 100644
--- a/geode-core/src/distributedTest/java/org/apache/geode/cache30/ProxyDUnitTest.java
+++ b/geode-core/src/distributedTest/java/org/apache/geode/cache30/ProxyDUnitTest.java
@@ -43,7 +43,7 @@ import org.apache.geode.cache.Scope;
 import org.apache.geode.cache.SubscriptionAttributes;
 import org.apache.geode.cache.util.CacheWriterAdapter;
 import org.apache.geode.distributed.DistributedMember;
-import org.apache.geode.distributed.internal.DMStats;
+import org.apache.geode.stats.common.distributed.internal.DMStats;
 import org.apache.geode.test.dunit.Host;
 import org.apache.geode.test.dunit.VM;
 import org.apache.geode.test.dunit.cache.internal.JUnit4CacheTestCase;
diff --git a/geode-core/src/distributedTest/java/org/apache/geode/cache30/QueueMsgDUnitTest.java b/geode-core/src/distributedTest/java/org/apache/geode/cache30/QueueMsgDUnitTest.java
index da5442a..cb4e1c0 100644
--- a/geode-core/src/distributedTest/java/org/apache/geode/cache30/QueueMsgDUnitTest.java
+++ b/geode-core/src/distributedTest/java/org/apache/geode/cache30/QueueMsgDUnitTest.java
@@ -37,8 +37,8 @@ import org.apache.geode.cache.Region;
 import org.apache.geode.cache.RegionEvent;
 import org.apache.geode.cache.Scope;
 import org.apache.geode.cache.SubscriptionAttributes;
-import org.apache.geode.internal.cache.CachePerfStats;
 import org.apache.geode.internal.cache.DistributedRegion;
+import org.apache.geode.stats.common.internal.cache.CachePerfStats;
 import org.apache.geode.test.dunit.Host;
 import org.apache.geode.test.dunit.SerializableRunnable;
 import org.apache.geode.test.dunit.VM;
diff --git a/geode-core/src/distributedTest/java/org/apache/geode/cache30/SlowRecDUnitTest.java b/geode-core/src/distributedTest/java/org/apache/geode/cache30/SlowRecDUnitTest.java
index 14bd2fd..3f5e852 100644
--- a/geode-core/src/distributedTest/java/org/apache/geode/cache30/SlowRecDUnitTest.java
+++ b/geode-core/src/distributedTest/java/org/apache/geode/cache30/SlowRecDUnitTest.java
@@ -43,9 +43,9 @@ import org.apache.geode.cache.Region.Entry;
 import org.apache.geode.cache.RegionEvent;
 import org.apache.geode.cache.Scope;
 import org.apache.geode.cache.util.CacheListenerAdapter;
-import org.apache.geode.distributed.internal.DMStats;
 import org.apache.geode.distributed.internal.DistributionManager;
 import org.apache.geode.internal.tcp.Connection;
+import org.apache.geode.stats.common.distributed.internal.DMStats;
 import org.apache.geode.test.dunit.Host;
 import org.apache.geode.test.dunit.LogWriterUtils;
 import org.apache.geode.test.dunit.SerializableRunnable;
diff --git a/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/DeltaPropagationStatsDUnitTest.java b/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/DeltaPropagationStatsDUnitTest.java
index 3b64344..6a44c7e 100755
--- a/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/DeltaPropagationStatsDUnitTest.java
+++ b/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/DeltaPropagationStatsDUnitTest.java
@@ -41,11 +41,12 @@ import org.apache.geode.cache.client.PoolManager;
 import org.apache.geode.cache.server.CacheServer;
 import org.apache.geode.cache.util.CacheListenerAdapter;
 import org.apache.geode.distributed.DistributedSystem;
-import org.apache.geode.distributed.internal.DistributionStats;
+import org.apache.geode.distributed.internal.DistributionStatsImpl;
 import org.apache.geode.internal.AvailablePort;
 import org.apache.geode.internal.cache.tier.sockets.CacheClientNotifier;
 import org.apache.geode.internal.cache.tier.sockets.CacheClientProxy;
 import org.apache.geode.internal.tcp.ConnectionTable;
+import org.apache.geode.stats.common.internal.cache.CachePerfStats;
 import org.apache.geode.test.dunit.Host;
 import org.apache.geode.test.dunit.NetworkUtils;
 import org.apache.geode.test.dunit.VM;
@@ -453,7 +454,7 @@ public class DeltaPropagationStatsDUnitTest extends JUnit4DistributedTestCase {
       deltaTime = 1; // dummy assignment
     } else if (path == CLIENT_TO_SERVER) {
       numOfDeltasSent = region.getCachePerfStats().getDeltasSent();
-      if (DistributionStats.enableClockStats) {
+      if (DistributionStatsImpl.enableClockStats) {
         deltaTime = region.getCachePerfStats().getDeltasPreparedTime();
       } else {
         deltaTime = 1; // dummy assignment
diff --git a/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/GIIDeltaDUnitTest.java b/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/GIIDeltaDUnitTest.java
index 0174fba..4aa101c 100644
--- a/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/GIIDeltaDUnitTest.java
+++ b/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/GIIDeltaDUnitTest.java
@@ -54,6 +54,7 @@ import org.apache.geode.internal.cache.entries.DiskEntry;
 import org.apache.geode.internal.cache.persistence.DiskStoreID;
 import org.apache.geode.internal.cache.versions.RegionVersionVector;
 import org.apache.geode.internal.cache.versions.VersionTag;
+import org.apache.geode.stats.common.internal.cache.CachePerfStats;
 import org.apache.geode.test.dunit.Assert;
 import org.apache.geode.test.dunit.AsyncInvocation;
 import org.apache.geode.test.dunit.Host;
diff --git a/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/GIIFlowControlDUnitTest.java b/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/GIIFlowControlDUnitTest.java
index b3e1e81..b631a80 100644
--- a/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/GIIFlowControlDUnitTest.java
+++ b/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/GIIFlowControlDUnitTest.java
@@ -29,10 +29,10 @@ import org.apache.geode.cache.Region;
 import org.apache.geode.cache.RegionFactory;
 import org.apache.geode.cache.Scope;
 import org.apache.geode.distributed.internal.ClusterDistributionManager;
-import org.apache.geode.distributed.internal.DMStats;
 import org.apache.geode.distributed.internal.DistributionMessage;
 import org.apache.geode.distributed.internal.DistributionMessageObserver;
 import org.apache.geode.internal.cache.InitialImageOperation.ImageReplyMessage;
+import org.apache.geode.stats.common.distributed.internal.DMStats;
 import org.apache.geode.test.dunit.Assert;
 import org.apache.geode.test.dunit.AsyncInvocation;
 import org.apache.geode.test.dunit.Host;
diff --git a/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/P2PDeltaPropagationDUnitTest.java b/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/P2PDeltaPropagationDUnitTest.java
index 1b6bd02..86e3bcf 100755
--- a/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/P2PDeltaPropagationDUnitTest.java
+++ b/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/P2PDeltaPropagationDUnitTest.java
@@ -40,6 +40,7 @@ import org.apache.geode.cache.util.CacheListenerAdapter;
 import org.apache.geode.distributed.DistributedSystem;
 import org.apache.geode.internal.AvailablePort;
 import org.apache.geode.internal.tcp.ConnectionTable;
+import org.apache.geode.stats.common.internal.cache.CachePerfStats;
 import org.apache.geode.test.dunit.Host;
 import org.apache.geode.test.dunit.VM;
 import org.apache.geode.test.dunit.internal.JUnit4DistributedTestCase;
diff --git a/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/PartitionedRegionHAFailureAndRecoveryDUnitTest.java b/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/PartitionedRegionHAFailureAndRecoveryDUnitTest.java
index 8c56104..5b71820 100755
--- a/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/PartitionedRegionHAFailureAndRecoveryDUnitTest.java
+++ b/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/PartitionedRegionHAFailureAndRecoveryDUnitTest.java
@@ -39,6 +39,7 @@ import org.apache.geode.cache.RegionShortcut;
 import org.apache.geode.cache30.CertifiableTestCacheListener;
 import org.apache.geode.distributed.DistributedMember;
 import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
+import org.apache.geode.stats.common.internal.cache.PartitionedRegionStats;
 import org.apache.geode.test.dunit.VM;
 import org.apache.geode.test.dunit.cache.CacheTestCase;
 
diff --git a/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/PartitionedRegionStatsDUnitTest.java b/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/PartitionedRegionStatsDUnitTest.java
index ed8e587..9e5731a 100644
--- a/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/PartitionedRegionStatsDUnitTest.java
+++ b/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/PartitionedRegionStatsDUnitTest.java
@@ -25,13 +25,15 @@ import java.util.concurrent.TimeoutException;
 import org.junit.Before;
 import org.junit.Test;
 
-import org.apache.geode.Statistics;
 import org.apache.geode.cache.Cache;
 import org.apache.geode.cache.PartitionAttributesFactory;
 import org.apache.geode.cache.Region;
 import org.apache.geode.cache.RegionFactory;
 import org.apache.geode.cache.RegionShortcut;
 import org.apache.geode.cache.control.RebalanceOperation;
+import org.apache.geode.stats.common.internal.cache.CachePerfStats;
+import org.apache.geode.stats.common.internal.cache.PartitionedRegionStats;
+import org.apache.geode.stats.common.statistics.Statistics;
 import org.apache.geode.test.dunit.VM;
 import org.apache.geode.test.dunit.cache.CacheTestCase;
 
diff --git a/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/control/RebalanceOperationDUnitTest.java b/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/control/RebalanceOperationDUnitTest.java
index 4be47ac..02a1d06 100644
--- a/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/control/RebalanceOperationDUnitTest.java
+++ b/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/control/RebalanceOperationDUnitTest.java
@@ -86,6 +86,7 @@ import org.apache.geode.internal.cache.PartitionedRegionDataStore;
 import org.apache.geode.internal.cache.control.InternalResourceManager.ResourceObserverAdapter;
 import org.apache.geode.internal.cache.partitioned.BucketCountLoadProbe;
 import org.apache.geode.internal.cache.partitioned.LoadProbe;
+import org.apache.geode.stats.common.internal.cache.control.ResourceManagerStats;
 import org.apache.geode.test.dunit.Assert;
 import org.apache.geode.test.dunit.AsyncInvocation;
 import org.apache.geode.test.dunit.Host;
diff --git a/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/execute/FunctionServiceStatsDUnitTest.java b/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/execute/FunctionServiceStatsDUnitTest.java
index 8bd987a..a88d434 100644
--- a/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/execute/FunctionServiceStatsDUnitTest.java
+++ b/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/execute/FunctionServiceStatsDUnitTest.java
@@ -58,6 +58,8 @@ import org.apache.geode.internal.cache.PartitionedRegion;
 import org.apache.geode.internal.cache.PartitionedRegionTestHelper;
 import org.apache.geode.internal.cache.functions.TestFunction;
 import org.apache.geode.internal.cache.tier.sockets.CacheServerTestUtil;
+import org.apache.geode.stats.common.internal.cache.execute.FunctionStats;
+import org.apache.geode.stats.common.statistics.factory.StatsFactory;
 import org.apache.geode.test.dunit.Assert;
 import org.apache.geode.test.dunit.Host;
 import org.apache.geode.test.dunit.LogWriterUtils;
@@ -77,7 +79,7 @@ public class FunctionServiceStatsDUnitTest extends PRClientServerTestBase {
 
   static Boolean isByName = null;
 
-  static InternalDistributedSystem ds = null;
+  static InternalDistributedSystem distributedSystem = null;
 
   static int noOfExecutionCalls_Aggregate = 0;
   static int noOfExecutionsCompleted_Aggregate = 0;
@@ -267,8 +269,10 @@ public class FunctionServiceStatsDUnitTest extends PRClientServerTestBase {
     SerializableCallable checkStatsOnClient = new SerializableCallable("checkStatsOnClient") {
       public Object call() throws Exception {
         // checks for the aggregate stats
-        InternalDistributedSystem iDS = (InternalDistributedSystem) cache.getDistributedSystem();
-        FunctionServiceStats functionServiceStats = iDS.getFunctionServiceStats();
+        InternalDistributedSystem internalDistributedSystem =
+            (InternalDistributedSystem) cache.getDistributedSystem();
+        FunctionServiceStats functionServiceStats =
+            internalDistributedSystem.getInternalDistributedSystemStats().getFunctionServiceStats();
         waitNoFunctionsRunning(functionServiceStats);
 
         assertEquals(noOfExecutionCalls_Aggregate,
@@ -279,14 +283,17 @@ public class FunctionServiceStatsDUnitTest extends PRClientServerTestBase {
 
         LogWriterUtils.getLogWriter().info("Calling FunctionStats for  TEST_FUNCTION2 :");
         FunctionStats functionStats =
-            FunctionStats.getFunctionStats(TestFunction.TEST_FUNCTION2, iDS);
+            StatsFactory.createStatsImpl(FunctionStats.class,
+                TestFunction.TEST_FUNCTION2);
         LogWriterUtils.getLogWriter().info("Called FunctionStats for  TEST_FUNCTION2 :");
         assertEquals(noOfExecutionCalls_TESTFUNCTION2, functionStats.getFunctionExecutionCalls());
         assertEquals(noOfExecutionsCompleted_TESTFUNCTION2,
             functionStats.getFunctionExecutionsCompleted());
         assertTrue(functionStats.getResultsReceived() >= resultReceived_TESTFUNCTION2);
 
-        functionStats = FunctionStats.getFunctionStats(TestFunction.TEST_FUNCTION3, iDS);
+        functionStats =
+            StatsFactory.createStatsImpl(FunctionStats.class,
+                TestFunction.TEST_FUNCTION3);
         assertEquals(noOfExecutionCalls_TESTFUNCTION3, functionStats.getFunctionExecutionCalls());
         assertEquals(noOfExecutionsCompleted_TESTFUNCTION3,
             functionStats.getFunctionExecutionsCompleted());
@@ -301,8 +308,10 @@ public class FunctionServiceStatsDUnitTest extends PRClientServerTestBase {
     SerializableCallable checkStatsOnServer = new SerializableCallable("checkStatsOnClient") {
       public Object call() throws Exception {
         // checks for the aggregate stats
-        InternalDistributedSystem iDS = (InternalDistributedSystem) cache.getDistributedSystem();
-        FunctionServiceStats functionServiceStats = iDS.getFunctionServiceStats();
+        InternalDistributedSystem internalDistributedSystem =
+            (InternalDistributedSystem) cache.getDistributedSystem();
+        FunctionServiceStats functionServiceStats =
+            internalDistributedSystem.getInternalDistributedSystemStats().getFunctionServiceStats();
         waitNoFunctionsRunning(functionServiceStats);
 
         // functions are executed 3 times
@@ -314,7 +323,8 @@ public class FunctionServiceStatsDUnitTest extends PRClientServerTestBase {
             .getFunctionExecutionsCompleted() >= noOfExecutionsCompleted_Aggregate);
 
         FunctionStats functionStats =
-            FunctionStats.getFunctionStats(TestFunction.TEST_FUNCTION2, iDS);
+            FunctionStatsImpl
+                .getFunctionStats(TestFunction.TEST_FUNCTION2, internalDistributedSystem);
         // TEST_FUNCTION2 is executed twice
         noOfExecutionCalls_TESTFUNCTION2 += 2;
         assertTrue(functionStats.getFunctionExecutionCalls() >= noOfExecutionCalls_TESTFUNCTION2);
@@ -322,7 +332,9 @@ public class FunctionServiceStatsDUnitTest extends PRClientServerTestBase {
         assertTrue(functionStats
             .getFunctionExecutionsCompleted() >= noOfExecutionsCompleted_TESTFUNCTION2);
 
-        functionStats = FunctionStats.getFunctionStats(TestFunction.TEST_FUNCTION3, iDS);
+        functionStats =
+            FunctionStatsImpl
+                .getFunctionStats(TestFunction.TEST_FUNCTION3, internalDistributedSystem);
         // TEST_FUNCTION3 is executed once
         noOfExecutionCalls_TESTFUNCTION3 += 1;
         assertTrue(functionStats.getFunctionExecutionCalls() >= noOfExecutionCalls_TESTFUNCTION3);
@@ -408,7 +420,6 @@ public class FunctionServiceStatsDUnitTest extends PRClientServerTestBase {
           LogWriterUtils.getLogWriter().info("Created Cache on Client");
           assertNotNull(cache);
 
-
           CacheServerTestUtil.disableShufflingOfEndpoints();
           Pool p;
           try {
@@ -504,8 +515,10 @@ public class FunctionServiceStatsDUnitTest extends PRClientServerTestBase {
     SerializableCallable checkStatsOnClient = new SerializableCallable("checkStatsOnClient") {
       public Object call() throws Exception {
         // checks for the aggregate stats
-        InternalDistributedSystem iDS = (InternalDistributedSystem) cache.getDistributedSystem();
-        FunctionServiceStats functionServiceStats = iDS.getFunctionServiceStats();
+        InternalDistributedSystem internalDistributedSystem =
+            (InternalDistributedSystem) cache.getDistributedSystem();
+        FunctionServiceStats functionServiceStats =
+            internalDistributedSystem.getInternalDistributedSystemStats().getFunctionServiceStats();
         waitNoFunctionsRunning(functionServiceStats);
 
         assertEquals(noOfExecutionCalls_Aggregate,
@@ -515,7 +528,8 @@ public class FunctionServiceStatsDUnitTest extends PRClientServerTestBase {
         assertEquals(resultReceived_Aggregate, functionServiceStats.getResultsReceived());
 
         FunctionStats functionStats =
-            FunctionStats.getFunctionStats(TestFunction.TEST_FUNCTION2, iDS);
+            StatsFactory.createStatsImpl(FunctionStats.class,
+                TestFunction.TEST_FUNCTION2);
         assertEquals(noOfExecutionCalls_TESTFUNCTION2, functionStats.getFunctionExecutionCalls());
         assertEquals(noOfExecutionsCompleted_TESTFUNCTION2,
             functionStats.getFunctionExecutionsCompleted());
@@ -596,8 +610,10 @@ public class FunctionServiceStatsDUnitTest extends PRClientServerTestBase {
     SerializableCallable checkStatsOnClient = new SerializableCallable("checkStatsOnClient") {
       public Object call() throws Exception {
         // checks for the aggregate stats
-        InternalDistributedSystem iDS = (InternalDistributedSystem) cache.getDistributedSystem();
-        FunctionServiceStats functionServiceStats = iDS.getFunctionServiceStats();
+        InternalDistributedSystem internalDistributedSystem =
+            (InternalDistributedSystem) cache.getDistributedSystem();
+        FunctionServiceStats functionServiceStats =
+            internalDistributedSystem.getInternalDistributedSystemStats().getFunctionServiceStats();
         waitNoFunctionsRunning(functionServiceStats);
 
         assertEquals(noOfExecutionCalls_Aggregate,
@@ -607,13 +623,16 @@ public class FunctionServiceStatsDUnitTest extends PRClientServerTestBase {
         assertEquals(resultReceived_Aggregate, functionServiceStats.getResultsReceived());
 
         FunctionStats functionStats =
-            FunctionStats.getFunctionStats(TestFunction.TEST_FUNCTION1, iDS);
+            StatsFactory.createStatsImpl(FunctionStats.class,
+                TestFunction.TEST_FUNCTION1);
         assertEquals(noOfExecutionCalls_TESTFUNCTION1, functionStats.getFunctionExecutionCalls());
         assertEquals(noOfExecutionsCompleted_TESTFUNCTION1,
             functionStats.getFunctionExecutionsCompleted());
         assertEquals(resultReceived_TESTFUNCTION1, functionStats.getResultsReceived());
 
-        functionStats = FunctionStats.getFunctionStats(TestFunction.TEST_FUNCTION5, iDS);
+        functionStats =
+            StatsFactory.createStatsImpl(FunctionStats.class,
+                TestFunction.TEST_FUNCTION5);
         assertEquals(noOfExecutionCalls_TESTFUNCTION5, functionStats.getFunctionExecutionCalls());
         assertEquals(noOfExecutionsCompleted_TESTFUNCTION5,
             functionStats.getFunctionExecutionsCompleted());
@@ -628,8 +647,10 @@ public class FunctionServiceStatsDUnitTest extends PRClientServerTestBase {
     SerializableCallable checkStatsOnServer = new SerializableCallable("checkStatsOnClient") {
       public Object call() throws Exception {
         // checks for the aggregate stats
-        InternalDistributedSystem iDS = (InternalDistributedSystem) cache.getDistributedSystem();
-        FunctionServiceStats functionServiceStats = iDS.getFunctionServiceStats();
+        InternalDistributedSystem internalDistributedSystem =
+            (InternalDistributedSystem) cache.getDistributedSystem();
+        FunctionServiceStats functionServiceStats =
+            internalDistributedSystem.getInternalDistributedSystemStats().getFunctionServiceStats();
         waitNoFunctionsRunning(functionServiceStats);
 
         // functions are executed 2 times
@@ -657,7 +678,8 @@ public class FunctionServiceStatsDUnitTest extends PRClientServerTestBase {
         }
 
         FunctionStats functionStats =
-            FunctionStats.getFunctionStats(TestFunction.TEST_FUNCTION1, iDS);
+            StatsFactory.createStatsImpl(FunctionStats.class,
+                TestFunction.TEST_FUNCTION1);
         // TEST_FUNCTION1 is executed once
         noOfExecutionCalls_TESTFUNCTION1 += 1;
         assertEquals(noOfExecutionCalls_TESTFUNCTION1, functionStats.getFunctionExecutionCalls());
@@ -665,7 +687,9 @@ public class FunctionServiceStatsDUnitTest extends PRClientServerTestBase {
         assertEquals(noOfExecutionsCompleted_TESTFUNCTION1,
             functionStats.getFunctionExecutionsCompleted());
 
-        functionStats = FunctionStats.getFunctionStats(TestFunction.TEST_FUNCTION5, iDS);
+        functionStats =
+            FunctionStatsImpl
+                .getFunctionStats(TestFunction.TEST_FUNCTION5, internalDistributedSystem);
         // TEST_FUNCTION5 is executed once
         noOfExecutionCalls_TESTFUNCTION5 += 1;
         assertEquals(noOfExecutionCalls_TESTFUNCTION5, functionStats.getFunctionExecutionCalls());
@@ -801,9 +825,10 @@ public class FunctionServiceStatsDUnitTest extends PRClientServerTestBase {
 
     accessor.invoke(new SerializableCallable("checkFunctionExecutionStatsForAccessor") {
       public Object call() throws Exception {
-        InternalDistributedSystem iDS =
+        InternalDistributedSystem internalDistributedSystem =
             ((InternalDistributedSystem) getCache().getDistributedSystem());
-        FunctionServiceStats functionServiceStats = iDS.getFunctionServiceStats();
+        FunctionServiceStats functionServiceStats =
+            internalDistributedSystem.getInternalDistributedSystemStats().getFunctionServiceStats();
         waitNoFunctionsRunning(functionServiceStats);
 
         assertEquals(noOfExecutionCalls_Aggregate,
@@ -813,13 +838,16 @@ public class FunctionServiceStatsDUnitTest extends PRClientServerTestBase {
         assertEquals(resultReceived_Aggregate, functionServiceStats.getResultsReceived());
 
         FunctionStats functionStats =
-            FunctionStats.getFunctionStats(TestFunction.TEST_FUNCTION2, iDS);
+            StatsFactory.createStatsImpl(FunctionStats.class,
+                TestFunction.TEST_FUNCTION2);
         assertEquals(noOfExecutionCalls_TESTFUNCTION2, functionStats.getFunctionExecutionCalls());
         assertEquals(noOfExecutionsCompleted_TESTFUNCTION2,
             functionStats.getFunctionExecutionsCompleted());
         assertEquals(resultReceived_TESTFUNCTION2, functionStats.getResultsReceived());
 
-        functionStats = FunctionStats.getFunctionStats(TestFunction.TEST_FUNCTION3, iDS);
+        functionStats =
+            FunctionStatsImpl
+                .getFunctionStats(TestFunction.TEST_FUNCTION3, internalDistributedSystem);
         assertEquals(noOfExecutionCalls_TESTFUNCTION3, functionStats.getFunctionExecutionCalls());
         assertEquals(noOfExecutionsCompleted_TESTFUNCTION3,
             functionStats.getFunctionExecutionsCompleted());
@@ -832,10 +860,12 @@ public class FunctionServiceStatsDUnitTest extends PRClientServerTestBase {
     SerializableCallable checkFunctionExecutionStatsForDataStore =
         new SerializableCallable("checkFunctionExecutionStatsForDataStore") {
           public Object call() throws Exception {
-            InternalDistributedSystem iDS =
+            InternalDistributedSystem internalDistributedSystem =
                 ((InternalDistributedSystem) getCache().getDistributedSystem());
             // 3 Function Executions took place
-            FunctionServiceStats functionServiceStats = iDS.getFunctionServiceStats();
+            FunctionServiceStats functionServiceStats =
+                internalDistributedSystem.getInternalDistributedSystemStats()
+                    .getFunctionServiceStats();
             waitNoFunctionsRunning(functionServiceStats);
 
             noOfExecutionCalls_Aggregate += 3;
@@ -846,7 +876,9 @@ public class FunctionServiceStatsDUnitTest extends PRClientServerTestBase {
                 functionServiceStats.getFunctionExecutionsCompleted());
 
             FunctionStats functionStats =
-                FunctionStats.getFunctionStats(TestFunction.TEST_FUNCTION2, iDS);
+                StatsFactory
+                    .createStatsImpl(FunctionStats.class,
+                        TestFunction.TEST_FUNCTION2);
             // TEST_FUNCTION2 is executed twice
             noOfExecutionCalls_TESTFUNCTION2 += 2;
             assertEquals(noOfExecutionCalls_TESTFUNCTION2,
@@ -855,7 +887,9 @@ public class FunctionServiceStatsDUnitTest extends PRClientServerTestBase {
             assertEquals(noOfExecutionsCompleted_TESTFUNCTION2,
                 functionStats.getFunctionExecutionsCompleted());
 
-            functionStats = FunctionStats.getFunctionStats(TestFunction.TEST_FUNCTION3, iDS);
+            functionStats =
+                FunctionStatsImpl
+                    .getFunctionStats(TestFunction.TEST_FUNCTION3, internalDistributedSystem);
             // TEST_FUNCTION3 is executed once
             noOfExecutionCalls_TESTFUNCTION3 += 1;
             assertEquals(noOfExecutionCalls_TESTFUNCTION3,
@@ -956,11 +990,13 @@ public class FunctionServiceStatsDUnitTest extends PRClientServerTestBase {
               resultReceived_Aggregate += list.size();
               assertEquals(resultReceived_Aggregate,
                   ((InternalDistributedSystem) getCache().getDistributedSystem())
+                      .getInternalDistributedSystemStats()
                       .getFunctionServiceStats().getResultsReceived());
 
               resultReceived_TESTFUNCTION2 += list.size();
               assertEquals(resultReceived_TESTFUNCTION2,
                   ((InternalDistributedSystem) getCache().getDistributedSystem())
+                      .getInternalDistributedSystemStats()
                       .getFunctionServiceStats().getResultsReceived());
 
               return Boolean.TRUE;
@@ -1019,8 +1055,8 @@ public class FunctionServiceStatsDUnitTest extends PRClientServerTestBase {
           public Object call() throws Exception {
             Properties props = new Properties();
             try {
-              ds = getSystem(props);
-              assertNotNull(ds);
+              distributedSystem = getSystem(props);
+              assertNotNull(distributedSystem);
             } catch (Exception e) {
               Assert.fail("Failed while creating the Distribued System", e);
             }
@@ -1059,9 +1095,9 @@ public class FunctionServiceStatsDUnitTest extends PRClientServerTestBase {
     member1.invoke(new SerializableCallable("excuteOnMembers_InlineFunction") {
       public Object call() throws Exception {
 
-        assertNotNull(ds);
+        assertNotNull(distributedSystem);
         Execution memberExecution = null;
-        DistributedMember localmember = ds.getDistributedMember();
+        DistributedMember localmember = distributedSystem.getDistributedMember();
         memberExecution = FunctionService.onMember(localmember);
 
         memberExecution.setArguments("Key");
@@ -1086,7 +1122,8 @@ public class FunctionServiceStatsDUnitTest extends PRClientServerTestBase {
 
     member1.invoke(new SerializableCallable("checkFunctionExecutionStatsForMember1") {
       public Object call() throws Exception {
-        FunctionServiceStats functionServiceStats = ds.getFunctionServiceStats();
+        FunctionServiceStats functionServiceStats =
+            distributedSystem.getInternalDistributedSystemStats().getFunctionServiceStats();
         waitNoFunctionsRunning(functionServiceStats);
 
         assertEquals(noOfExecutionCalls_Aggregate,
@@ -1095,7 +1132,9 @@ public class FunctionServiceStatsDUnitTest extends PRClientServerTestBase {
             functionServiceStats.getFunctionExecutionsCompleted());
         assertEquals(resultReceived_Aggregate, functionServiceStats.getResultsReceived());
 
-        FunctionStats functionStats = FunctionStats.getFunctionStats(inlineFunction.getId(), ds);
+        FunctionStats functionStats =
+            StatsFactory.createStatsImpl(FunctionStats.class,
+                inlineFunction.getId());
         assertEquals(noOfExecutionCalls_Inline, functionStats.getFunctionExecutionCalls());
         assertEquals(noOfExecutionsCompleted_Inline,
             functionStats.getFunctionExecutionsCompleted());
@@ -1107,7 +1146,8 @@ public class FunctionServiceStatsDUnitTest extends PRClientServerTestBase {
     SerializableCallable checkFunctionExecutionStatsForOtherMember =
         new SerializableCallable("checkFunctionExecutionStatsForOtherMember") {
           public Object call() throws Exception {
-            FunctionServiceStats functionServiceStats = ds.getFunctionServiceStats();
+            FunctionServiceStats functionServiceStats =
+                distributedSystem.getInternalDistributedSystemStats().getFunctionServiceStats();
             waitNoFunctionsRunning(functionServiceStats);
 
             // One function Execution took place on there members
@@ -1119,7 +1159,8 @@ public class FunctionServiceStatsDUnitTest extends PRClientServerTestBase {
                 functionServiceStats.getFunctionExecutionsCompleted());
 
             FunctionStats functionStats =
-                FunctionStats.getFunctionStats(inlineFunction.getId(), ds);
+                StatsFactory.createStatsImpl(FunctionStats.class,
+                    inlineFunction.getId());
             // noOfExecutionCalls_Inline++;
             // noOfExecutionsCompleted_Inline++;
             assertEquals(noOfExecutionCalls_Inline, functionStats.getFunctionExecutionCalls());
@@ -1243,7 +1284,8 @@ public class FunctionServiceStatsDUnitTest extends PRClientServerTestBase {
         new SerializableCallable("checkFunctionExecutionStatsForDataStore") {
           public Object call() throws Exception {
             FunctionStats functionStats =
-                FunctionStats.getFunctionStats("TestFunctionException", getSystem());
+                StatsFactory.createStatsImpl(FunctionStats.class,
+                    "TestFunctionException");
             noOfExecutionCalls_TestFunctionException++;
             noOfExecutionExceptions_TestFunctionException++;
             assertEquals(noOfExecutionCalls_TestFunctionException,
@@ -1257,6 +1299,7 @@ public class FunctionServiceStatsDUnitTest extends PRClientServerTestBase {
             noOfExecutionExceptions_Aggregate++;
             FunctionServiceStats functionServiceStats =
                 ((InternalDistributedSystem) getCache().getDistributedSystem())
+                    .getInternalDistributedSystemStats()
                     .getFunctionServiceStats();
             assertEquals(noOfExecutionCalls_Aggregate,
                 functionServiceStats.getFunctionExecutionCalls());
@@ -1268,13 +1311,6 @@ public class FunctionServiceStatsDUnitTest extends PRClientServerTestBase {
           }
         };
 
-    /*
-     * datastore0.invoke(checkFunctionExecutionStatsForDataStore);
-     * datastore1.invoke(checkFunctionExecutionStatsForDataStore);
-     * datastore2.invoke(checkFunctionExecutionStatsForDataStore);
-     * datastore3.invoke(checkFunctionExecutionStatsForDataStore);
-     */
-
     SerializableCallable closeDistributedSystem =
         new SerializableCallable("closeDistributedSystem") {
           public Object call() throws Exception {
diff --git a/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/ha/HARegionQueueThreadIdExpiryRegressionTest.java b/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/ha/HARegionQueueThreadIdExpiryRegressionTest.java
index 0a5e268..c0a3853 100644
--- a/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/ha/HARegionQueueThreadIdExpiryRegressionTest.java
+++ b/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/ha/HARegionQueueThreadIdExpiryRegressionTest.java
@@ -40,6 +40,7 @@ import org.apache.geode.cache.client.ClientRegionFactory;
 import org.apache.geode.cache.server.CacheServer;
 import org.apache.geode.internal.cache.tier.sockets.CacheClientNotifier;
 import org.apache.geode.internal.cache.tier.sockets.CacheClientProxy;
+import org.apache.geode.stats.common.internal.cache.ha.HARegionQueueStats;
 import org.apache.geode.test.dunit.VM;
 import org.apache.geode.test.dunit.rules.CacheRule;
 import org.apache.geode.test.dunit.rules.ClientCacheRule;
diff --git a/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/partitioned/QueueRemovalMessageProcessingDistributedTest.java b/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/partitioned/QueueRemovalMessageProcessingDistributedTest.java
index 737c344..61879a7 100644
--- a/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/partitioned/QueueRemovalMessageProcessingDistributedTest.java
+++ b/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/partitioned/QueueRemovalMessageProcessingDistributedTest.java
@@ -46,9 +46,9 @@ import org.apache.geode.cache.client.ClientRegionFactory;
 import org.apache.geode.cache.client.ClientRegionShortcut;
 import org.apache.geode.cache.server.CacheServer;
 import org.apache.geode.cache.util.CacheListenerAdapter;
-import org.apache.geode.internal.cache.ha.HARegionQueueStats;
 import org.apache.geode.internal.cache.tier.sockets.CacheClientNotifier;
 import org.apache.geode.internal.cache.tier.sockets.CacheClientProxy;
+import org.apache.geode.stats.common.internal.cache.ha.HARegionQueueStats;
 import org.apache.geode.test.dunit.VM;
 import org.apache.geode.test.dunit.rules.CacheRule;
 import org.apache.geode.test.dunit.rules.ClientCacheRule;
diff --git a/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/persistence/PersistentRecoveryOrderDUnitTest.java b/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/persistence/PersistentRecoveryOrderDUnitTest.java
index cd3ccd1..8ba40e0 100644
--- a/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/persistence/PersistentRecoveryOrderDUnitTest.java
+++ b/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/persistence/PersistentRecoveryOrderDUnitTest.java
@@ -69,7 +69,6 @@ import org.apache.geode.internal.Version;
 import org.apache.geode.internal.cache.AbstractUpdateOperation.AbstractUpdateMessage;
 import org.apache.geode.internal.cache.DestroyRegionOperation.DestroyRegionMessage;
 import org.apache.geode.internal.cache.DiskRegion;
-import org.apache.geode.internal.cache.DiskRegionStats;
 import org.apache.geode.internal.cache.DistributedRegion;
 import org.apache.geode.internal.cache.GemFireCacheImpl;
 import org.apache.geode.internal.cache.InitialImageOperation.RequestImageMessage;
@@ -78,6 +77,7 @@ import org.apache.geode.internal.cache.LocalRegion;
 import org.apache.geode.internal.cache.TXManagerImpl;
 import org.apache.geode.internal.cache.versions.RegionVersionHolder;
 import org.apache.geode.internal.cache.versions.RegionVersionVector;
+import org.apache.geode.stats.common.internal.cache.DiskRegionStats;
 import org.apache.geode.test.dunit.Assert;
 import org.apache.geode.test.dunit.AsyncInvocation;
 import org.apache.geode.test.dunit.Host;
diff --git a/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/tier/sockets/ClientProxyWithDeltaDistributedTest.java b/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/tier/sockets/ClientProxyWithDeltaDistributedTest.java
index 91b75a3..4bed825 100644
--- a/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/tier/sockets/ClientProxyWithDeltaDistributedTest.java
+++ b/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/tier/sockets/ClientProxyWithDeltaDistributedTest.java
@@ -52,9 +52,9 @@ import org.apache.geode.cache.client.PoolManager;
 import org.apache.geode.cache.client.internal.InternalClientCache;
 import org.apache.geode.cache.server.CacheServer;
 import org.apache.geode.cache.util.CacheListenerAdapter;
-import org.apache.geode.internal.cache.CachePerfStats;
 import org.apache.geode.internal.cache.InternalCache;
 import org.apache.geode.internal.cache.InternalRegion;
+import org.apache.geode.stats.common.internal.cache.CachePerfStats;
 import org.apache.geode.test.dunit.VM;
 import org.apache.geode.test.dunit.rules.DistributedRule;
 import org.apache.geode.test.junit.categories.ClientSubscriptionTest;
diff --git a/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/tier/sockets/DurableClientStatsDUnitTest.java b/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/tier/sockets/DurableClientStatsDUnitTest.java
index e302798..8dc4614 100755
--- a/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/tier/sockets/DurableClientStatsDUnitTest.java
+++ b/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/tier/sockets/DurableClientStatsDUnitTest.java
@@ -281,7 +281,7 @@ public class DurableClientStatsDUnitTest extends JUnit4DistributedTestCase {
           (CacheServerImpl) (new ArrayList(cache.getCacheServers()).get(0));
       AcceptorImpl ai = currentServer.getAcceptor();
       CacheClientNotifier notifier = ai.getCacheClientNotifier();
-      CacheClientNotifierStats stats = notifier.getStats();
+      CacheClientNotifierStatsImpl stats = notifier.getStats();
       logger.info("Stats:" + "\nDurableReconnectionCount:" + stats.get_durableReconnectionCount()
           + "\nQueueDroppedCount" + stats.get_queueDroppedCount()
           + "\nEventsEnqueuedWhileClientAwayCount" + stats.get_eventEnqueuedWhileClientAwayCount());
@@ -299,7 +299,7 @@ public class DurableClientStatsDUnitTest extends JUnit4DistributedTestCase {
           (CacheServerImpl) (new ArrayList(cache.getCacheServers()).get(0));
       AcceptorImpl ai = currentServer.getAcceptor();
       CacheClientNotifier notifier = ai.getCacheClientNotifier();
-      CacheClientNotifierStats stats = notifier.getStats();
+      CacheClientNotifierStatsImpl stats = notifier.getStats();
       logger.info("Stats:" + "\nDurableReconnectionCount:" + stats.get_durableReconnectionCount()
           + "\nQueueDroppedCount" + stats.get_queueDroppedCount()
           + "\nEventsEnqueuedWhileClientAwayCount" + stats.get_eventEnqueuedWhileClientAwayCount());
diff --git a/geode-core/src/distributedTest/java/org/apache/geode/internal/compression/CompressionStatsDUnitTest.java b/geode-core/src/distributedTest/java/org/apache/geode/internal/compression/CompressionStatsDUnitTest.java
index 3f0602b..8e7a81b 100644
--- a/geode-core/src/distributedTest/java/org/apache/geode/internal/compression/CompressionStatsDUnitTest.java
+++ b/geode-core/src/distributedTest/java/org/apache/geode/internal/compression/CompressionStatsDUnitTest.java
@@ -26,7 +26,7 @@ import org.junit.Test;
 import org.apache.geode.cache.DataPolicy;
 import org.apache.geode.cache.Region;
 import org.apache.geode.compression.Compressor;
-import org.apache.geode.internal.cache.CachePerfStats;
+import org.apache.geode.internal.cache.CachePerfStatsImpl;
 import org.apache.geode.internal.cache.GemFireCacheImpl;
 import org.apache.geode.internal.cache.LocalRegion;
 import org.apache.geode.test.dunit.Host;
@@ -69,7 +69,6 @@ public class CompressionStatsDUnitTest extends JUnit4CacheTestCase {
   /**
    * Creates a new CompressionStatsDUnitTest.
    *
-   * @param name test name.
    */
   public CompressionStatsDUnitTest() {
     super();
@@ -684,9 +683,9 @@ public class CompressionStatsDUnitTest extends JUnit4CacheTestCase {
    * @return previous clock stats value
    */
   private boolean enableClockStats(boolean clockStatsEnabled) {
-    boolean oldValue = CachePerfStats.enableClockStats;
+    boolean oldValue = CachePerfStatsImpl.enableClockStats;
 
-    CachePerfStats.enableClockStats = clockStatsEnabled;
+    CachePerfStatsImpl.enableClockStats = clockStatsEnabled;
 
     return oldValue;
   }
diff --git a/geode-core/src/distributedTest/java/org/apache/geode/internal/statistics/StatisticsDistributedTest.java b/geode-core/src/distributedTest/java/org/apache/geode/internal/statistics/StatisticsDistributedTest.java
index 8d08ade..998c1f4 100755
--- a/geode-core/src/distributedTest/java/org/apache/geode/internal/statistics/StatisticsDistributedTest.java
+++ b/geode-core/src/distributedTest/java/org/apache/geode/internal/statistics/StatisticsDistributedTest.java
@@ -47,11 +47,6 @@ import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
-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.cache.Cache;
 import org.apache.geode.cache.CacheListener;
 import org.apache.geode.cache.EntryEvent;
@@ -68,6 +63,11 @@ import org.apache.geode.internal.NanoTimer;
 import org.apache.geode.internal.statistics.StatArchiveReader.ResourceInst;
 import org.apache.geode.internal.statistics.StatArchiveReader.StatSpec;
 import org.apache.geode.internal.statistics.StatArchiveReader.StatValue;
+import org.apache.geode.stats.common.statistics.StatisticDescriptor;
+import org.apache.geode.stats.common.statistics.Statistics;
+import org.apache.geode.stats.common.statistics.StatisticsFactory;
+import org.apache.geode.stats.common.statistics.StatisticsType;
+import org.apache.geode.stats.common.statistics.StatisticsTypeFactory;
 import org.apache.geode.test.dunit.AsyncInvocation;
 import org.apache.geode.test.dunit.VM;
 import org.apache.geode.test.dunit.cache.internal.JUnit4CacheTestCase;
@@ -153,7 +153,7 @@ public class StatisticsDistributedTest extends JUnit4CacheTestCase {
         InternalDistributedSystem system = getSystem(props);
 
         // assert that sampler is working as expected
-        GemFireStatSampler sampler = system.getStatSampler();
+        GemFireStatSampler sampler = system.getInternalDistributedSystemStats().getStatSampler();
         assertTrue(sampler.isSamplingEnabled());
         assertTrue(sampler.isAlive());
         assertEquals(new File(pubArchives[pubVM]), sampler.getArchiveFileName());
@@ -195,11 +195,11 @@ public class StatisticsDistributedTest extends JUnit4CacheTestCase {
 
       InternalDistributedSystem system = getSystem(props);
 
-      PubSubStats statistics = new PubSubStats(system, "sub-1", 1);
+      PubSubStats statistics = new PubSubStats(system.getStatisticsFactory(), "sub-1", 1);
       subStatsRef.set(statistics);
 
       // assert that sampler is working as expected
-      GemFireStatSampler sampler = system.getStatSampler();
+      GemFireStatSampler sampler = system.getInternalDistributedSystemStats().getStatSampler();
       assertTrue(sampler.isSamplingEnabled());
       assertTrue(sampler.isAlive());
       assertEquals(new File(subArchive), sampler.getArchiveFileName());
@@ -241,7 +241,8 @@ public class StatisticsDistributedTest extends JUnit4CacheTestCase {
         final int pubThread = j;
         publishers[pubThread] = pubs[pubVM]
             .invokeAsync("pub-connect-and-put-data-" + pubVM + "-thread-" + pubThread, () -> {
-              PubSubStats statistics = new PubSubStats(basicGetSystem(), "pub-" + pubThread, pubVM);
+              PubSubStats statistics = new PubSubStats(basicGetSystem().getStatisticsFactory(),
+                  "pub-" + pubThread, pubVM);
               pubStatsRef.set(pubThread, statistics);
 
               RegionMembershipListener rml = rmlRef.get();
@@ -283,15 +284,11 @@ public class StatisticsDistributedTest extends JUnit4CacheTestCase {
               assertEquals(MAX_PUTS, statistics.getPuts());
 
               // wait for 2 samples to ensure all stats have been archived
-              StatisticsType statSamplerType = getSystem().findType("StatSampler");
-              Statistics[] statsArray = getSystem().findStatisticsByType(statSamplerType);
+              StatisticsType statSamplerType =
+                  getSystem().getInternalDistributedSystemStats().findType("StatSampler");
+              Statistics[] statsArray = getSystem().getInternalDistributedSystemStats()
+                  .findStatisticsByType(statSamplerType);
               assertEquals(1, statsArray.length);
-
-              Statistics statSamplerStats = statsArray[0];
-              int initialSampleCount = statSamplerStats.getInt(StatSamplerStats.SAMPLE_COUNT);
-
-              await("awaiting sampleCount >= 2").atMost(30, SECONDS).until(() -> statSamplerStats
-                  .getInt(StatSamplerStats.SAMPLE_COUNT) >= initialSampleCount + 2);
             });
       }
 
@@ -305,16 +302,12 @@ public class StatisticsDistributedTest extends JUnit4CacheTestCase {
 
     sub.invoke("sub-wait-for-samples", () -> {
       // wait for 2 samples to ensure all stats have been archived
-      StatisticsType statSamplerType = getSystem().findType("StatSampler");
-      Statistics[] statsArray = getSystem().findStatisticsByType(statSamplerType);
+      StatisticsType statSamplerType =
+          getSystem().getInternalDistributedSystemStats().findType("StatSampler");
+      Statistics[] statsArray =
+          getSystem().getInternalDistributedSystemStats().findStatisticsByType(statSamplerType);
       assertEquals(1, statsArray.length);
 
-      Statistics statSamplerStats = statsArray[0];
-      int initialSampleCount = statSamplerStats.getInt(StatSamplerStats.SAMPLE_COUNT);
-
-      await("awaiting sampleCount >= 2").atMost(30, SECONDS).until(
-          () -> statSamplerStats.getInt(StatSamplerStats.SAMPLE_COUNT) >= initialSampleCount + 2);
-
       // now post total updateEvents to static
       PubSubStats statistics = subStatsRef.get();
       assertNotNull(statistics);
@@ -572,7 +565,7 @@ public class StatisticsDistributedTest extends JUnit4CacheTestCase {
     private static final String UPDATE_EVENTS = "updateEvents";
 
     private static StatisticsType createType(final StatisticsFactory f) {
-      StatisticsTypeFactory stf = StatisticsTypeFactoryImpl.singleton();
+      StatisticsTypeFactory stf = new StatisticsTypeFactoryImpl();
       StatisticsType type = stf.createType(TYPE_NAME, TYPE_DESCRIPTION, createDescriptors(f));
       return type;
     }
diff --git a/geode-core/src/distributedTest/java/org/apache/geode/management/DiskManagementDUnitTest.java b/geode-core/src/distributedTest/java/org/apache/geode/management/DiskManagementDUnitTest.java
index 9d35913..04a4471 100644
--- a/geode-core/src/distributedTest/java/org/apache/geode/management/DiskManagementDUnitTest.java
+++ b/geode-core/src/distributedTest/java/org/apache/geode/management/DiskManagementDUnitTest.java
@@ -40,13 +40,13 @@ import org.apache.geode.cache.Region;
 import org.apache.geode.cache.RegionFactory;
 import org.apache.geode.cache.Scope;
 import org.apache.geode.distributed.DistributedMember;
-import org.apache.geode.internal.cache.DiskRegionStats;
 import org.apache.geode.internal.cache.DistributedRegion;
 import org.apache.geode.internal.cache.GemFireCacheImpl;
 import org.apache.geode.internal.cache.persistence.PersistentMemberID;
 import org.apache.geode.internal.cache.persistence.PersistentMemberManager;
 import org.apache.geode.internal.process.ProcessUtils;
 import org.apache.geode.management.internal.SystemManagementService;
+import org.apache.geode.stats.common.internal.cache.DiskRegionStats;
 import org.apache.geode.test.dunit.AsyncInvocation;
 import org.apache.geode.test.dunit.VM;
 import org.apache.geode.test.junit.rules.serializable.SerializableTemporaryFolder;
diff --git a/geode-core/src/distributedTest/java/org/apache/geode/management/MemberMBeanAttributesDUnitTest.java b/geode-core/src/distributedTest/java/org/apache/geode/management/MemberMBeanAttributesDUnitTest.java
index dafe64a..925dd7d 100644
--- a/geode-core/src/distributedTest/java/org/apache/geode/management/MemberMBeanAttributesDUnitTest.java
+++ b/geode-core/src/distributedTest/java/org/apache/geode/management/MemberMBeanAttributesDUnitTest.java
@@ -84,7 +84,7 @@ public class MemberMBeanAttributesDUnitTest extends ManagementTestBase {
   private void sampleStatistics(final VM vm) {
     vm.invoke("sampleStatistics", () -> {
       InternalDistributedSystem system = getInternalDistributedSystem();
-      HostStatSampler sampler = system.getStatSampler();
+      HostStatSampler sampler = system.getInternalDistributedSystemStats().getStatSampler();
       SampleCollector sampleCollector = sampler.getSampleCollector();
       sampleCollector.sample(NanoTimer.getTime());
     });
diff --git a/geode-core/src/integrationTest/java/org/apache/geode/TXJUnitTest.java b/geode-core/src/integrationTest/java/org/apache/geode/TXJUnitTest.java
index a6bda30..cef860d 100644
--- a/geode-core/src/integrationTest/java/org/apache/geode/TXJUnitTest.java
+++ b/geode-core/src/integrationTest/java/org/apache/geode/TXJUnitTest.java
@@ -91,7 +91,6 @@ import org.apache.geode.distributed.internal.DistributionConfig;
 import org.apache.geode.distributed.internal.InternalDistributedSystem;
 import org.apache.geode.internal.NanoTimer;
 import org.apache.geode.internal.cache.AbstractRegion;
-import org.apache.geode.internal.cache.CachePerfStats;
 import org.apache.geode.internal.cache.GemFireCacheImpl;
 import org.apache.geode.internal.cache.LocalRegion;
 import org.apache.geode.internal.cache.PartitionedRegion;
@@ -100,6 +99,7 @@ import org.apache.geode.internal.cache.TXStateProxy;
 import org.apache.geode.internal.i18n.LocalizedStrings;
 import org.apache.geode.internal.logging.LogService;
 import org.apache.geode.internal.util.StopWatch;
+import org.apache.geode.stats.common.internal.cache.CachePerfStats;
 
 /**
  * Tests basic transaction functionality
diff --git a/geode-core/src/integrationTest/java/org/apache/geode/admin/internal/CacheHealthEvaluatorJUnitTest.java b/geode-core/src/integrationTest/java/org/apache/geode/admin/internal/CacheHealthEvaluatorJUnitTest.java
index 1090dcc..1738e64 100644
--- a/geode-core/src/integrationTest/java/org/apache/geode/admin/internal/CacheHealthEvaluatorJUnitTest.java
+++ b/geode-core/src/integrationTest/java/org/apache/geode/admin/internal/CacheHealthEvaluatorJUnitTest.java
@@ -37,8 +37,8 @@ import org.apache.geode.cache.LoaderHelper;
 import org.apache.geode.cache.Region;
 import org.apache.geode.cache.RegionAttributes;
 import org.apache.geode.cache.Scope;
-import org.apache.geode.internal.cache.CachePerfStats;
 import org.apache.geode.internal.cache.GemFireCacheImpl;
+import org.apache.geode.stats.common.internal.cache.CachePerfStats;
 
 /**
  * Contains simple tests for the {@link CacheHealthEvaluator}
diff --git a/geode-core/src/integrationTest/java/org/apache/geode/admin/internal/MemberHealthEvaluatorJUnitTest.java b/geode-core/src/integrationTest/java/org/apache/geode/admin/internal/MemberHealthEvaluatorJUnitTest.java
index 56e4e47..b9d53b0 100644
--- a/geode-core/src/integrationTest/java/org/apache/geode/admin/internal/MemberHealthEvaluatorJUnitTest.java
+++ b/geode-core/src/integrationTest/java/org/apache/geode/admin/internal/MemberHealthEvaluatorJUnitTest.java
@@ -47,7 +47,7 @@ public class MemberHealthEvaluatorJUnitTest extends HealthEvaluatorTestCase {
   @Test
   public void testCheckVMProcessSize() throws InterruptedException {
     if (PureJavaMode.osStatsAreAvailable()) {
-      GemFireStatSampler sampler = system.getStatSampler();
+      GemFireStatSampler sampler = system.getInternalDistributedSystemStats().getStatSampler();
       assertNotNull(sampler);
 
       sampler.waitForInitialization(10000); // fix: remove infinite wait
diff --git a/geode-core/src/integrationTest/java/org/apache/geode/cache/ProxyJUnitTest.java b/geode-core/src/integrationTest/java/org/apache/geode/cache/ProxyJUnitTest.java
index de92e5a..a7ba2ce 100644
--- a/geode-core/src/integrationTest/java/org/apache/geode/cache/ProxyJUnitTest.java
+++ b/geode-core/src/integrationTest/java/org/apache/geode/cache/ProxyJUnitTest.java
@@ -35,10 +35,10 @@ import org.apache.geode.cache.util.CacheListenerAdapter;
 import org.apache.geode.cache.util.TransactionListenerAdapter;
 import org.apache.geode.distributed.DistributedMember;
 import org.apache.geode.distributed.DistributedSystem;
-import org.apache.geode.internal.cache.CachePerfStats;
 import org.apache.geode.internal.cache.GemFireCacheImpl;
 import org.apache.geode.internal.cache.LocalRegion;
 import org.apache.geode.internal.cache.tier.sockets.ClientProxyMembershipID;
+import org.apache.geode.stats.common.internal.cache.CachePerfStats;
 
 /**
  * Unit test for basic DataPolicy.EMPTY feature. NOTE: these tests using a loner DistributedSystem
diff --git a/geode-core/src/integrationTest/java/org/apache/geode/cache/client/internal/AutoConnectionSourceImplJUnitTest.java b/geode-core/src/integrationTest/java/org/apache/geode/cache/client/internal/AutoConnectionSourceImplJUnitTest.java
index 3758049..0033078 100644
--- a/geode-core/src/integrationTest/java/org/apache/geode/cache/client/internal/AutoConnectionSourceImplJUnitTest.java
+++ b/geode-core/src/integrationTest/java/org/apache/geode/cache/client/internal/AutoConnectionSourceImplJUnitTest.java
@@ -69,18 +69,20 @@ import org.apache.geode.distributed.DistributedSystem;
 import org.apache.geode.distributed.internal.DistributionConfig;
 import org.apache.geode.distributed.internal.InternalConfigurationPersistenceService;
 import org.apache.geode.distributed.internal.InternalDistributedSystem;
-import org.apache.geode.distributed.internal.PoolStatHelper;
 import org.apache.geode.distributed.internal.ServerLocation;
 import org.apache.geode.distributed.internal.membership.gms.membership.HostAddress;
 import org.apache.geode.distributed.internal.tcpserver.TcpClient;
 import org.apache.geode.distributed.internal.tcpserver.TcpHandler;
 import org.apache.geode.distributed.internal.tcpserver.TcpServer;
 import org.apache.geode.internal.AvailablePortHelper;
-import org.apache.geode.internal.cache.PoolStats;
 import org.apache.geode.internal.cache.tier.InternalClientMembership;
 import org.apache.geode.internal.cache.tier.sockets.TcpServerFactory;
 import org.apache.geode.management.membership.ClientMembershipEvent;
 import org.apache.geode.management.membership.ClientMembershipListener;
+import org.apache.geode.stats.common.cache.client.internal.ConnectionStats;
+import org.apache.geode.stats.common.distributed.internal.PoolStatHelper;
+import org.apache.geode.stats.common.internal.cache.PoolStats;
+import org.apache.geode.stats.common.statistics.factory.StatsFactory;
 import org.apache.geode.test.dunit.NetworkUtils;
 import org.apache.geode.test.junit.categories.ClientServerTest;
 
@@ -106,9 +108,10 @@ public class AutoConnectionSourceImplJUnitTest {
     props.setProperty(MCAST_PORT, "0");
     props.setProperty(LOCATORS, "");
 
-    DistributedSystem ds = DistributedSystem.connect(props);
-    cache = CacheFactory.create(ds);
-    poolStats = new PoolStats(ds, "pool");
+    DistributedSystem distributedSystem = DistributedSystem.connect(props);
+    cache = CacheFactory.create(distributedSystem);
+    poolStats =
+        StatsFactory.createStatsImpl(PoolStats.class, "pool");
     port = AvailablePortHelper.getRandomAvailableTCPPort();
 
     handler = new FakeHandler();
@@ -216,7 +219,6 @@ public class AutoConnectionSourceImplJUnitTest {
     la.add(new HostAddress(floc2, floc2.getHostName()));
     AutoConnectionSourceImpl src = new AutoConnectionSourceImpl(locators, la, "", 60 * 1000);
 
-
     InetSocketAddress b1 = new InetSocketAddress("fakeLocalHost1", port);
     InetSocketAddress b2 = new InetSocketAddress("fakeLocalHost3", port);
 
@@ -224,7 +226,6 @@ public class AutoConnectionSourceImplJUnitTest {
     bla.add(new HostAddress(b1, b1.getHostName()));
     bla.add(new HostAddress(b2, b2.getHostName()));
 
-
     src.addbadLocators(la, bla);
 
     System.out.println("new locatores " + la);
@@ -320,7 +321,6 @@ public class AutoConnectionSourceImplJUnitTest {
 
   /**
    * This tests that discovery works even after one of two locators was shut down
-   *
    */
   @Test
   public void test_DiscoverLocators_whenOneLocatorWasShutdown() throws Exception {
diff --git a/geode-core/src/integrationTest/java/org/apache/geode/cache/client/internal/QueueManagerJUnitTest.java b/geode-core/src/integrationTest/java/org/apache/geode/cache/client/internal/QueueManagerJUnitTest.java
index 1d444cb..4d7cc3e 100644
--- a/geode-core/src/integrationTest/java/org/apache/geode/cache/client/internal/QueueManagerJUnitTest.java
+++ b/geode-core/src/integrationTest/java/org/apache/geode/cache/client/internal/QueueManagerJUnitTest.java
@@ -50,12 +50,14 @@ import org.apache.geode.cache.query.QueryService;
 import org.apache.geode.distributed.DistributedSystem;
 import org.apache.geode.distributed.internal.ServerLocation;
 import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
-import org.apache.geode.internal.cache.PoolStats;
 import org.apache.geode.internal.cache.tier.sockets.ClientProxyMembershipID;
 import org.apache.geode.internal.cache.tier.sockets.ServerQueueStatus;
 import org.apache.geode.internal.logging.InternalLogWriter;
 import org.apache.geode.internal.logging.LocalLogWriter;
 import org.apache.geode.internal.util.StopWatch;
+import org.apache.geode.stats.common.cache.client.internal.ConnectionStats;
+import org.apache.geode.stats.common.internal.cache.PoolStats;
+import org.apache.geode.stats.common.statistics.factory.StatsFactory;
 import org.apache.geode.test.junit.categories.ClientServerTest;
 
 @Category({ClientServerTest.class})
@@ -68,7 +70,7 @@ public class QueueManagerJUnitTest {
 
   private DummyPool pool;
   private LocalLogWriter logger;
-  private DistributedSystem ds;
+  private DistributedSystem distributedSystem;
   private EndpointManagerImpl endpoints;
   private DummySource source;
   private DummyFactory factory;
@@ -82,10 +84,11 @@ public class QueueManagerJUnitTest {
     Properties properties = new Properties();
     properties.put(MCAST_PORT, "0");
     properties.put(LOCATORS, "");
-    ds = DistributedSystem.connect(properties);
-    stats = new PoolStats(ds, "QueueManagerJUnitTest");
+    distributedSystem = DistributedSystem.connect(properties);
+    stats = StatsFactory.createStatsImpl(PoolStats.class, "QueueManagerJUnitTest");
     pool = new DummyPool();
-    endpoints = new EndpointManagerImpl("pool", ds, ds.getCancelCriterion(), pool.getStats());
+    endpoints = new EndpointManagerImpl("pool", distributedSystem,
+        distributedSystem.getCancelCriterion(), pool.getStats());
     source = new DummySource();
     factory = new DummyFactory();
     background = Executors.newSingleThreadScheduledExecutor();
@@ -93,8 +96,8 @@ public class QueueManagerJUnitTest {
         "<ExpectedException action=add>" + expectedPrimaryErrorMsg + "</ExpectedException>";
     final String addExpectedREM =
         "<ExpectedException action=add>" + expectedRedundantErrorMsg + "</ExpectedException>";
-    ds.getLogWriter().info(addExpectedPEM);
-    ds.getLogWriter().info(addExpectedREM);
+    distributedSystem.getLogWriter().info(addExpectedPEM);
+    distributedSystem.getLogWriter().info(addExpectedREM);
   }
 
   @After
@@ -107,10 +110,10 @@ public class QueueManagerJUnitTest {
     final String removeExpectedREM =
         "<ExpectedException action=remove>" + expectedRedundantErrorMsg + "</ExpectedException>";
 
-    ds.getLogWriter().info(removeExpectedPEM);
-    ds.getLogWriter().info(removeExpectedREM);
+    distributedSystem.getLogWriter().info(removeExpectedPEM);
+    distributedSystem.getLogWriter().info(removeExpectedREM);
 
-    ds.disconnect();
+    distributedSystem.disconnect();
   }
 
   @Test
@@ -119,7 +122,7 @@ public class QueueManagerJUnitTest {
     factory.addConnection(0, 0, 2);
     factory.addConnection(0, 0, 3);
     manager = new QueueManagerImpl(pool, endpoints, source, factory, 2, 2000, logger,
-        ClientProxyMembershipID.getNewProxyMembership(ds));
+        ClientProxyMembershipID.getNewProxyMembership(distributedSystem));
     manager.start(background);
     assertPortEquals(1, manager.getAllConnections().getPrimary());
     assertPortEquals(new int[] {2, 3}, manager.getAllConnections().getBackups());
@@ -131,7 +134,7 @@ public class QueueManagerJUnitTest {
     factory.addConnection(1, 23, 2);
     factory.addConnection(1, 11, 3);
     manager = new QueueManagerImpl(pool, endpoints, source, factory, 2, 2000, logger,
-        ClientProxyMembershipID.getNewProxyMembership(ds));
+        ClientProxyMembershipID.getNewProxyMembership(distributedSystem));
     manager.start(background);
     assertPortEquals(2, manager.getAllConnections().getPrimary());
     assertPortEquals(new int[] {3, 1}, manager.getAllConnections().getBackups());
@@ -148,7 +151,7 @@ public class QueueManagerJUnitTest {
     factory.addError();
     factory.addConnection(0, 0, 3);
     manager = new QueueManagerImpl(pool, endpoints, source, factory, 3, 2000, logger,
-        ClientProxyMembershipID.getNewProxyMembership(ds));
+        ClientProxyMembershipID.getNewProxyMembership(distributedSystem));
     manager.start(background);
 
     // The primary queue can be set before we try to fill in for all of the failed backup servers,
@@ -181,7 +184,7 @@ public class QueueManagerJUnitTest {
     factory.addConnection(0, 0, 2);
     factory.addConnection(0, 0, 3);
     manager = new QueueManagerImpl(pool, endpoints, source, factory, 3, 2000, logger,
-        ClientProxyMembershipID.getNewProxyMembership(ds));
+        ClientProxyMembershipID.getNewProxyMembership(distributedSystem));
     manager.start(background);
 
     // wait for backups to come online.
@@ -210,7 +213,7 @@ public class QueueManagerJUnitTest {
     factory.addConnection(0, 0, 4);
     factory.addConnection(0, 0, 5);
     manager = new QueueManagerImpl(pool, endpoints, source, factory, 3, 2000, logger,
-        ClientProxyMembershipID.getNewProxyMembership(ds));
+        ClientProxyMembershipID.getNewProxyMembership(distributedSystem));
     manager.start(background);
     assertPortEquals(1, manager.getAllConnections().getPrimary());
     assertPortEquals(new int[] {2, 3, 4}, manager.getAllConnections().getBackups());
@@ -228,7 +231,7 @@ public class QueueManagerJUnitTest {
     factory.addConnection(0, 0, 1);
     factory.addConnection(0, 0, 2);
     manager = new QueueManagerImpl(pool, endpoints, source, factory, 2, 20, logger,
-        ClientProxyMembershipID.getNewProxyMembership(ds));
+        ClientProxyMembershipID.getNewProxyMembership(distributedSystem));
     manager.start(background);
     assertPortEquals(1, manager.getAllConnections().getPrimary());
     assertPortEquals(new int[] {2}, manager.getAllConnections().getBackups());
@@ -253,7 +256,7 @@ public class QueueManagerJUnitTest {
   public void testWaitForPrimary() throws Exception {
     factory.addConnection(0, 0, 1);
     manager = new QueueManagerImpl(pool, endpoints, source, factory, 2, 20, logger,
-        ClientProxyMembershipID.getNewProxyMembership(ds));
+        ClientProxyMembershipID.getNewProxyMembership(distributedSystem));
     manager.start(background);
     manager.getAllConnections().getPrimary().destroy();
 
diff --git a/geode-core/src/integrationTest/java/org/apache/geode/cache/client/internal/pooling/ConnectionManagerJUnitTest.java b/geode-core/src/integrationTest/java/org/apache/geode/cache/client/internal/pooling/ConnectionManagerJUnitTest.java
index 37858fc..cca031c 100644
--- a/geode-core/src/integrationTest/java/org/apache/geode/cache/client/internal/pooling/ConnectionManagerJUnitTest.java
+++ b/geode-core/src/integrationTest/java/org/apache/geode/cache/client/internal/pooling/ConnectionManagerJUnitTest.java
@@ -46,7 +46,6 @@ import org.apache.geode.cache.client.NoAvailableServersException;
 import org.apache.geode.cache.client.internal.ClientUpdater;
 import org.apache.geode.cache.client.internal.Connection;
 import org.apache.geode.cache.client.internal.ConnectionFactory;
-import org.apache.geode.cache.client.internal.ConnectionStats;
 import org.apache.geode.cache.client.internal.Endpoint;
 import org.apache.geode.cache.client.internal.EndpointManager;
 import org.apache.geode.cache.client.internal.EndpointManagerImpl;
@@ -57,10 +56,12 @@ import org.apache.geode.distributed.DistributedMember;
 import org.apache.geode.distributed.DistributedSystem;
 import org.apache.geode.distributed.internal.ServerLocation;
 import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
-import org.apache.geode.internal.cache.PoolStats;
 import org.apache.geode.internal.cache.tier.sockets.ServerQueueStatus;
 import org.apache.geode.internal.logging.InternalLogWriter;
 import org.apache.geode.internal.logging.LocalLogWriter;
+import org.apache.geode.stats.common.cache.client.internal.ConnectionStats;
+import org.apache.geode.stats.common.internal.cache.PoolStats;
+import org.apache.geode.stats.common.statistics.factory.StatsFactory;
 import org.apache.geode.test.dunit.ThreadUtils;
 import org.apache.geode.test.dunit.Wait;
 import org.apache.geode.test.dunit.WaitCriterion;
@@ -76,7 +77,7 @@ public class ConnectionManagerJUnitTest {
   ConnectionManager manager;
   private InternalLogWriter logger;
   protected DummyFactory factory;
-  private DistributedSystem ds;
+  private DistributedSystem distributedSystem;
   private ScheduledExecutorService background;
   protected EndpointManager endpointManager;
   private CancelCriterion cancelCriterion;
@@ -90,10 +91,12 @@ public class ConnectionManagerJUnitTest {
     Properties properties = new Properties();
     properties.put(MCAST_PORT, "0");
     properties.put(LOCATORS, "");
-    ds = DistributedSystem.connect(properties);
+    distributedSystem = DistributedSystem.connect(properties);
     background = Executors.newSingleThreadScheduledExecutor();
-    poolStats = new PoolStats(ds, "connectionManagerJUnitTest");
-    endpointManager = new EndpointManagerImpl("pool", ds, ds.getCancelCriterion(), poolStats);
+    poolStats =
+        StatsFactory.createStatsImpl(PoolStats.class, "connectionManagerJUnitTest");
+    endpointManager = new EndpointManagerImpl("pool", distributedSystem,
+        distributedSystem.getCancelCriterion(), poolStats);
     cancelCriterion = new CancelCriterion() {
 
       public String cancelInProgress() {
@@ -108,7 +111,7 @@ public class ConnectionManagerJUnitTest {
 
   @After
   public void tearDown() throws InterruptedException {
-    ds.disconnect();
+    distributedSystem.disconnect();
     if (manager != null) {
       manager.close(false);
     }
@@ -152,7 +155,7 @@ public class ConnectionManagerJUnitTest {
     Assert.assertEquals(3, factory.creates);
 
     try {
-      conn[4] = manager.borrowConnection(10);
+      conn[3] = manager.borrowConnection(10);
       fail("Should have received an all connections in use exception");
     } catch (AllConnectionsInUseException e) {
       // expected exception
diff --git a/geode-core/src/integrationTest/java/org/apache/geode/cache/query/QueryJUnitTest.java b/geode-core/src/integrationTest/java/org/apache/geode/cache/query/QueryJUnitTest.java
index 523e1a9..4d47661 100644
--- a/geode-core/src/integrationTest/java/org/apache/geode/cache/query/QueryJUnitTest.java
+++ b/geode-core/src/integrationTest/java/org/apache/geode/cache/query/QueryJUnitTest.java
@@ -52,6 +52,7 @@ import org.apache.geode.cache.RegionShortcut;
 import org.apache.geode.cache.query.data.Portfolio;
 import org.apache.geode.cache.query.internal.DefaultQuery;
 import org.apache.geode.cache.query.internal.index.IndexProtocol;
+import org.apache.geode.stats.common.cache.query.QueryStatistics;
 import org.apache.geode.test.junit.categories.OQLQueryTest;
 
 @FixMethodOrder(NAME_ASCENDING)
diff --git a/geode-core/src/integrationTest/java/org/apache/geode/cache/query/functional/IteratorTypeDefaultTypesJUnitTest.java b/geode-core/src/integrationTest/java/org/apache/geode/cache/query/functional/IteratorTypeDefaultTypesJUnitTest.java
index 846286b..096a32d 100755
--- a/geode-core/src/integrationTest/java/org/apache/geode/cache/query/functional/IteratorTypeDefaultTypesJUnitTest.java
+++ b/geode-core/src/integrationTest/java/org/apache/geode/cache/query/functional/IteratorTypeDefaultTypesJUnitTest.java
@@ -135,9 +135,9 @@ public class IteratorTypeDefaultTypesJUnitTest {
   public void testIteratorDefBoolean() throws Exception {
     boolean b1 = true;
     boolean b2 = false;
-    HashSet C1 = new HashSet();
-    C1.add(new Boolean(b1));
-    C1.add(new Boolean(b2));
+    HashSet<Boolean> C1 = new HashSet<>();
+    C1.add(b1);
+    C1.add(b2);
     Object params[] = new Object[1];
     params[0] = C1;
     String queries[] = {"SELECT DISTINCT booleanValue from (set<boolean>) $1",
diff --git a/geode-core/src/integrationTest/java/org/apache/geode/cache30/DiskRegionIntegrationTest.java b/geode-core/src/integrationTest/java/org/apache/geode/cache30/DiskRegionIntegrationTest.java
index 24df0d6..73fa5ff 100644
--- a/geode-core/src/integrationTest/java/org/apache/geode/cache30/DiskRegionIntegrationTest.java
+++ b/geode-core/src/integrationTest/java/org/apache/geode/cache30/DiskRegionIntegrationTest.java
@@ -50,11 +50,11 @@ import org.apache.geode.cache.RegionShortcut;
 import org.apache.geode.cache.Scope;
 import org.apache.geode.internal.cache.CachedDeserializable;
 import org.apache.geode.internal.cache.DiskRegion;
-import org.apache.geode.internal.cache.DiskRegionStats;
 import org.apache.geode.internal.cache.InternalRegion;
 import org.apache.geode.internal.cache.LocalRegion;
 import org.apache.geode.internal.cache.eviction.CountLRUEviction;
 import org.apache.geode.internal.cache.eviction.EvictionCounters;
+import org.apache.geode.stats.common.internal.cache.DiskRegionStats;
 
 /**
  * Tests the functionality of cache regions whose contents may be written to disk.
diff --git a/geode-core/src/integrationTest/java/org/apache/geode/distributed/internal/membership/MembershipJUnitTest.java b/geode-core/src/integrationTest/java/org/apache/geode/distributed/internal/membership/MembershipJUnitTest.java
index e6eb9ba..97f2475 100755
--- a/geode-core/src/integrationTest/java/org/apache/geode/distributed/internal/membership/MembershipJUnitTest.java
+++ b/geode-core/src/integrationTest/java/org/apache/geode/distributed/internal/membership/MembershipJUnitTest.java
@@ -41,7 +41,6 @@ import org.apache.geode.GemFireConfigException;
 import org.apache.geode.distributed.ConfigurationProperties;
 import org.apache.geode.distributed.Locator;
 import org.apache.geode.distributed.internal.ClusterDistributionManager;
-import org.apache.geode.distributed.internal.DMStats;
 import org.apache.geode.distributed.internal.DistributionConfig;
 import org.apache.geode.distributed.internal.DistributionConfigImpl;
 import org.apache.geode.distributed.internal.InternalLocator;
@@ -65,6 +64,7 @@ import org.apache.geode.internal.AvailablePortHelper;
 import org.apache.geode.internal.admin.remote.RemoteTransportConfig;
 import org.apache.geode.internal.net.SocketCreator;
 import org.apache.geode.internal.security.SecurityServiceFactory;
+import org.apache.geode.stats.common.distributed.internal.DMStats;
 
 @Category({MembershipJUnitTest.class})
 public class MembershipJUnitTest {
diff --git a/geode-core/src/integrationTest/java/org/apache/geode/distributed/internal/membership/gms/fd/GMSHealthMonitorJUnitTest.java b/geode-core/src/integrationTest/java/org/apache/geode/distributed/internal/membership/gms/fd/GMSHealthMonitorJUnitTest.java
index 6f4b86f..4752443 100644
--- a/geode-core/src/integrationTest/java/org/apache/geode/distributed/internal/membership/gms/fd/GMSHealthMonitorJUnitTest.java
+++ b/geode-core/src/integrationTest/java/org/apache/geode/distributed/internal/membership/gms/fd/GMSHealthMonitorJUnitTest.java
@@ -70,7 +70,6 @@ import org.apache.geode.distributed.internal.ClusterDistributionManager;
 import org.apache.geode.distributed.internal.DistributionConfig;
 import org.apache.geode.distributed.internal.DistributionConfigImpl;
 import org.apache.geode.distributed.internal.DistributionManager;
-import org.apache.geode.distributed.internal.DistributionStats;
 import org.apache.geode.distributed.internal.InternalDistributedSystem;
 import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
 import org.apache.geode.distributed.internal.membership.NetView;
@@ -91,6 +90,8 @@ import org.apache.geode.internal.HeapDataOutputStream;
 import org.apache.geode.internal.Version;
 import org.apache.geode.internal.net.SocketCreator;
 import org.apache.geode.internal.net.SocketCreatorFactory;
+import org.apache.geode.stats.common.distributed.internal.DistributionStats;
+import org.apache.geode.stats.common.statistics.factory.StatsFactory;
 import org.apache.geode.test.junit.categories.MembershipTest;
 
 @Category({MembershipTest.class})
@@ -145,7 +146,8 @@ public class GMSHealthMonitorJUnitTest {
     when(services.getJoinLeave()).thenReturn(joinLeave);
     when(services.getCancelCriterion()).thenReturn(stopper);
     when(services.getManager()).thenReturn(manager);
-    when(services.getStatistics()).thenReturn(new DistributionStats(system, statsId));
+    when(services.getStatistics()).thenReturn(StatsFactory
+        .createStatsImpl(DistributionStats.class, String.valueOf(statsId)));
     when(services.getTimer()).thenReturn(new Timer("Geode Membership Timer", true));
     when(stopper.isCancelInProgress()).thenReturn(false);
 
@@ -373,7 +375,7 @@ public class GMSHealthMonitorJUnitTest {
     recipient.add(mockMembers.get(0));
     ArrayList<SuspectRequest> as = new ArrayList<SuspectRequest>();
     SuspectRequest sr = new SuspectRequest(mockMembers.get(1), "Not Responding");// removing member
-                                                                                 // 1
+    // 1
     as.add(sr);
     SuspectMembersMessage sm = new SuspectMembersMessage(recipient, as);
     sm.setSender(mockMembers.get(0));
@@ -407,7 +409,7 @@ public class GMSHealthMonitorJUnitTest {
     recipient.add(mockMembers.get(0));
     ArrayList<SuspectRequest> as = new ArrayList<SuspectRequest>();
     SuspectRequest sr = new SuspectRequest(mockMembers.get(1), "Not Responding");// removing member
-                                                                                 // 1
+    // 1
     as.add(sr);
     SuspectMembersMessage sm = new SuspectMembersMessage(recipient, as);
     sm.setSender(mockMembers.get(0));
@@ -447,7 +449,7 @@ public class GMSHealthMonitorJUnitTest {
     recipient.add(mockMembers.get(1));
     ArrayList<SuspectRequest> as = new ArrayList<SuspectRequest>();
     SuspectRequest sr = new SuspectRequest(mockMembers.get(0), "Not Responding");// removing
-                                                                                 // coordinator
+    // coordinator
     as.add(sr);
     SuspectMembersMessage sm = new SuspectMembersMessage(recipient, as);
     sm.setSender(mockMembers.get(myAddressIndex + 1));// member 4 sends suspect message
@@ -610,7 +612,6 @@ public class GMSHealthMonitorJUnitTest {
   }
 
 
-
   @Test
   public void testInitiatorRewatchesSuspectAfterSuccessfulFinalCheck() {
     NetView v = installAView();
@@ -727,7 +728,6 @@ public class GMSHealthMonitorJUnitTest {
     testMember.setVmViewId(viewId);
     gmsMember.setBirthViewId(viewId);
 
-
     // Set up the incoming/received bytes. We just wrap output streams and write out the gms member
     // information
     byte[] receivedBytes = writeMemberToBytes(otherMember);
@@ -829,7 +829,8 @@ public class GMSHealthMonitorJUnitTest {
   }
 
   private InternalDistributedMember createInternalDistributedMember(short version, int viewId,
-      long msb, long lsb) throws UnknownHostException {
+      long msb, long lsb)
+      throws UnknownHostException {
     GMSMember gmsMember = createGMSMember(version, viewId, msb, lsb);
     InternalDistributedMember idm =
         new InternalDistributedMember("localhost", 9000, Version.CURRENT, gmsMember);
diff --git a/geode-core/src/integrationTest/java/org/apache/geode/distributed/internal/membership/gms/locator/GMSLocatorRecoveryJUnitTest.java b/geode-core/src/integrationTest/java/org/apache/geode/distributed/internal/membership/gms/locator/GMSLocatorRecoveryJUnitTest.java
index e7154c1..e8529e1 100644
--- a/geode-core/src/integrationTest/java/org/apache/geode/distributed/internal/membership/gms/locator/GMSLocatorRecoveryJUnitTest.java
+++ b/geode-core/src/integrationTest/java/org/apache/geode/distributed/internal/membership/gms/locator/GMSLocatorRecoveryJUnitTest.java
@@ -40,10 +40,8 @@ import org.apache.geode.DataSerializer;
 import org.apache.geode.InternalGemFireException;
 import org.apache.geode.distributed.Locator;
 import org.apache.geode.distributed.internal.ClusterDistributionManager;
-import org.apache.geode.distributed.internal.DMStats;
 import org.apache.geode.distributed.internal.DistributionConfigImpl;
 import org.apache.geode.distributed.internal.InternalLocator;
-import org.apache.geode.distributed.internal.LocatorStats;
 import org.apache.geode.distributed.internal.membership.DistributedMembershipListener;
 import org.apache.geode.distributed.internal.membership.MemberFactory;
 import org.apache.geode.distributed.internal.membership.MembershipManager;
@@ -53,6 +51,9 @@ import org.apache.geode.internal.Version;
 import org.apache.geode.internal.admin.remote.RemoteTransportConfig;
 import org.apache.geode.internal.net.SocketCreator;
 import org.apache.geode.internal.security.SecurityServiceFactory;
+import org.apache.geode.stats.common.distributed.internal.DMStats;
+import org.apache.geode.stats.common.distributed.internal.LocatorStats;
+import org.apache.geode.stats.common.statistics.factory.StatsFactory;
 import org.apache.geode.test.junit.categories.MembershipTest;
 
 @Category({MembershipTest.class})
@@ -67,7 +68,9 @@ public class GMSLocatorRecoveryJUnitTest {
     if (this.tempStateFile.exists()) {
       this.tempStateFile.delete();
     }
-    this.locator = new GMSLocator(null, null, false, false, new LocatorStats(), "");
+    this.locator = new GMSLocator(null, null, false, false,
+        StatsFactory.createStatsImpl(LocatorStats.class, "testLocator"),
+        "");
     locator.setViewFile(tempStateFile);
     // System.out.println("temp state file: " + tempStateFile);
   }
@@ -177,7 +180,9 @@ public class GMSLocatorRecoveryJUnitTest {
       ((InternalLocator) l).getLocatorHandler().setMembershipManager(m1);
 
       GMSLocator l2 = new GMSLocator(SocketCreator.getLocalHost(),
-          m1.getLocalMember().getHost() + "[" + port + "]", true, true, new LocatorStats(), "");
+          m1.getLocalMember().getHost() + "[" + port + "]", true, true,
+          StatsFactory.createStatsImpl(LocatorStats.class, "testLocator"),
+          "");
       l2.setViewFile(new File("l2.dat"));
       l2.init(null);
 
diff --git a/geode-core/src/integrationTest/java/org/apache/geode/distributed/internal/membership/gms/messenger/JGroupsMessengerJUnitTest.java b/geode-core/src/integrationTest/java/org/apache/geode/distributed/internal/membership/gms/messenger/JGroupsMessengerJUnitTest.java
index 37cbf9e..7eda10f 100755
--- a/geode-core/src/integrationTest/java/org/apache/geode/distributed/internal/membership/gms/messenger/JGroupsMessengerJUnitTest.java
+++ b/geode-core/src/integrationTest/java/org/apache/geode/distributed/internal/membership/gms/messenger/JGroupsMessengerJUnitTest.java
@@ -75,7 +75,7 @@ import org.apache.geode.distributed.internal.ClusterDistributionManager;
 import org.apache.geode.distributed.internal.DistributionConfigImpl;
 import org.apache.geode.distributed.internal.DistributionManager;
 import org.apache.geode.distributed.internal.DistributionMessage;
-import org.apache.geode.distributed.internal.DistributionStats;
+import org.apache.geode.distributed.internal.DistributionStatsImpl;
 import org.apache.geode.distributed.internal.InternalDistributedSystem;
 import org.apache.geode.distributed.internal.SerialAckedMessage;
 import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
@@ -103,6 +103,8 @@ import org.apache.geode.internal.Version;
 import org.apache.geode.internal.admin.remote.RemoteTransportConfig;
 import org.apache.geode.internal.cache.DistributedCacheOperation;
 import org.apache.geode.internal.logging.log4j.AlertAppender;
+import org.apache.geode.stats.common.distributed.internal.DistributionStats;
+import org.apache.geode.stats.common.statistics.factory.StatsFactory;
 import org.apache.geode.test.junit.categories.MembershipTest;
 
 @Category({MembershipTest.class})
@@ -167,7 +169,8 @@ public class JGroupsMessengerJUnitTest {
     DistributionManager dm = mock(DistributionManager.class);
     InternalDistributedSystem system =
         InternalDistributedSystem.newInstanceForTesting(dm, nonDefault);
-    when(services.getStatistics()).thenReturn(new DistributionStats(system, statsId));
+    when(services.getStatistics()).thenReturn(StatsFactory
+        .createStatsImpl(DistributionStats.class, String.valueOf(statsId)));
 
     messenger = new JGroupsMessenger();
     messenger.init(services);
@@ -472,7 +475,6 @@ public class JGroupsMessengerJUnitTest {
     NetView v = new NetView(addr);
     when(joinLeave.getView()).thenReturn(v);
 
-
     InternalDistributedMember sender = createAddress(8888);
 
     JoinRequestMessage msg = new JoinRequestMessage(messenger.localAddress, sender, null, -1, 0);
@@ -559,8 +561,9 @@ public class JGroupsMessengerJUnitTest {
     for (Message m : messages) {
       m.setSrc(fakeMember);
       UNICAST3.Header oldHeader = (UNICAST3.Header) m.getHeader(unicastHeaderId);
-      if (oldHeader == null)
+      if (oldHeader == null) {
         continue;
+      }
       UNICAST3.Header newHeader =
           UNICAST3.Header.createDataHeader(seqno, oldHeader.connId(), seqno == 1);
       seqno += 1;
@@ -833,7 +836,7 @@ public class JGroupsMessengerJUnitTest {
   @Test
   public void testReceiver() throws Exception {
     try {
-      DistributionStats.enableClockStats = true;
+      DistributionStatsImpl.enableClockStats = true;
       initMocks(false);
       JGroupsReceiver receiver = (JGroupsReceiver) messenger.myChannel.getReceiver();
 
@@ -872,7 +875,7 @@ public class JGroupsMessengerJUnitTest {
       assertTrue("There should be UDPDispatchRequestTime stats",
           services.getStatistics().getUDPDispatchRequestTime() > 0);
     } finally {
-      DistributionStats.enableClockStats = false;
+      DistributionStatsImpl.enableClockStats = false;
     }
   }
 
diff --git a/geode-core/src/integrationTest/java/org/apache/geode/distributed/internal/membership/gms/mgr/GMSMembershipManagerJUnitTest.java b/geode-core/src/integrationTest/java/org/apache/geode/distributed/internal/membership/gms/mgr/GMSMembershipManagerJUnitTest.java
index 5aa4b46..4a3d5b7 100644
--- a/geode-core/src/integrationTest/java/org/apache/geode/distributed/internal/membership/gms/mgr/GMSMembershipManagerJUnitTest.java
+++ b/geode-core/src/integrationTest/java/org/apache/geode/distributed/internal/membership/gms/mgr/GMSMembershipManagerJUnitTest.java
@@ -57,7 +57,6 @@ import org.junit.experimental.categories.Category;
 
 import org.apache.geode.distributed.DistributedSystemDisconnectedException;
 import org.apache.geode.distributed.internal.ClusterDistributionManager;
-import org.apache.geode.distributed.internal.DMStats;
 import org.apache.geode.distributed.internal.DistributionConfig;
 import org.apache.geode.distributed.internal.DistributionConfigImpl;
 import org.apache.geode.distributed.internal.DistributionManager;
@@ -83,6 +82,7 @@ import org.apache.geode.distributed.internal.membership.gms.mgr.GMSMembershipMan
 import org.apache.geode.internal.admin.remote.AlertListenerMessage;
 import org.apache.geode.internal.admin.remote.RemoteTransportConfig;
 import org.apache.geode.internal.tcp.ConnectExceptions;
+import org.apache.geode.stats.common.distributed.internal.DMStats;
 import org.apache.geode.test.junit.categories.MembershipTest;
 
 @Category({MembershipTest.class})
diff --git a/geode-core/src/integrationTest/java/org/apache/geode/distributed/internal/tcpserver/TCPClientSSLIntegrationTest.java b/geode-core/src/integrationTest/java/org/apache/geode/distributed/internal/tcpserver/TCPClientSSLIntegrationTest.java
index dc55a23..420edce 100644
--- a/geode-core/src/integrationTest/java/org/apache/geode/distributed/internal/tcpserver/TCPClientSSLIntegrationTest.java
+++ b/geode-core/src/integrationTest/java/org/apache/geode/distributed/internal/tcpserver/TCPClientSSLIntegrationTest.java
@@ -39,13 +39,13 @@ import org.apache.geode.cache.ssl.CertStores;
 import org.apache.geode.cache.ssl.TestSSLUtils.CertificateBuilder;
 import org.apache.geode.distributed.internal.DistributionConfig;
 import org.apache.geode.distributed.internal.DistributionConfigImpl;
-import org.apache.geode.distributed.internal.PoolStatHelper;
 import org.apache.geode.internal.AvailablePort;
 import org.apache.geode.internal.admin.SSLConfig;
 import org.apache.geode.internal.net.SSLConfigurationFactory;
 import org.apache.geode.internal.net.SocketCreator;
 import org.apache.geode.internal.net.SocketCreatorFactory;
 import org.apache.geode.internal.security.SecurableCommunicationChannel;
+import org.apache.geode.stats.common.distributed.internal.PoolStatHelper;
 import org.apache.geode.test.junit.categories.MembershipTest;
 
 @Category({MembershipTest.class})
diff --git a/geode-core/src/integrationTest/java/org/apache/geode/distributed/internal/tcpserver/TCPServerSSLJUnitTest.java b/geode-core/src/integrationTest/java/org/apache/geode/distributed/internal/tcpserver/TCPServerSSLJUnitTest.java
index 06e30da..60b6720 100644
--- a/geode-core/src/integrationTest/java/org/apache/geode/distributed/internal/tcpserver/TCPServerSSLJUnitTest.java
+++ b/geode-core/src/integrationTest/java/org/apache/geode/distributed/internal/tcpserver/TCPServerSSLJUnitTest.java
@@ -40,7 +40,6 @@ import org.mockito.Mockito;
 
 import org.apache.geode.distributed.internal.DistributionConfig;
 import org.apache.geode.distributed.internal.DistributionConfigImpl;
-import org.apache.geode.distributed.internal.PoolStatHelper;
 import org.apache.geode.internal.AvailablePort;
 import org.apache.geode.internal.admin.SSLConfig;
 import org.apache.geode.internal.net.DummySocketCreator;
@@ -48,6 +47,7 @@ import org.apache.geode.internal.net.SSLConfigurationFactory;
 import org.apache.geode.internal.net.SocketCreator;
 import org.apache.geode.internal.net.SocketCreatorFactory;
 import org.apache.geode.internal.security.SecurableCommunicationChannel;
+import org.apache.geode.stats.common.distributed.internal.PoolStatHelper;
 import org.apache.geode.test.junit.categories.MembershipTest;
 import org.apache.geode.util.test.TestUtil;
 
diff --git a/geode-core/src/integrationTest/java/org/apache/geode/distributed/internal/tcpserver/TcpServerJUnitTest.java b/geode-core/src/integrationTest/java/org/apache/geode/distributed/internal/tcpserver/TcpServerJUnitTest.java
index 71a1eba..b88a7c6 100644
--- a/geode-core/src/integrationTest/java/org/apache/geode/distributed/internal/tcpserver/TcpServerJUnitTest.java
+++ b/geode-core/src/integrationTest/java/org/apache/geode/distributed/internal/tcpserver/TcpServerJUnitTest.java
@@ -40,10 +40,10 @@ import org.apache.geode.cache.GemFireCache;
 import org.apache.geode.distributed.DistributedSystem;
 import org.apache.geode.distributed.internal.DistributionConfigImpl;
 import org.apache.geode.distributed.internal.InternalConfigurationPersistenceService;
-import org.apache.geode.distributed.internal.PoolStatHelper;
 import org.apache.geode.internal.AvailablePort;
 import org.apache.geode.internal.cache.tier.sockets.TcpServerFactory;
 import org.apache.geode.internal.net.SocketCreatorFactory;
+import org.apache.geode.stats.common.distributed.internal.PoolStatHelper;
 import org.apache.geode.test.junit.categories.MembershipTest;
 
 @Category({MembershipTest.class})
diff --git a/geode-core/src/integrationTest/java/org/apache/geode/internal/cache/ComplexDiskRegionJUnitTest.java b/geode-core/src/integrationTest/java/org/apache/geode/internal/cache/ComplexDiskRegionJUnitTest.java
index 9ff29a0..d982c7f 100755
--- a/geode-core/src/integrationTest/java/org/apache/geode/internal/cache/ComplexDiskRegionJUnitTest.java
+++ b/geode-core/src/integrationTest/java/org/apache/geode/internal/cache/ComplexDiskRegionJUnitTest.java
@@ -19,7 +19,6 @@ import static org.junit.Assert.fail;
 
 import org.junit.Test;
 
-import org.apache.geode.StatisticsFactory;
 import org.apache.geode.cache.Scope;
 
 /**
@@ -52,10 +51,9 @@ public class ComplexDiskRegionJUnitTest extends DiskRegionTestingBase {
     diskProps.setAllowForceCompaction(true);
     region = DiskRegionHelperFactory.getSyncPersistOnlyRegion(cache, diskProps, Scope.LOCAL);
     DiskRegion dr = ((LocalRegion) region).getDiskRegion();
-    StatisticsFactory factory = region.getCache().getDistributedSystem();
-    Oplog oplog1 = new Oplog(11, dr.getOplogSet(), new DirectoryHolder(factory, dirs[1], 1000, 0));
-    Oplog oplog2 = new Oplog(12, dr.getOplogSet(), new DirectoryHolder(factory, dirs[2], 1000, 1));
-    Oplog oplog3 = new Oplog(13, dr.getOplogSet(), new DirectoryHolder(factory, dirs[3], 1000, 2));
+    Oplog oplog1 = new Oplog(11, dr.getOplogSet(), new DirectoryHolder(dirs[1], 1000, 0));
+    Oplog oplog2 = new Oplog(12, dr.getOplogSet(), new DirectoryHolder(dirs[2], 1000, 1));
+    Oplog oplog3 = new Oplog(13, dr.getOplogSet(), new DirectoryHolder(dirs[3], 1000, 2));
     // give these Oplogs some fake "live" entries
     oplog1.incTotalCount();
     oplog1.incLiveCount();
@@ -109,10 +107,9 @@ public class ComplexDiskRegionJUnitTest extends DiskRegionTestingBase {
     diskProps.setRolling(false);
     region = DiskRegionHelperFactory.getSyncPersistOnlyRegion(cache, diskProps, Scope.LOCAL);
     DiskRegion dr = ((LocalRegion) region).getDiskRegion();
-    StatisticsFactory factory = region.getCache().getDistributedSystem();
-    Oplog oplog1 = new Oplog(11, dr.getOplogSet(), new DirectoryHolder(factory, dirs[1], 1000, 0));
-    Oplog oplog2 = new Oplog(12, dr.getOplogSet(), new DirectoryHolder(factory, dirs[2], 1000, 1));
-    Oplog oplog3 = new Oplog(13, dr.getOplogSet(), new DirectoryHolder(factory, dirs[3], 1000, 2));
+    Oplog oplog1 = new Oplog(11, dr.getOplogSet(), new DirectoryHolder(dirs[1], 1000, 0));
+    Oplog oplog2 = new Oplog(12, dr.getOplogSet(), new DirectoryHolder(dirs[2], 1000, 1));
+    Oplog oplog3 = new Oplog(13, dr.getOplogSet(), new DirectoryHolder(dirs[3], 1000, 2));
     // give these Oplogs some fake "live" entries
     oplog1.incTotalCount();
     oplog1.incLiveCount();
diff --git a/geode-core/src/integrationTest/java/org/apache/geode/internal/cache/DiskInitFileJUnitTest.java b/geode-core/src/integrationTest/java/org/apache/geode/internal/cache/DiskInitFileJUnitTest.java
index 927616d..e761679 100644
--- a/geode-core/src/integrationTest/java/org/apache/geode/internal/cache/DiskInitFileJUnitTest.java
+++ b/geode-core/src/integrationTest/java/org/apache/geode/internal/cache/DiskInitFileJUnitTest.java
@@ -15,8 +15,6 @@
 package org.apache.geode.internal.cache;
 
 import static org.assertj.core.api.Assertions.assertThat;
-import static org.mockito.ArgumentMatchers.any;
-import static org.mockito.ArgumentMatchers.anyString;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.spy;
 import static org.mockito.Mockito.when;
@@ -31,8 +29,6 @@ import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.TemporaryFolder;
 
-import org.apache.geode.Statistics;
-import org.apache.geode.StatisticsFactory;
 import org.apache.geode.internal.cache.persistence.DiskRegionView;
 import org.apache.geode.internal.cache.persistence.DiskStoreID;
 
@@ -47,14 +43,9 @@ public class DiskInitFileJUnitTest {
   public void setUp() throws Exception {
     File testDirectory = temporaryFolder.newFolder("_" + getClass().getSimpleName());
 
-    // Mock statistics factory for creating directory holders.
-    final StatisticsFactory mockStatisticsFactory = mock(StatisticsFactory.class);
-    when(mockStatisticsFactory.createStatistics(any(), anyString()))
-        .thenReturn(mock(Statistics.class));
-
     // Mock disk store impl. All we need to do is return this init file directory.
     mockedDiskStoreImpl = mock(DiskStoreImpl.class);
-    DirectoryHolder holder = new DirectoryHolder(mockStatisticsFactory, testDirectory, 0, 0);
+    DirectoryHolder holder = new DirectoryHolder(testDirectory, 0, 0);
     when(mockedDiskStoreImpl.getInfoFileDir()).thenReturn(holder);
     when(mockedDiskStoreImpl.getDiskStoreID()).thenReturn(mock(DiskStoreID.class));
     when(mockedDiskStoreImpl.getBackupLock()).thenReturn(mock(ReentrantLock.class));
diff --git a/geode-core/src/integrationTest/java/org/apache/geode/internal/cache/DiskRegOplogSwtchingAndRollerJUnitTest.java b/geode-core/src/integrationTest/java/org/apache/geode/internal/cache/DiskRegOplogSwtchingAndRollerJUnitTest.java
index 612d94e..546f69e 100644
--- a/geode-core/src/integrationTest/java/org/apache/geode/internal/cache/DiskRegOplogSwtchingAndRollerJUnitTest.java
+++ b/geode-core/src/integrationTest/java/org/apache/geode/internal/cache/DiskRegOplogSwtchingAndRollerJUnitTest.java
@@ -30,6 +30,7 @@ import org.apache.geode.cache.EntryNotFoundException;
 import org.apache.geode.cache.Region;
 import org.apache.geode.cache.Scope;
 import org.apache.geode.internal.cache.eviction.EvictionCounters;
+import org.apache.geode.stats.common.internal.cache.DiskRegionStats;
 
 /**
  * Miscellaneous disk tests
diff --git a/geode-core/src/integrationTest/java/org/apache/geode/internal/cache/DiskRegionJUnitTest.java b/geode-core/src/integrationTest/java/org/apache/geode/internal/cache/DiskRegionJUnitTest.java
index 95acb04..3591e07 100755
--- a/geode-core/src/integrationTest/java/org/apache/geode/internal/cache/DiskRegionJUnitTest.java
+++ b/geode-core/src/integrationTest/java/org/apache/geode/internal/cache/DiskRegionJUnitTest.java
@@ -71,6 +71,8 @@ import org.apache.geode.internal.cache.entries.DiskEntry;
 import org.apache.geode.internal.cache.eviction.EvictionCounters;
 import org.apache.geode.internal.cache.persistence.DiskRecoveryStore;
 import org.apache.geode.internal.cache.persistence.UninterruptibleFileChannel;
+import org.apache.geode.stats.common.internal.cache.CachePerfStats;
+import org.apache.geode.stats.common.internal.cache.DiskRegionStats;
 import org.apache.geode.test.dunit.IgnoredException;
 import org.apache.geode.test.dunit.Wait;
 import org.apache.geode.test.junit.rules.ExecutorServiceRule;
diff --git a/geode-core/src/integrationTest/java/org/apache/geode/internal/cache/DiskStoreImplIntegrationTest.java b/geode-core/src/integrationTest/java/org/apache/geode/internal/cache/DiskStoreImplIntegrationTest.java
index 7303449..ba23678 100644
--- a/geode-core/src/integrationTest/java/org/apache/geode/internal/cache/DiskStoreImplIntegrationTest.java
+++ b/geode-core/src/integrationTest/java/org/apache/geode/internal/cache/DiskStoreImplIntegrationTest.java
@@ -39,6 +39,7 @@ import org.apache.geode.cache.RegionFactory;
 import org.apache.geode.cache.RegionShortcut;
 import org.apache.geode.distributed.ConfigurationProperties;
 import org.apache.geode.internal.cache.backup.BackupService;
+import org.apache.geode.stats.common.internal.cache.DiskStoreStats;
 
 
 public class DiskStoreImplIntegrationTest {
diff --git a/geode-core/src/integrationTest/java/org/apache/geode/internal/cache/LIFOEvictionAlgoEnabledRegionJUnitTest.java b/geode-core/src/integrationTest/java/org/apache/geode/internal/cache/LIFOEvictionAlgoEnabledRegionJUnitTest.java
index b4a804d..0bf7121 100755
--- a/geode-core/src/integrationTest/java/org/apache/geode/internal/cache/LIFOEvictionAlgoEnabledRegionJUnitTest.java
+++ b/geode-core/src/integrationTest/java/org/apache/geode/internal/cache/LIFOEvictionAlgoEnabledRegionJUnitTest.java
@@ -44,6 +44,7 @@ import org.apache.geode.cache.Scope;
 import org.apache.geode.distributed.DistributedSystem;
 import org.apache.geode.internal.cache.eviction.EvictionCounters;
 import org.apache.geode.internal.cache.eviction.EvictionList;
+import org.apache.geode.stats.common.internal.cache.DiskRegionStats;
 
 /**
  * This is a test verifies region is LIFO enabled by ENTRY COUNT verifies correct stats updating and
diff --git a/geode-core/src/integrationTest/java/org/apache/geode/internal/cache/LIFOEvictionAlgoMemoryEnabledRegionJUnitTest.java b/geode-core/src/integrationTest/java/org/apache/geode/internal/cache/LIFOEvictionAlgoMemoryEnabledRegionJUnitTest.java
index ffc927b..25b28e5 100755
--- a/geode-core/src/integrationTest/java/org/apache/geode/internal/cache/LIFOEvictionAlgoMemoryEnabledRegionJUnitTest.java
+++ b/geode-core/src/integrationTest/java/org/apache/geode/internal/cache/LIFOEvictionAlgoMemoryEnabledRegionJUnitTest.java
@@ -63,6 +63,7 @@ import org.apache.geode.internal.cache.persistence.DiskRecoveryStore;
 import org.apache.geode.internal.cache.versions.VersionSource;
 import org.apache.geode.internal.cache.versions.VersionStamp;
 import org.apache.geode.internal.cache.versions.VersionTag;
+import org.apache.geode.stats.common.internal.cache.DiskRegionStats;
 
 /**
  * This is a test verifies region is LIFO enabled by MEMORY verifies correct stats updating and
diff --git a/geode-core/src/integrationTest/java/org/apache/geode/internal/cache/OplogJUnitTest.java b/geode-core/src/integrationTest/java/org/apache/geode/internal/cache/OplogJUnitTest.java
index 9d2b9d2..8499c5f 100755
--- a/geode-core/src/integrationTest/java/org/apache/geode/internal/cache/OplogJUnitTest.java
+++ b/geode-core/src/integrationTest/java/org/apache/geode/internal/cache/OplogJUnitTest.java
@@ -41,7 +41,6 @@ import org.junit.Assert;
 import org.junit.Ignore;
 import org.junit.Test;
 
-import org.apache.geode.StatisticsFactory;
 import org.apache.geode.cache.CacheWriterException;
 import org.apache.geode.cache.DiskAccessException;
 import org.apache.geode.cache.DiskStore;
@@ -56,6 +55,8 @@ import org.apache.geode.internal.InternalDataSerializer;
 import org.apache.geode.internal.cache.Oplog.OPLOG_TYPE;
 import org.apache.geode.internal.cache.entries.AbstractDiskLRURegionEntry;
 import org.apache.geode.internal.cache.entries.DiskEntry;
+import org.apache.geode.stats.common.internal.cache.DiskStoreStats;
+import org.apache.geode.stats.common.statistics.StatisticsFactory;
 import org.apache.geode.test.dunit.ThreadUtils;
 
 /**
@@ -63,10 +64,8 @@ import org.apache.geode.test.dunit.ThreadUtils;
  */
 public class OplogJUnitTest extends DiskRegionTestingBase {
 
-  private boolean proceed = false;
-
   private final DiskRegionProperties diskProps = new DiskRegionProperties();
-
+  private boolean proceed = false;
   private long delta;
 
   private volatile boolean assertDone = false;
@@ -193,9 +192,9 @@ public class OplogJUnitTest extends DiskRegionTestingBase {
       long id = oplog.getOplogId();
       oplog.close();
 
-      StatisticsFactory factory = cache.getDistributedSystem();
+      StatisticsFactory factory = cache.getDistributedSystem().getStatisticsFactory();
       Oplog newOplog =
-          new Oplog(id, dr.getOplogSet(), new DirectoryHolder(factory, dirs[0], 1000, 0));
+          new Oplog(id, dr.getOplogSet(), new DirectoryHolder(dirs[0], 1000, 0));
       dr.getOplogSet().setChild(newOplog);
       closeDown();
     }
@@ -214,9 +213,9 @@ public class OplogJUnitTest extends DiskRegionTestingBase {
       Oplog oplog = dr.testHook_getChild();
       long id = oplog.getOplogId();
       oplog.close();
-      StatisticsFactory factory = cache.getDistributedSystem();
+      StatisticsFactory factory = cache.getDistributedSystem().getStatisticsFactory();
       Oplog newOplog =
-          new Oplog(id, dr.getOplogSet(), new DirectoryHolder(factory, dirs[0], 1000, 2));
+          new Oplog(id, dr.getOplogSet(), new DirectoryHolder(dirs[0], 1000, 2));
       dr.setChild(newOplog);
       closeDown();
     }
@@ -338,7 +337,6 @@ public class OplogJUnitTest extends DiskRegionTestingBase {
 
   /**
    * Tests whether the data is written in the right format on the disk
-   *
    */
   @Test
   public void testFaultInOfValuesFromDisk() {
@@ -388,7 +386,6 @@ public class OplogJUnitTest extends DiskRegionTestingBase {
 
   /**
    * Tests the original ByteBufferPool gets transferred to the new Oplog for synch mode
-   *
    */
   @Test
   public void testByteBufferPoolTransferForSynchMode() {
@@ -428,7 +425,6 @@ public class OplogJUnitTest extends DiskRegionTestingBase {
    * Tests the bug which arises in case of asynch mode during oplog switching caused by conflation
    * of create/destroy operation.The bug occurs if a create operation is followed by destroy but
    * before destroy proceeds some other operation causes oplog switching
-   *
    */
   @Test
   public void testBug34615() {
@@ -543,7 +539,6 @@ public class OplogJUnitTest extends DiskRegionTestingBase {
 
   /**
    * This tests the retrieval of empty byte array when present in asynch buffers
-   *
    */
   @Test
   public void testGetEmptyByteArrayInAsynchBuffer() {
@@ -571,7 +566,6 @@ public class OplogJUnitTest extends DiskRegionTestingBase {
 
   /**
    * This tests the retrieval of empty byte array in synch mode
-   *
    */
   @Test
   public void testGetEmptyByteArrayInSynchMode() {
@@ -602,7 +596,6 @@ public class OplogJUnitTest extends DiskRegionTestingBase {
    * is Token.Removed This bug can occur if a remove operation causes oplog switching & hence roller
    * thread gets notified, & the roller thread obtains the iterator of the concurrent region map
    * before the remove
-   *
    */
   @Test
   public void testBug34702() {
@@ -689,7 +682,6 @@ public class OplogJUnitTest extends DiskRegionTestingBase {
    * such case, when the roller will try to roll the entries referencing the current oplog , it will
    * not be able to acquire the lock on the entry as the switching thread has already taken a lock
    * on it.
-   *
    */
   @Test
   public void testRollingDeadlockSituation() {
@@ -720,7 +712,6 @@ public class OplogJUnitTest extends DiskRegionTestingBase {
    * operation & hence the 4 bytes field for recording the value length is absent & also since the
    * value length is zero no byte for it should also get added. Similary during recover from HTree
    * as well as Oplog , the empty byte array should be read correctly
-   *
    */
   @Test
   public void testEmptyByteArrayPutAndRecovery() {
@@ -786,7 +777,6 @@ public class OplogJUnitTest extends DiskRegionTestingBase {
    * switching of Oplog if it happens just after the remove operation has destroyed the in memory
    * entry & is about to acquire the readlock in DiskRegion to record the same. If the Oplog has
    * switched during that duration , the bug would appear
-   *
    */
 
   @Test
@@ -831,7 +821,6 @@ public class OplogJUnitTest extends DiskRegionTestingBase {
   /**
    * Tests if buffer size & time are not set , the asynch writer gets awakened on time basis of
    * default 1 second
-   *
    */
   @Test
   public void testAsynchWriterAttribBehaviour1() throws Exception {
@@ -935,7 +924,6 @@ public class OplogJUnitTest extends DiskRegionTestingBase {
   /**
    * Tests if buffer size & time interval are explicitly set to zero then the flush will occur due
    * to asynchForceFlush or due to switching of Oplog
-   *
    */
   @Test
   public void testAsynchWriterAttribBehaviour3() throws Exception {
@@ -977,7 +965,6 @@ public class OplogJUnitTest extends DiskRegionTestingBase {
     Thread.sleep(1000);
     assertTrue(LocalRegion.ISSUE_CALLBACKS_TO_CACHE_OBSERVER);
 
-
     ((LocalRegion) region).getDiskRegion().forceRolling();
     synchronized (this) {
       if (LocalRegion.ISSUE_CALLBACKS_TO_CACHE_OBSERVER) {
@@ -991,7 +978,6 @@ public class OplogJUnitTest extends DiskRegionTestingBase {
 
   /**
    * Tests if the byte buffer pool in asynch mode tries to contain the pool size
-   *
    */
   @Test
   public void testByteBufferPoolContainment() throws Exception {
@@ -1070,7 +1056,6 @@ public class OplogJUnitTest extends DiskRegionTestingBase {
 
   /**
    * Tests delayed creation of DiskID in overflow only mode
-   *
    */
   @Test
   public void testDelayedDiskIdCreationInOverflowOnlyMode() {
@@ -1097,7 +1082,6 @@ public class OplogJUnitTest extends DiskRegionTestingBase {
 
   /**
    * Tests immediate creation of DiskID in overflow With Persistence mode
-   *
    */
   @Test
   public void testImmediateDiskIdCreationInOverflowWithPersistMode() {
@@ -1123,8 +1107,8 @@ public class OplogJUnitTest extends DiskRegionTestingBase {
   }
 
   /**
-   * An entry which is evicted to disk will have the flag already written to disk, appropriately set
-   *
+   * An entry which is evicted to disk will have the flag already written to disk, appropriately
+   * set
    */
   @Test
   public void testEntryAlreadyWrittenIsCorrectlyUnmarkedForOverflowOnly() throws Exception {
@@ -1184,7 +1168,6 @@ public class OplogJUnitTest extends DiskRegionTestingBase {
   /**
    * An persistent or overflow with persistence entry which is evicted to disk, will have the flag
    * already written to disk, appropriately set
-   *
    */
   @Test
   public void testEntryAlreadyWrittenIsCorrectlyUnmarkedForOverflowWithPersistence() {
@@ -1382,7 +1365,6 @@ public class OplogJUnitTest extends DiskRegionTestingBase {
    * re-'put'), the entry will actually become a create in the VM The new Oplog should record it as
    * a create even though the Htree ref in ThreadLocal will not match with the current Htree Ref.
    * But the operation is valid & should get recorded in Oplog
-   *
    */
   @Test
   public void testPutClearCreate() throws Exception {
@@ -1941,10 +1923,10 @@ public class OplogJUnitTest extends DiskRegionTestingBase {
     private final int tombstoneKey1;
     private final int tombstoneKey2;
     private final AtomicBoolean exceptionOccurred = new AtomicBoolean(true);
-    private volatile long spaceUsageBefore = -1;
-    private DirectoryHolder dh;
     private final AtomicLong oplogsSize = new AtomicLong();
     private final CountDownLatch putsCompleted;
+    private volatile long spaceUsageBefore = -1;
+    private DirectoryHolder dh;
 
     StatSizeTestCacheObserverAdapter(DiskRegion dr, int key3Size, int tombstoneKey1,
         int tombstoneKey2, CountDownLatch putsCompleted) {
diff --git a/geode-core/src/integrationTest/java/org/apache/geode/internal/cache/OplogRVVJUnitTest.java b/geode-core/src/integrationTest/java/org/apache/geode/internal/cache/OplogRVVJUnitTest.java
index 1d6baaf..320bf29 100644
--- a/geode-core/src/integrationTest/java/org/apache/geode/internal/cache/OplogRVVJUnitTest.java
+++ b/geode-core/src/integrationTest/java/org/apache/geode/internal/cache/OplogRVVJUnitTest.java
@@ -38,19 +38,20 @@ import org.junit.Test;
 import org.junit.rules.TemporaryFolder;
 
 import org.apache.geode.CancelCriterion;
-import org.apache.geode.Statistics;
-import org.apache.geode.StatisticsFactory;
 import org.apache.geode.internal.cache.DiskInitFile.DiskRegionFlag;
 import org.apache.geode.internal.cache.DiskStoreImpl.OplogEntryIdSet;
 import org.apache.geode.internal.cache.persistence.DiskRecoveryStore;
 import org.apache.geode.internal.cache.persistence.DiskStoreID;
 import org.apache.geode.internal.cache.versions.DiskRegionVersionVector;
+import org.apache.geode.stats.common.internal.cache.DiskStoreStats;
+import org.apache.geode.stats.common.statistics.Statistics;
+import org.apache.geode.stats.common.statistics.StatisticsFactory;
+import org.apache.geode.stats.common.statistics.factory.StatsFactory;
 
 public class OplogRVVJUnitTest {
-  private File testDirectory;
-
   @Rule
   public TemporaryFolder temporaryFolder = new TemporaryFolder();
+  private File testDirectory;
 
   @Before
   public void setUp() throws Exception {
@@ -84,12 +85,12 @@ public class OplogRVVJUnitTest {
     when(sf.createStatistics(any(), anyString())).thenReturn(mock(Statistics.class));
     when(sf.createAtomicStatistics(any(), anyString())).thenReturn(mock(Statistics.class));
 
-    DirectoryHolder dirHolder = new DirectoryHolder(sf, testDirectory, 0, 0);
+    DirectoryHolder dirHolder = new DirectoryHolder(testDirectory, 0, 0);
     when(cache.cacheTimeMillis()).thenReturn(System.currentTimeMillis());
     when(parent.getCache()).thenReturn(cache);
     when(parent.getMaxOplogSizeInBytes()).thenReturn(10000L);
     when(parent.getName()).thenReturn("test");
-    DiskStoreStats diskStoreStats = new DiskStoreStats(sf, "stats");
+    DiskStoreStats diskStoreStats = StatsFactory.createStatsImpl(DiskStoreStats.class, "stats");
     when(parent.getStats()).thenReturn(diskStoreStats);
     when(parent.getDiskInitFile()).thenReturn(df);
     when(parent.getDiskStoreID()).thenReturn(DiskStoreID.random());
diff --git a/geode-core/src/integrationTest/java/org/apache/geode/internal/cache/PartitionedRegionStatsJUnitTest.java b/geode-core/src/integrationTest/java/org/apache/geode/internal/cache/PartitionedRegionStatsJUnitTest.java
index d0d59b8..27c192d 100644
--- a/geode-core/src/integrationTest/java/org/apache/geode/internal/cache/PartitionedRegionStatsJUnitTest.java
+++ b/geode-core/src/integrationTest/java/org/apache/geode/internal/cache/PartitionedRegionStatsJUnitTest.java
@@ -34,7 +34,6 @@ import org.junit.Before;
 import org.junit.Test;
 
 import org.apache.geode.LogWriter;
-import org.apache.geode.Statistics;
 import org.apache.geode.cache.AttributesFactory;
 import org.apache.geode.cache.Cache;
 import org.apache.geode.cache.DataPolicy;
@@ -43,6 +42,8 @@ import org.apache.geode.cache.EvictionAttributes;
 import org.apache.geode.cache.PartitionAttributesFactory;
 import org.apache.geode.cache.PartitionedRegionStorageException;
 import org.apache.geode.cache.RegionExistsException;
+import org.apache.geode.stats.common.internal.cache.DiskRegionStats;
+import org.apache.geode.stats.common.statistics.Statistics;
 
 public class PartitionedRegionStatsJUnitTest {
   private static final File DISK_DIR = new File("PRStatsTest");
diff --git a/geode-core/src/integrationTest/java/org/apache/geode/internal/cache/SimpleDiskRegionJUnitTest.java b/geode-core/src/integrationTest/java/org/apache/geode/internal/cache/SimpleDiskRegionJUnitTest.java
index 7a1b326..77279cb 100755
--- a/geode-core/src/integrationTest/java/org/apache/geode/internal/cache/SimpleDiskRegionJUnitTest.java
+++ b/geode-core/src/integrationTest/java/org/apache/geode/internal/cache/SimpleDiskRegionJUnitTest.java
@@ -30,7 +30,7 @@ import java.util.Set;
 
 import org.junit.Test;
 
-import org.apache.geode.StatisticsFactory;
+import org.apache.geode.stats.common.statistics.StatisticsFactory;
 import org.apache.geode.test.dunit.ThreadUtils;
 
 /**
@@ -240,9 +240,9 @@ public class SimpleDiskRegionJUnitTest extends DiskRegionTestingBase {
     Oplog oplog = dr.testHook_getChild();
     long id = oplog.getOplogId();
 
-    StatisticsFactory factory = region.getCache().getDistributedSystem();
+    StatisticsFactory factory = region.getCache().getDistributedSystem().getStatisticsFactory();
     Oplog newOplog =
-        new Oplog(id, dr.getOplogSet(), new DirectoryHolder(factory, dirs[0], 1000000, 0));
+        new Oplog(id, dr.getOplogSet(), new DirectoryHolder(dirs[0], 1000000, 0));
     dr.getDiskStore().getPersistentOplogs().setChild(newOplog);
     assertEquals(newOplog, dr.testHook_getChild());
     dr.setChild(oplog);
diff --git a/geode-core/src/integrationTest/java/org/apache/geode/internal/cache/control/MemoryMonitorOffHeapJUnitTest.java b/geode-core/src/integrationTest/java/org/apache/geode/internal/cache/control/MemoryMonitorOffHeapJUnitTest.java
index c0f6330..d653e75 100644
--- a/geode-core/src/integrationTest/java/org/apache/geode/internal/cache/control/MemoryMonitorOffHeapJUnitTest.java
+++ b/geode-core/src/integrationTest/java/org/apache/geode/internal/cache/control/MemoryMonitorOffHeapJUnitTest.java
@@ -39,6 +39,7 @@ import org.apache.geode.internal.cache.GemFireCacheImpl;
 import org.apache.geode.internal.cache.control.InternalResourceManager.ResourceType;
 import org.apache.geode.internal.i18n.LocalizedStrings;
 import org.apache.geode.internal.logging.LogService;
+import org.apache.geode.stats.common.internal.cache.control.ResourceManagerStats;
 
 public class MemoryMonitorOffHeapJUnitTest {
   private static final Logger logger = LogService.getLogger();
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 0270177..3b6ef90 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
@@ -55,8 +55,6 @@ import org.powermock.modules.junit4.PowerMockRunner;
 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;
@@ -84,7 +82,6 @@ import org.apache.geode.internal.cache.InternalRegionArguments;
 import org.apache.geode.internal.cache.LocalRegion;
 import org.apache.geode.internal.cache.VMCachedDeserializable;
 import org.apache.geode.internal.cache.tier.sockets.CacheClientNotifier;
-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.ClientUpdateMessage;
 import org.apache.geode.internal.cache.tier.sockets.ClientUpdateMessageImpl;
@@ -93,6 +90,10 @@ import org.apache.geode.internal.cache.tier.sockets.HAEventWrapper;
 import org.apache.geode.internal.concurrent.ConcurrentHashSet;
 import org.apache.geode.internal.util.BlobHelper;
 import org.apache.geode.internal.util.concurrent.StoppableReentrantReadWriteLock;
+import org.apache.geode.stats.common.internal.cache.ha.HARegionQueueStats;
+import org.apache.geode.stats.common.internal.cache.tier.sockets.CacheServerStats;
+import org.apache.geode.stats.common.statistics.Statistics;
+import org.apache.geode.stats.common.statistics.StatisticsType;
 
 @RunWith(PowerMockRunner.class)
 @PowerMockIgnore({"javax.script.*", "javax.management.*", "org.springframework.shell.event.*",
@@ -151,7 +152,9 @@ public class HARegionQueueIntegrationTest {
 
     doReturn(Mockito.mock(InternalDistributedMember.class)).when(mockInternalDistributedSystem)
         .getDistributedMember();
-    doReturn(Mockito.mock(Statistics.class)).when(mockInternalDistributedSystem)
+    doReturn(Mockito.mock(Statistics.class))
+        .when(mockInternalDistributedSystem.getInternalDistributedSystemStats()
+            .getStatisticsFactory())
         .createAtomicStatistics(any(StatisticsType.class), any(String.class));
     doReturn(Mockito.mock(DistributionConfig.class)).when(mockDistributionManager).getConfig();
     doReturn(mockDistributionManager).when(mockInternalDistributedSystem).getDistributionManager();
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..26deae6 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
@@ -33,6 +33,7 @@ import org.apache.geode.cache.Scope;
 import org.apache.geode.internal.cache.Conflatable;
 import org.apache.geode.internal.cache.EventID;
 import org.apache.geode.internal.cache.InternalCache;
+import org.apache.geode.stats.common.internal.cache.ha.HARegionQueueStats;
 import org.apache.geode.test.junit.categories.ClientSubscriptionTest;
 
 /**
diff --git a/geode-core/src/integrationTest/java/org/apache/geode/internal/cache/tier/sockets/CacheClientNotifierIntegrationTest.java b/geode-core/src/integrationTest/java/org/apache/geode/internal/cache/tier/sockets/CacheClientNotifierIntegrationTest.java
index d65dd6e..52d3e47 100644
--- a/geode-core/src/integrationTest/java/org/apache/geode/internal/cache/tier/sockets/CacheClientNotifierIntegrationTest.java
+++ b/geode-core/src/integrationTest/java/org/apache/geode/internal/cache/tier/sockets/CacheClientNotifierIntegrationTest.java
@@ -52,8 +52,6 @@ import org.powermock.modules.junit4.PowerMockRunner;
 
 import org.apache.geode.CancelCriterion;
 import org.apache.geode.DataSerializer;
-import org.apache.geode.Statistics;
-import org.apache.geode.StatisticsType;
 import org.apache.geode.cache.AttributesMutator;
 import org.apache.geode.cache.Operation;
 import org.apache.geode.cache.RegionAttributes;
@@ -76,6 +74,9 @@ import org.apache.geode.internal.cache.InternalCache;
 import org.apache.geode.internal.cache.InternalRegionArguments;
 import org.apache.geode.internal.cache.LocalRegion;
 import org.apache.geode.internal.security.SecurityService;
+import org.apache.geode.stats.common.internal.cache.tier.sockets.CacheServerStats;
+import org.apache.geode.stats.common.statistics.Statistics;
+import org.apache.geode.stats.common.statistics.StatisticsType;
 import org.apache.geode.test.junit.categories.ClientSubscriptionTest;
 
 @Category({ClientSubscriptionTest.class})
@@ -267,7 +268,9 @@ public class CacheClientNotifierIntegrationTest {
 
     doReturn(mock(InternalDistributedMember.class)).when(mockInternalDistributedSystem)
         .getDistributedMember();
-    doReturn(mock(Statistics.class)).when(mockInternalDistributedSystem)
+    doReturn(mock(Statistics.class))
+        .when(mockInternalDistributedSystem.getInternalDistributedSystemStats())
+        .getStatisticsFactory()
         .createAtomicStatistics(any(StatisticsType.class), any(String.class));
     doReturn(mock(DistributionConfig.class)).when(mockDistributionManager).getConfig();
     doReturn(mockDistributionManager).when(mockInternalDistributedSystem).getDistributionManager();
diff --git a/geode-core/src/integrationTest/java/org/apache/geode/internal/cache/tier/sockets/CacheServerMaxConnectionsJUnitTest.java b/geode-core/src/integrationTest/java/org/apache/geode/internal/cache/tier/sockets/CacheServerMaxConnectionsJUnitTest.java
index bff62c5..f0a196b 100644
--- a/geode-core/src/integrationTest/java/org/apache/geode/internal/cache/tier/sockets/CacheServerMaxConnectionsJUnitTest.java
+++ b/geode-core/src/integrationTest/java/org/apache/geode/internal/cache/tier/sockets/CacheServerMaxConnectionsJUnitTest.java
@@ -26,8 +26,6 @@ import org.junit.After;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
-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;
@@ -41,6 +39,8 @@ import org.apache.geode.cache.client.internal.PoolImpl;
 import org.apache.geode.cache.server.CacheServer;
 import org.apache.geode.distributed.DistributedSystem;
 import org.apache.geode.internal.AvailablePort;
+import org.apache.geode.stats.common.statistics.Statistics;
+import org.apache.geode.stats.common.statistics.StatisticsType;
 import org.apache.geode.test.dunit.Wait;
 import org.apache.geode.test.dunit.WaitCriterion;
 import org.apache.geode.test.junit.categories.ClientServerTest;
@@ -137,8 +137,8 @@ public class CacheServerMaxConnectionsJUnitTest {
   public void testMaxCnxLimit() throws Exception {
     PORT = createServer();
     createProxyAndRegionForClient();
-    StatisticsType st = this.system.findType("CacheServerStats");
-    final Statistics s = this.system.findStatisticsByType(st)[0];
+    StatisticsType st = this.system.getStatisticsFactory().findType("CacheServerStats");
+    final Statistics s = this.system.getStatisticsFactory().findStatisticsByType(st)[0];
     assertEquals(0, s.getInt("currentClients"));
     assertEquals(0, s.getInt("currentClientConnections"));
     Connection[] cnxs = new Connection[MAX_CNXS];
diff --git a/geode-core/src/integrationTest/java/org/apache/geode/internal/cache/tier/sockets/ClientHealthMonitorIntegrationTest.java b/geode-core/src/integrationTest/java/org/apache/geode/internal/cache/tier/sockets/ClientHealthMonitorIntegrationTest.java
index 8df649d..34daca0 100755
--- a/geode-core/src/integrationTest/java/org/apache/geode/internal/cache/tier/sockets/ClientHealthMonitorIntegrationTest.java
+++ b/geode-core/src/integrationTest/java/org/apache/geode/internal/cache/tier/sockets/ClientHealthMonitorIntegrationTest.java
@@ -30,8 +30,6 @@ import org.junit.Test;
 import org.junit.contrib.java.lang.system.RestoreSystemProperties;
 import org.junit.experimental.categories.Category;
 
-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;
@@ -47,6 +45,8 @@ import org.apache.geode.distributed.DistributedSystem;
 import org.apache.geode.internal.AvailablePort;
 import org.apache.geode.internal.cache.EventID;
 import org.apache.geode.internal.cache.InternalCache;
+import org.apache.geode.stats.common.statistics.Statistics;
+import org.apache.geode.stats.common.statistics.StatisticsType;
 import org.apache.geode.test.junit.categories.ClientServerTest;
 
 /**
@@ -142,14 +142,14 @@ public class ClientHealthMonitorIntegrationTest {
 
     assertEquals(monitorInterval,
         ClientHealthMonitor
-            .getInstance(mock(InternalCache.class), 0, mock(CacheClientNotifierStats.class))
+            .getInstance(mock(InternalCache.class), 0, mock(CacheClientNotifierStatsImpl.class))
             .getMonitorInterval());
   }
 
   @Test
   public void monitorIntervalDefaultsWhenNotSet() {
     assertNotNull(ClientHealthMonitor
-        .getInstance(mock(InternalCache.class), 0, mock(CacheClientNotifierStats.class))
+        .getInstance(mock(InternalCache.class), 0, mock(CacheClientNotifierStatsImpl.class))
         .getMonitorInterval());
   }
 
@@ -160,7 +160,7 @@ public class ClientHealthMonitorIntegrationTest {
         monitorInterval);
 
     assertNotNull(ClientHealthMonitor
-        .getInstance(mock(InternalCache.class), 0, mock(CacheClientNotifierStats.class))
+        .getInstance(mock(InternalCache.class), 0, mock(CacheClientNotifierStatsImpl.class))
         .getMonitorInterval());
   }
 
@@ -179,8 +179,9 @@ public class ClientHealthMonitorIntegrationTest {
     System.setProperty(ClientHealthMonitor.CLIENT_HEALTH_MONITOR_INTERVAL_PROPERTY, "100");
     PORT = createServer();
     createProxyAndRegionForClient();
-    StatisticsType statisticsType = this.system.findType("CacheServerStats");
-    final Statistics statistics = this.system.findStatisticsByType(statisticsType)[0];
+    StatisticsType statisticsType = this.system.getStatisticsFactory().findType("CacheServerStats");
+    final Statistics statistics =
+        this.system.getStatisticsFactory().findStatisticsByType(statisticsType)[0];
     assertEquals(0, statistics.getInt("currentClients"));
     assertEquals(0, statistics.getInt("currentClientConnections"));
     this.system.getLogWriter()
diff --git a/geode-core/src/integrationTest/java/org/apache/geode/internal/cache/tier/sockets/ServerConnectionIntegrationTest.java b/geode-core/src/integrationTest/java/org/apache/geode/internal/cache/tier/sockets/ServerConnectionIntegrationTest.java
index 7fc34b3..e4cc1fb 100644
--- a/geode-core/src/integrationTest/java/org/apache/geode/internal/cache/tier/sockets/ServerConnectionIntegrationTest.java
+++ b/geode-core/src/integrationTest/java/org/apache/geode/internal/cache/tier/sockets/ServerConnectionIntegrationTest.java
@@ -63,7 +63,7 @@ public class ServerConnectionIntegrationTest {
   private Socket socket;
   private InternalCache cache;
   private SecurityService securityService;
-  private CacheServerStats stats;
+  private CacheServerStatsImpl stats;
 
   @Before
   public void setUp() throws IOException {
@@ -78,7 +78,7 @@ public class ServerConnectionIntegrationTest {
     cache = mock(InternalCache.class);
     securityService = mock(SecurityService.class);
 
-    stats = mock(CacheServerStats.class);
+    stats = mock(CacheServerStatsImpl.class);
   }
 
   class TestMessage extends Message {
@@ -124,7 +124,7 @@ public class ServerConnectionIntegrationTest {
      * client over a given <code>Socket</code>.
      */
     public TestServerConnection(Socket socket, InternalCache internalCache,
-        CachedRegionHelper helper, CacheServerStats stats, int hsTimeout, int socketBufferSize,
+        CachedRegionHelper helper, CacheServerStatsImpl stats, int hsTimeout, int socketBufferSize,
         String communicationModeStr, byte communicationMode, Acceptor acceptor,
         SecurityService securityService) {
       super(socket, internalCache, helper, stats, hsTimeout, socketBufferSize, communicationModeStr,
@@ -178,7 +178,7 @@ public class ServerConnectionIntegrationTest {
   @Test
   public void terminatingConnectionHandlesNewRequestsGracefully() throws Exception {
     when(cache.getCacheTransactionManager()).thenReturn(mock(TXManagerImpl.class));
-    ClientHealthMonitor.createInstance(cache, 100, mock(CacheClientNotifierStats.class));
+    ClientHealthMonitor.createInstance(cache, 100, mock(CacheClientNotifierStatsImpl.class));
     ClientHealthMonitor clientHealthMonitor = ClientHealthMonitor.getInstance();
     when(acceptor.getClientHealthMonitor()).thenReturn(clientHealthMonitor);
     when(acceptor.getConnectionListener()).thenReturn(mock(ConnectionListener.class));
diff --git a/geode-core/src/integrationTest/java/org/apache/geode/internal/offheap/MemoryAllocatorFillPatternIntegrationTest.java b/geode-core/src/integrationTest/java/org/apache/geode/internal/offheap/MemoryAllocatorFillPatternIntegrationTest.java
index 0f8f22d..f27a6f5 100644
--- a/geode-core/src/integrationTest/java/org/apache/geode/internal/offheap/MemoryAllocatorFillPatternIntegrationTest.java
+++ b/geode-core/src/integrationTest/java/org/apache/geode/internal/offheap/MemoryAllocatorFillPatternIntegrationTest.java
@@ -92,7 +92,7 @@ public class MemoryAllocatorFillPatternIntegrationTest {
     System.setProperty(DistributionConfig.GEMFIRE_PREFIX + "validateOffHeapWithFill", "true");
     this.slab = new SlabImpl(SLAB_SIZE);
     this.allocator = MemoryAllocatorImpl.createForUnitTest(new NullOutOfOffHeapMemoryListener(),
-        new NullOffHeapMemoryStats(), new SlabImpl[] {this.slab});
+        new NullOffHeapStorageStats(), new SlabImpl[] {this.slab});
   }
 
   /**
diff --git a/geode-core/src/integrationTest/java/org/apache/geode/internal/statistics/DiskSpaceLimitIntegrationTest.java b/geode-core/src/integrationTest/java/org/apache/geode/internal/statistics/DiskSpaceLimitIntegrationTest.java
index a7b1e5e..9656162 100644
--- a/geode-core/src/integrationTest/java/org/apache/geode/internal/statistics/DiskSpaceLimitIntegrationTest.java
+++ b/geode-core/src/integrationTest/java/org/apache/geode/internal/statistics/DiskSpaceLimitIntegrationTest.java
@@ -39,13 +39,13 @@ import org.junit.experimental.categories.Category;
 import org.junit.rules.TemporaryFolder;
 import org.junit.rules.TestName;
 
-import org.apache.geode.StatisticDescriptor;
-import org.apache.geode.Statistics;
-import org.apache.geode.StatisticsType;
 import org.apache.geode.internal.NanoTimer;
 import org.apache.geode.internal.io.MainWithChildrenRollingFileHandler;
 import org.apache.geode.internal.io.RollingFileHandler;
 import org.apache.geode.internal.util.ArrayUtils;
+import org.apache.geode.stats.common.statistics.StatisticDescriptor;
+import org.apache.geode.stats.common.statistics.Statistics;
+import org.apache.geode.stats.common.statistics.StatisticsType;
 import org.apache.geode.test.junit.categories.StatisticsTest;
 
 /**
@@ -129,9 +129,7 @@ public class DiskSpaceLimitIntegrationTest {
   }
 
   @After
-  public void tearDown() throws Exception {
-    StatisticsTypeFactoryImpl.clear();
-  }
+  public void tearDown() throws Exception {}
 
   @Test
   public void zeroKeepsAllFiles() throws Exception {
diff --git a/geode-core/src/integrationTest/java/org/apache/geode/internal/statistics/DistributedSystemStatisticsIntegrationTest.java b/geode-core/src/integrationTest/java/org/apache/geode/internal/statistics/DistributedSystemStatisticsIntegrationTest.java
index 6747722..07eabf9 100644
--- a/geode-core/src/integrationTest/java/org/apache/geode/internal/statistics/DistributedSystemStatisticsIntegrationTest.java
+++ b/geode-core/src/integrationTest/java/org/apache/geode/internal/statistics/DistributedSystemStatisticsIntegrationTest.java
@@ -30,11 +30,11 @@ import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.junit.rules.TestName;
 
-import org.apache.geode.StatisticDescriptor;
-import org.apache.geode.Statistics;
-import org.apache.geode.StatisticsFactory;
-import org.apache.geode.StatisticsType;
 import org.apache.geode.distributed.DistributedSystem;
+import org.apache.geode.stats.common.statistics.StatisticDescriptor;
+import org.apache.geode.stats.common.statistics.Statistics;
+import org.apache.geode.stats.common.statistics.StatisticsFactory;
+import org.apache.geode.stats.common.statistics.StatisticsType;
 import org.apache.geode.test.junit.categories.StatisticsTest;
 
 /**
@@ -232,7 +232,7 @@ public class DistributedSystemStatisticsIntegrationTest {
   }
 
   private StatisticsFactory factory() {
-    return this.system;
+    return this.system.getStatisticsFactory();
   }
 
   private Statistics setUpIntStatistics(final int count) {
diff --git a/geode-core/src/integrationTest/java/org/apache/geode/internal/statistics/DistributedSystemStatisticsTypeIntegrationTest.java b/geode-core/src/integrationTest/java/org/apache/geode/internal/statistics/DistributedSystemStatisticsTypeIntegrationTest.java
index a3a324f..a5cbb76 100644
--- a/geode-core/src/integrationTest/java/org/apache/geode/internal/statistics/DistributedSystemStatisticsTypeIntegrationTest.java
+++ b/geode-core/src/integrationTest/java/org/apache/geode/internal/statistics/DistributedSystemStatisticsTypeIntegrationTest.java
@@ -29,10 +29,10 @@ import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.junit.rules.TestName;
 
-import org.apache.geode.StatisticDescriptor;
-import org.apache.geode.StatisticsFactory;
-import org.apache.geode.StatisticsType;
 import org.apache.geode.distributed.DistributedSystem;
+import org.apache.geode.stats.common.statistics.StatisticDescriptor;
+import org.apache.geode.stats.common.statistics.StatisticsFactory;
+import org.apache.geode.stats.common.statistics.StatisticsType;
 import org.apache.geode.test.junit.categories.StatisticsTest;
 
 /**
@@ -85,7 +85,7 @@ public class DistributedSystemStatisticsTypeIntegrationTest {
   }
 
   private StatisticsFactory factory() {
-    return this.system;
+    return this.system.getStatisticsFactory();
   }
 
 }
diff --git a/geode-core/src/integrationTest/java/org/apache/geode/internal/statistics/FileSizeLimitIntegrationTest.java b/geode-core/src/integrationTest/java/org/apache/geode/internal/statistics/FileSizeLimitIntegrationTest.java
index be13360..eb3959e 100644
--- a/geode-core/src/integrationTest/java/org/apache/geode/internal/statistics/FileSizeLimitIntegrationTest.java
+++ b/geode-core/src/integrationTest/java/org/apache/geode/internal/statistics/FileSizeLimitIntegrationTest.java
@@ -32,11 +32,11 @@ import org.junit.rules.TestName;
 import org.mockito.invocation.InvocationOnMock;
 import org.mockito.stubbing.Answer;
 
-import org.apache.geode.StatisticDescriptor;
-import org.apache.geode.Statistics;
-import org.apache.geode.StatisticsType;
 import org.apache.geode.internal.NanoTimer;
 import org.apache.geode.internal.io.MainWithChildrenRollingFileHandler;
+import org.apache.geode.stats.common.statistics.StatisticDescriptor;
+import org.apache.geode.stats.common.statistics.Statistics;
+import org.apache.geode.stats.common.statistics.StatisticsType;
 import org.apache.geode.test.junit.categories.StatisticsTest;
 
 @Category({StatisticsTest.class})
@@ -110,9 +110,7 @@ public class FileSizeLimitIntegrationTest {
   }
 
   @After
-  public void tearDown() throws Exception {
-    StatisticsTypeFactoryImpl.clear();
-  }
+  public void tearDown() throws Exception {}
 
   @Test
   public void rollsWhenLimitIsReached() throws Exception { // TODO: add test to assert size is
diff --git a/geode-core/src/integrationTest/java/org/apache/geode/internal/statistics/GemFireStatSamplerIntegrationTest.java b/geode-core/src/integrationTest/java/org/apache/geode/internal/statistics/GemFireStatSamplerIntegrationTest.java
index dbb227f..49bdb50 100644
--- a/geode-core/src/integrationTest/java/org/apache/geode/internal/statistics/GemFireStatSamplerIntegrationTest.java
+++ b/geode-core/src/integrationTest/java/org/apache/geode/internal/statistics/GemFireStatSamplerIntegrationTest.java
@@ -47,8 +47,6 @@ import org.junit.experimental.categories.Category;
 import org.junit.rules.TemporaryFolder;
 import org.junit.rules.TestName;
 
-import org.apache.geode.Statistics;
-import org.apache.geode.StatisticsType;
 import org.apache.geode.distributed.DistributedSystem;
 import org.apache.geode.distributed.internal.InternalDistributedSystem;
 import org.apache.geode.internal.GemFireVersion;
@@ -61,6 +59,8 @@ import org.apache.geode.internal.statistics.platform.OsStatisticsFactory;
 import org.apache.geode.internal.statistics.platform.ProcessStats;
 import org.apache.geode.internal.stats50.VMStats50;
 import org.apache.geode.internal.util.StopWatch;
+import org.apache.geode.stats.common.statistics.Statistics;
+import org.apache.geode.stats.common.statistics.StatisticsType;
 import org.apache.geode.test.junit.categories.StatisticsTest;
 
 /**
@@ -75,7 +75,7 @@ public class GemFireStatSamplerIntegrationTest extends StatSamplerTestCase {
 
   private static final int STAT_SAMPLE_RATE = 1000;
 
-  private DistributedSystem system;
+  private DistributedSystem distributedSystem;
   private File testDir;
 
   @Rule
@@ -128,7 +128,7 @@ public class GemFireStatSamplerIntegrationTest extends StatSamplerTestCase {
     Assert.assertEquals(SocketCreator.getHostName(SocketCreator.getLocalHost()),
         statSampler.getSystemDirectoryPath());
 
-    VMStatsContract vmStats = statSampler.getVMStats();
+    VMStats vmStats = statSampler.getVMStats();
     assertNotNull(vmStats);
     assertTrue(vmStats instanceof VMStats50);
     /*
@@ -374,7 +374,7 @@ public class GemFireStatSamplerIntegrationTest extends StatSamplerTestCase {
     final File archiveFile2 = new File(dirName + File.separator + this.testName + "-01-02.gfs");
     final File archiveFile3 = new File(dirName + File.separator + this.testName + "-01-03.gfs");
 
-    // set the system property to use KB instead of MB for file size
+    // set the distributedSystem property to use KB instead of MB for file size
     System.setProperty(HostStatSampler.TEST_FILE_SIZE_LIMIT_IN_KB_PROPERTY, "true");
     Properties props = createGemFireProperties();
     props.setProperty(ARCHIVE_FILE_SIZE_LIMIT, "1");
@@ -422,7 +422,7 @@ public class GemFireStatSamplerIntegrationTest extends StatSamplerTestCase {
 
     final int sampleRate = 1000;
 
-    // set the system property to use KB instead of MB for file size
+    // set the distributedSystem property to use KB instead of MB for file size
     System.setProperty(HostStatSampler.TEST_FILE_SIZE_LIMIT_IN_KB_PROPERTY, "true");
     Properties props = createGemFireProperties();
     props.setProperty(STATISTIC_ARCHIVE_FILE, archiveFileName);
@@ -478,7 +478,7 @@ public class GemFireStatSamplerIntegrationTest extends StatSamplerTestCase {
     final String tenuredPoolName = HeapMemoryMonitor.getTenuredMemoryPoolMXBean().getName();
     logger.info("TenuredPoolName: {}", tenuredPoolName);
 
-    final List<Statistics> list = ((StatisticsManager) this.system).getStatsList();
+    final List<Statistics> list = ((StatisticsManager) this.distributedSystem).getStatsList();
     assertFalse(list.isEmpty());
 
     boolean done = false;
@@ -532,15 +532,18 @@ public class GemFireStatSamplerIntegrationTest extends StatSamplerTestCase {
 
   @Override
   protected StatisticsManager getStatisticsManager() {
-    return (InternalDistributedSystem) this.system;
+    return ((InternalDistributedSystem) this.distributedSystem).getInternalDistributedSystemStats()
+        .getStatisticsManager();
   }
 
   protected OsStatisticsFactory getOsStatisticsFactory() {
-    return (InternalDistributedSystem) this.system;
+    return ((InternalDistributedSystem) this.distributedSystem).getInternalDistributedSystemStats()
+        .getOSStatisticsFactory();
   }
 
   private GemFireStatSampler getGemFireStatSampler() {
-    return ((InternalDistributedSystem) this.system).getStatSampler();
+    return ((InternalDistributedSystem) this.distributedSystem).getInternalDistributedSystemStats()
+        .getStatSampler();
   }
 
   private SampleCollector getSampleCollector() {
@@ -565,65 +568,14 @@ public class GemFireStatSamplerIntegrationTest extends StatSamplerTestCase {
    */
   @SuppressWarnings("deprecation")
   private void connect(Properties props) {
-    this.system = DistributedSystem.connect(props);
+    this.distributedSystem = DistributedSystem.connect(props);
   }
 
   @SuppressWarnings("deprecation")
   private void disconnect() {
-    if (this.system != null) {
-      this.system.disconnect();
-      this.system = null;
+    if (this.distributedSystem != null) {
+      this.distributedSystem.disconnect();
+      this.distributedSystem = null;
     }
   }
-
-  // public static class AsyncInvoker {
-  // public static AsyncInvocation invokeAsync(Runnable r) {
-  // return invokeAsync(r, "run", new Object[0]);
-  // }
-  // public static AsyncInvocation invokeAsync(Callable c) {
-  // return invokeAsync(c, "call", new Object[0]);
-  // }
-  // public static AsyncInvocation invokeAsync(
-  // final Object o, final String methodName, final Object[] args) {
-  // AsyncInvocation ai =
-  // new AsyncInvocation(o, methodName, new Runnable() {
-  // public void run() {
-  // MethExecutorResult result =
-  // MethExecutor.executeObject(o, methodName, args);
-  // if (result.exceptionOccurred()) {
-  // throw new AsyncInvocationException(result.getException());
-  // }
-  // AsyncInvocation.setReturnValue(result.getResult());
-  // }
-  // });
-  // ai.start();
-  // return ai;
-  // }
-  //
-  // public static class AsyncInvocationException extends RuntimeException {
-  // private static final long serialVersionUID = -5522299018650622945L;
-  // /**
-  // * Creates a new <code>AsyncInvocationException</code>.
-  // */
-  // public AsyncInvocationException(String message) {
-  // super(message);
-  // }
-  //
-  // /**
-  // * Creates a new <code>AsyncInvocationException</code> that was
-  // * caused by a given exception
-  // */
-  // public AsyncInvocationException(String message, Throwable thr) {
-  // super(message, thr);
-  // }
-  //
-  // /**
-  // * Creates a new <code>AsyncInvocationException</code> that was
-  // * caused by a given exception
-  // */
-  // public AsyncInvocationException(Throwable thr) {
-  // super(thr.getMessage(), thr);
-  // }
-  // }
-  // }
 }
diff --git a/geode-core/src/integrationTest/java/org/apache/geode/internal/statistics/LinuxSystemStatsTest.java b/geode-core/src/integrationTest/java/org/apache/geode/internal/statistics/LinuxSystemStatsTest.java
index 62bdc84..dd56740 100644
--- a/geode-core/src/integrationTest/java/org/apache/geode/internal/statistics/LinuxSystemStatsTest.java
+++ b/geode-core/src/integrationTest/java/org/apache/geode/internal/statistics/LinuxSystemStatsTest.java
@@ -1,4 +1,5 @@
 /*
+ * /*
  * 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
@@ -41,9 +42,10 @@ import org.powermock.core.classloader.annotations.PrepareForTest;
 import org.powermock.modules.junit4.PowerMockRunner;
 
 import org.apache.geode.CancelCriterion;
-import org.apache.geode.Statistics;
 import org.apache.geode.internal.statistics.platform.LinuxProcFsStatistics;
 import org.apache.geode.internal.statistics.platform.LinuxSystemStats;
+import org.apache.geode.stats.common.statistics.Statistics;
+import org.apache.geode.stats.common.statistics.StatisticsFactory;
 import org.apache.geode.test.junit.categories.StatisticsTest;
 
 
@@ -76,7 +78,8 @@ public class LinuxSystemStatsTest extends StatSamplerTestCase {
         + File.separator + SimpleStatSampler.DEFAULT_ARCHIVE_FILE_NAME);
     LinuxProcFsStatistics.init();
     initStats();
-    StatisticsTypeImpl statisticsType = (StatisticsTypeImpl) LinuxSystemStats.getType();
+    StatisticsTypeImpl statisticsType = (StatisticsTypeImpl) new LinuxSystemStats(
+        (StatisticsFactory) new StatisticsTypeFactoryImpl()).getType();
     LocalStatisticsImpl statistics = (LocalStatisticsImpl) getStatisticsManager()
         .createStatistics(statisticsType, statisticsType.getName());
 
@@ -87,7 +90,6 @@ public class LinuxSystemStatsTest extends StatSamplerTestCase {
 
   @After
   public void tearDown() throws Exception {
-    StatisticsTypeFactoryImpl.clear();
     if (this.statisticsFactory != null) {
       this.statisticsFactory.close();
     }
diff --git a/geode-core/src/integrationTest/java/org/apache/geode/internal/statistics/SimpleStatSamplerIntegrationTest.java b/geode-core/src/integrationTest/java/org/apache/geode/internal/statistics/SimpleStatSamplerIntegrationTest.java
index b61b1ec..8f2326b 100755
--- a/geode-core/src/integrationTest/java/org/apache/geode/internal/statistics/SimpleStatSamplerIntegrationTest.java
+++ b/geode-core/src/integrationTest/java/org/apache/geode/internal/statistics/SimpleStatSamplerIntegrationTest.java
@@ -34,10 +34,10 @@ import org.junit.rules.TemporaryFolder;
 import org.junit.rules.TestName;
 
 import org.apache.geode.CancelCriterion;
-import org.apache.geode.Statistics;
-import org.apache.geode.StatisticsType;
 import org.apache.geode.internal.net.SocketCreator;
 import org.apache.geode.internal.stats50.VMStats50;
+import org.apache.geode.stats.common.statistics.Statistics;
+import org.apache.geode.stats.common.statistics.StatisticsType;
 import org.apache.geode.test.junit.categories.StatisticsTest;
 
 /**
@@ -104,7 +104,7 @@ public class SimpleStatSamplerIntegrationTest extends StatSamplerTestCase {
     assertEquals(SocketCreator.getHostName(SocketCreator.getLocalHost()),
         statSampler.getSystemDirectoryPath());
 
-    VMStatsContract vmStats = statSampler.getVMStats();
+    VMStats vmStats = statSampler.getVMStats();
     assertNotNull(vmStats);
     assertTrue(vmStats instanceof VMStats50);
     /*
diff --git a/geode-core/src/integrationTest/java/org/apache/geode/internal/statistics/StatArchiveWithConsecutiveResourceInstGenerator.java b/geode-core/src/integrationTest/java/org/apache/geode/internal/statistics/StatArchiveWithConsecutiveResourceInstGenerator.java
index 5312686..17c54ee 100644
--- a/geode-core/src/integrationTest/java/org/apache/geode/internal/statistics/StatArchiveWithConsecutiveResourceInstGenerator.java
+++ b/geode-core/src/integrationTest/java/org/apache/geode/internal/statistics/StatArchiveWithConsecutiveResourceInstGenerator.java
@@ -38,11 +38,11 @@ import org.junit.Test;
 import org.junit.rules.TemporaryFolder;
 import org.junit.rules.TestName;
 
-import org.apache.geode.StatisticDescriptor;
-import org.apache.geode.Statistics;
-import org.apache.geode.StatisticsType;
 import org.apache.geode.internal.statistics.StatArchiveReader.ResourceInst;
 import org.apache.geode.internal.statistics.StatArchiveReader.StatValue;
+import org.apache.geode.stats.common.statistics.StatisticDescriptor;
+import org.apache.geode.stats.common.statistics.Statistics;
+import org.apache.geode.stats.common.statistics.StatisticsType;
 
 /**
  * Generates the stat archive file that is committed under src/test/resources for
@@ -101,7 +101,7 @@ public class StatArchiveWithConsecutiveResourceInstGenerator {
 
   @After
   public void tearDown() throws Exception {
-    StatisticsTypeFactoryImpl.clear();
+    // StatisticsTypeFactoryImpl.clear();
   }
 
   @Test
diff --git a/geode-core/src/integrationTest/java/org/apache/geode/internal/statistics/StatArchiveWithMissingResourceTypeRegressionTest.java b/geode-core/src/integrationTest/java/org/apache/geode/internal/statistics/StatArchiveWithMissingResourceTypeRegressionTest.java
index ac1e2e7..70756db 100644
--- a/geode-core/src/integrationTest/java/org/apache/geode/internal/statistics/StatArchiveWithMissingResourceTypeRegressionTest.java
+++ b/geode-core/src/integrationTest/java/org/apache/geode/internal/statistics/StatArchiveWithMissingResourceTypeRegressionTest.java
@@ -64,9 +64,7 @@ public class StatArchiveWithMissingResourceTypeRegressionTest {
   }
 
   @After
-  public void tearDown() throws Exception {
-    StatisticsTypeFactoryImpl.clear();
-  }
+  public void tearDown() throws Exception {}
 
   @Test
   public void throwsIllegalStateExceptionWithMessage() throws Exception {
diff --git a/geode-core/src/integrationTest/java/org/apache/geode/internal/statistics/StatArchiveWriterReaderIntegrationTest.java b/geode-core/src/integrationTest/java/org/apache/geode/internal/statistics/StatArchiveWriterReaderIntegrationTest.java
index a320f2a..373b6a9 100755
--- a/geode-core/src/integrationTest/java/org/apache/geode/internal/statistics/StatArchiveWriterReaderIntegrationTest.java
+++ b/geode-core/src/integrationTest/java/org/apache/geode/internal/statistics/StatArchiveWriterReaderIntegrationTest.java
@@ -48,11 +48,11 @@ import org.junit.experimental.categories.Category;
 import org.junit.rules.TemporaryFolder;
 import org.junit.rules.TestName;
 
-import org.apache.geode.StatisticDescriptor;
-import org.apache.geode.Statistics;
-import org.apache.geode.StatisticsType;
 import org.apache.geode.internal.NanoTimer;
 import org.apache.geode.internal.statistics.StatArchiveReader.StatValue;
+import org.apache.geode.stats.common.statistics.StatisticDescriptor;
+import org.apache.geode.stats.common.statistics.Statistics;
+import org.apache.geode.stats.common.statistics.StatisticsType;
 import org.apache.geode.test.junit.categories.StatisticsTest;
 import org.apache.geode.util.test.TestUtil;
 
@@ -91,7 +91,6 @@ public class StatArchiveWriterReaderIntegrationTest {
   public void tearDown() throws Exception {
     this.statisticTypes = null;
     this.allStatistics = null;
-    StatisticsTypeFactoryImpl.clear();
   }
 
   @Test
@@ -1598,6 +1597,21 @@ public class StatArchiveWriterReaderIntegrationTest {
       public StatisticDescriptor nameToDescriptor(String name) {
         return null;
       }
+
+      @Override
+      public int getIntStatCount() {
+        return 0;
+      }
+
+      @Override
+      public int getLongStatCount() {
+        return 0;
+      }
+
+      @Override
+      public int getDoubleStatCount() {
+        return 0;
+      }
     };
   }
 
diff --git a/geode-core/src/integrationTest/java/org/apache/geode/internal/statistics/StatSamplerIntegrationTest.java b/geode-core/src/integrationTest/java/org/apache/geode/internal/statistics/StatSamplerIntegrationTest.java
index 59af828..124bffe 100755
--- a/geode-core/src/integrationTest/java/org/apache/geode/internal/statistics/StatSamplerIntegrationTest.java
+++ b/geode-core/src/integrationTest/java/org/apache/geode/internal/statistics/StatSamplerIntegrationTest.java
@@ -38,11 +38,11 @@ import org.junit.rules.TemporaryFolder;
 import org.junit.rules.TestName;
 
 import org.apache.geode.CancelCriterion;
-import org.apache.geode.StatisticDescriptor;
-import org.apache.geode.Statistics;
-import org.apache.geode.StatisticsType;
 import org.apache.geode.internal.logging.LogService;
 import org.apache.geode.internal.statistics.StatArchiveReader.StatValue;
+import org.apache.geode.stats.common.statistics.StatisticDescriptor;
+import org.apache.geode.stats.common.statistics.Statistics;
+import org.apache.geode.stats.common.statistics.StatisticsType;
 import org.apache.geode.test.junit.categories.StatisticsTest;
 
 /**
@@ -77,7 +77,6 @@ public class StatSamplerIntegrationTest {
   public void tearDown() {
     this.statisticTypes = null;
     this.allStatistics = null;
-    StatisticsTypeFactoryImpl.clear();
     StatArchiveWriter.clearTraceFilter();
   }
 
diff --git a/geode-core/src/integrationTest/java/org/apache/geode/internal/statistics/StatTypesAreRolledOverRegressionTest.java b/geode-core/src/integrationTest/java/org/apache/geode/internal/statistics/StatTypesAreRolledOverRegressionTest.java
index 37e61ea..784e7f5 100644
--- a/geode-core/src/integrationTest/java/org/apache/geode/internal/statistics/StatTypesAreRolledOverRegressionTest.java
+++ b/geode-core/src/integrationTest/java/org/apache/geode/internal/statistics/StatTypesAreRolledOverRegressionTest.java
@@ -33,12 +33,12 @@ import org.junit.rules.TemporaryFolder;
 import org.junit.rules.TestName;
 import org.mockito.stubbing.Answer;
 
-import org.apache.geode.StatisticDescriptor;
-import org.apache.geode.Statistics;
-import org.apache.geode.StatisticsType;
 import org.apache.geode.internal.NanoTimer;
 import org.apache.geode.internal.io.MainWithChildrenRollingFileHandler;
 import org.apache.geode.internal.statistics.StatArchiveReader.ResourceInst;
+import org.apache.geode.stats.common.statistics.StatisticDescriptor;
+import org.apache.geode.stats.common.statistics.Statistics;
+import org.apache.geode.stats.common.statistics.StatisticsType;
 import org.apache.geode.test.junit.categories.StatisticsTest;
 
 @Category({StatisticsTest.class})
@@ -104,9 +104,7 @@ public class StatTypesAreRolledOverRegressionTest {
   }
 
   @After
-  public void tearDown() throws Exception {
-    StatisticsTypeFactoryImpl.clear();
-  }
+  public void tearDown() throws Exception {}
 
   @Test
   public void closedInstanceShouldHaveTypeInRolledArchives() throws Exception {
diff --git a/geode-core/src/integrationTest/java/org/apache/geode/internal/statistics/ValueMonitorIntegrationTest.java b/geode-core/src/integrationTest/java/org/apache/geode/internal/statistics/ValueMonitorIntegrationTest.java
index 489cd53..e3b0e70 100755
--- a/geode-core/src/integrationTest/java/org/apache/geode/internal/statistics/ValueMonitorIntegrationTest.java
+++ b/geode-core/src/integrationTest/java/org/apache/geode/internal/statistics/ValueMonitorIntegrationTest.java
@@ -27,18 +27,19 @@ import java.util.Map;
 import java.util.concurrent.TimeUnit;
 
 import org.awaitility.Awaitility;
+import org.jetbrains.annotations.NotNull;
 import org.junit.Before;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.junit.rules.TestName;
 
-import org.apache.geode.StatisticDescriptor;
-import org.apache.geode.Statistics;
-import org.apache.geode.StatisticsType;
 import org.apache.geode.internal.NanoTimer;
 import org.apache.geode.internal.io.MainWithChildrenRollingFileHandler;
 import org.apache.geode.internal.statistics.StatisticsNotification.Type;
+import org.apache.geode.stats.common.statistics.StatisticDescriptor;
+import org.apache.geode.stats.common.statistics.Statistics;
+import org.apache.geode.stats.common.statistics.StatisticsType;
 import org.apache.geode.test.junit.categories.StatisticsTest;
 
 /**
@@ -229,36 +230,47 @@ public class ValueMonitorIntegrationTest {
     assertThat(statCount).isEqualTo(3);
 
     // validate no notification occurs when no stats are updated
-    timeStamp += NanoTimer.millisToNanos(1000);
-    sampleCollector.sample(timeStamp);
-    Awaitility.await().atMost(2, TimeUnit.SECONDS).pollInterval(10, TimeUnit.MILLISECONDS)
-        .until(() -> notifications.size() == 0);
-    assertThat(notifications.isEmpty()).isTrue();
+    timeStamp = validateTimeStamp(sampleCollector, notifications, timeStamp);
 
     // validate no notification occurs when only other stats are updated
     st1_2.incDouble("double_counter_1", 3.3);
     st1_2.incInt("int_counter_2", 1);
     st1_2.incLong("long_counter_3", 2);
-    timeStamp += NanoTimer.millisToNanos(1000);
-    sampleCollector.sample(timeStamp);
-    Awaitility.await().atMost(2, TimeUnit.SECONDS).pollInterval(10, TimeUnit.MILLISECONDS)
-        .until(() -> notifications.size() == 0);
-    assertThat(notifications.isEmpty()).isTrue();
+    timeStamp = validateTimeStamp(sampleCollector, notifications, timeStamp);
 
     // validate notification only contains stats added to monitor
     st1_1.incInt("int_counter_2", 100);
     st1_2.incInt("int_counter_2", 200);
     assertThat(sampleCollector.currentHandlersForTesting().size()).isEqualTo(2);
+    notification = validateTimeStamp2(sampleCollector, notifications, timeStamp);
+    expectedValues = new HashMap<>();
+    expectedValues.put("int_counter_2", 104);
+    statCount = assertStatisticsNotification(notification, expectedValues);
+    assertThat(statCount).isEqualTo(1);
+  }
+
+  @NotNull
+  private StatisticsNotification validateTimeStamp2(SampleCollector sampleCollector,
+      List<StatisticsNotification> notifications,
+      long timeStamp) {
+    StatisticsNotification notification;
     timeStamp += NanoTimer.millisToNanos(1000);
     sampleCollector.sample(timeStamp);
     Awaitility.await().atMost(2, TimeUnit.SECONDS).pollInterval(10, TimeUnit.MILLISECONDS)
         .until(() -> notifications.size() > 0);
     assertThat(notifications.size()).isEqualTo(1);
     notification = notifications.remove(0);
-    assertThat(notification.getType()).isEqualTo(StatisticsNotification.Type.VALUE_CHANGED);
-    expectedValues = new HashMap<>();
-    expectedValues.put("int_counter_2", 104);
-    statCount = assertStatisticsNotification(notification, expectedValues);
-    assertThat(statCount).isEqualTo(1);
+    assertThat(notification.getType()).isEqualTo(Type.VALUE_CHANGED);
+    return notification;
+  }
+
+  private long validateTimeStamp(SampleCollector sampleCollector,
+      List<StatisticsNotification> notifications, long timeStamp) {
+    timeStamp += NanoTimer.millisToNanos(1000);
+    sampleCollector.sample(timeStamp);
+    Awaitility.await().atMost(2, TimeUnit.SECONDS).pollInterval(10, TimeUnit.MILLISECONDS)
+        .until(() -> notifications.size() == 0);
+    assertThat(notifications.isEmpty()).isTrue();
+    return timeStamp;
   }
 }
diff --git a/geode-core/src/integrationTest/java/org/apache/geode/internal/stats50/AtomicStatsJUnitTest.java b/geode-core/src/integrationTest/java/org/apache/geode/internal/stats50/AtomicStatsJUnitTest.java
index 46753c9..3130a36 100644
--- a/geode-core/src/integrationTest/java/org/apache/geode/internal/stats50/AtomicStatsJUnitTest.java
+++ b/geode-core/src/integrationTest/java/org/apache/geode/internal/stats50/AtomicStatsJUnitTest.java
@@ -25,12 +25,12 @@ import java.util.concurrent.atomic.AtomicReference;
 
 import org.junit.Test;
 
-import org.apache.geode.StatisticDescriptor;
-import org.apache.geode.Statistics;
-import org.apache.geode.StatisticsType;
-import org.apache.geode.StatisticsTypeFactory;
 import org.apache.geode.distributed.DistributedSystem;
 import org.apache.geode.internal.statistics.StatisticsTypeFactoryImpl;
+import org.apache.geode.stats.common.statistics.StatisticDescriptor;
+import org.apache.geode.stats.common.statistics.Statistics;
+import org.apache.geode.stats.common.statistics.StatisticsType;
+import org.apache.geode.stats.common.statistics.StatisticsTypeFactory;
 
 public class AtomicStatsJUnitTest {
 
@@ -46,14 +46,14 @@ public class AtomicStatsJUnitTest {
     props.setProperty(MCAST_PORT, "0");
     // props.setProperty("statistic-sample-rate", "60000");
     props.setProperty(STATISTIC_SAMPLING_ENABLED, "false");
-    DistributedSystem ds = DistributedSystem.connect(props);
+    DistributedSystem distributedSystem = DistributedSystem.connect(props);
 
     String statName = "TestStats";
     String statDescription = "Tests stats";
 
     final String statDesc = "blah blah blah";
 
-    StatisticsTypeFactory f = StatisticsTypeFactoryImpl.singleton();
+    StatisticsTypeFactory f = new StatisticsTypeFactoryImpl();
 
     StatisticsType type = f.createType(statName, statDescription, new StatisticDescriptor[] {
         f.createIntGauge("stat", statDesc, "bottles of beer on the wall"),});
@@ -102,7 +102,8 @@ public class AtomicStatsJUnitTest {
       thread1.start();
       thread3.start();
       for (int i = 0; i < 5000; i++) {
-        Statistics stats = ds.createAtomicStatistics(type, "stats");
+        Statistics stats =
+            distributedSystem.getStatisticsFactory().createAtomicStatistics(type, "stats");
         statsRef.set(stats);
         beforeIncrement.await();
         afterIncrement.await();
@@ -111,7 +112,7 @@ public class AtomicStatsJUnitTest {
       }
 
     } finally {
-      ds.disconnect();
+      distributedSystem.disconnect();
     }
   }
 }
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 b777ac0..1c0073b 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
@@ -19,9 +19,9 @@ import static org.junit.Assert.assertEquals;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
-import org.apache.geode.StatisticDescriptor;
 import org.apache.geode.cache.asyncqueue.internal.AsyncEventQueueStats;
 import org.apache.geode.management.internal.beans.AsyncEventQueueMBeanBridge;
+import org.apache.geode.stats.common.statistics.StatisticDescriptor;
 import org.apache.geode.test.junit.categories.JMXTest;
 
 @Category({JMXTest.class})
@@ -32,7 +32,7 @@ public class AsyncEventQueueStatsJUnitTest extends MBeanStatsTestCase {
   private AsyncEventQueueStats asyncEventQueueStats;
 
   public void init() {
-    asyncEventQueueStats = new AsyncEventQueueStats(system, "test");
+    asyncEventQueueStats = new AsyncEventQueueStats(system.getStatisticsFactory(), "test");
 
     bridge = new AsyncEventQueueMBeanBridge();
     bridge.addAsyncEventQueueStats(asyncEventQueueStats);
@@ -57,7 +57,7 @@ public class AsyncEventQueueStatsJUnitTest extends MBeanStatsTestCase {
 
   @Test
   public void testStatDescriptors() {
-    StatisticDescriptor[] sds = asyncEventQueueStats.type.getStatistics();
+    StatisticDescriptor[] sds = asyncEventQueueStats.getType().getStatistics();
     int notQueueEvents = 0;
     int notQueueToPrimary = 0;
     int eventsProcessedByPQRM = 0;
diff --git a/geode-core/src/integrationTest/java/org/apache/geode/management/bean/stats/CacheServerStatsJUnitTest.java b/geode-core/src/integrationTest/java/org/apache/geode/management/bean/stats/CacheServerStatsJUnitTest.java
index ded4e91..fd699f7 100644
--- a/geode-core/src/integrationTest/java/org/apache/geode/management/bean/stats/CacheServerStatsJUnitTest.java
+++ b/geode-core/src/integrationTest/java/org/apache/geode/management/bean/stats/CacheServerStatsJUnitTest.java
@@ -21,8 +21,9 @@ import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
 import org.apache.geode.cache.server.ServerLoad;
-import org.apache.geode.internal.cache.tier.sockets.CacheServerStats;
 import org.apache.geode.management.internal.beans.CacheServerBridge;
+import org.apache.geode.stats.common.internal.cache.tier.sockets.CacheServerStats;
+import org.apache.geode.stats.common.statistics.factory.StatsFactory;
 import org.apache.geode.test.junit.categories.JMXTest;
 
 @Category({JMXTest.class})
@@ -33,7 +34,7 @@ public class CacheServerStatsJUnitTest extends MBeanStatsTestCase {
   private CacheServerStats cacheServerStats;
 
   public void init() {
-    cacheServerStats = new CacheServerStats("Test Sock Name");
+    cacheServerStats = StatsFactory.createStatsImpl(CacheServerStats.class, "Test Sock Name");
 
     bridge = new CacheServerBridge();
     bridge.addCacheServerStats(cacheServerStats);
@@ -58,7 +59,8 @@ public class CacheServerStatsJUnitTest extends MBeanStatsTestCase {
     cacheServerStats.incProcessPutTime(startTime);
 
     ServerLoad load = new ServerLoad(1, 1, 1, 1);
-    cacheServerStats.setLoad(load);
+    cacheServerStats.setLoad(load.getConnectionLoad(), load.getLoadPerConnection(),
+        load.getSubscriptionConnectionLoad(), load.getLoadPerSubscriptionConnection());
 
     sample();
 
diff --git a/geode-core/src/integrationTest/java/org/apache/geode/management/bean/stats/DiskStatsJUnitTest.java b/geode-core/src/integrationTest/java/org/apache/geode/management/bean/stats/DiskStatsJUnitTest.java
index 39cc270..b7bea98 100644
--- a/geode-core/src/integrationTest/java/org/apache/geode/management/bean/stats/DiskStatsJUnitTest.java
+++ b/geode-core/src/integrationTest/java/org/apache/geode/management/bean/stats/DiskStatsJUnitTest.java
@@ -22,8 +22,9 @@ import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
 import org.apache.geode.internal.NanoTimer;
-import org.apache.geode.internal.cache.DiskStoreStats;
 import org.apache.geode.management.internal.beans.DiskStoreMBeanBridge;
+import org.apache.geode.stats.common.internal.cache.DiskStoreStats;
+import org.apache.geode.stats.common.statistics.factory.StatsFactory;
 import org.apache.geode.test.junit.categories.JMXTest;
 
 @Category({JMXTest.class})
@@ -36,7 +37,8 @@ public class DiskStatsJUnitTest extends MBeanStatsTestCase {
   private static long testStartTime = NanoTimer.getTime();
 
   public void init() {
-    diskStoreStats = new DiskStoreStats(system, "test");
+    diskStoreStats =
+        StatsFactory.createStatsImpl(DiskStoreStats.class, "test");
 
     bridge = new DiskStoreMBeanBridge();
     bridge.addDiskStoreStats(diskStoreStats);
diff --git a/geode-core/src/integrationTest/java/org/apache/geode/management/bean/stats/DistributedSystemStatsJUnitTest.java b/geode-core/src/integrationTest/java/org/apache/geode/management/bean/stats/DistributedSystemStatsJUnitTest.java
index d44402d..ff85ecd 100644
--- a/geode-core/src/integrationTest/java/org/apache/geode/management/bean/stats/DistributedSystemStatsJUnitTest.java
+++ b/geode-core/src/integrationTest/java/org/apache/geode/management/bean/stats/DistributedSystemStatsJUnitTest.java
@@ -37,11 +37,11 @@ import org.apache.geode.cache.CacheFactory;
 import org.apache.geode.distributed.DistributedSystem;
 import org.apache.geode.distributed.internal.InternalDistributedSystem;
 import org.apache.geode.internal.NanoTimer;
-import org.apache.geode.internal.cache.CachePerfStats;
 import org.apache.geode.internal.cache.GemFireCacheImpl;
 import org.apache.geode.management.DistributedSystemMXBean;
 import org.apache.geode.management.ManagementService;
 import org.apache.geode.management.internal.SystemManagementService;
+import org.apache.geode.stats.common.internal.cache.CachePerfStats;
 import org.apache.geode.test.junit.categories.JMXTest;
 
 /**
@@ -75,8 +75,9 @@ public class DistributedSystemStatsJUnitTest {
     props.setProperty(JMX_MANAGER_PORT, "0");
 
     this.system = (InternalDistributedSystem) DistributedSystem.connect(props);
-    assertNotNull(this.system.getStatSampler());
-    assertNotNull(this.system.getStatSampler().waitForSampleCollector(TIMEOUT));
+    assertNotNull(this.system.getInternalDistributedSystemStats().getStatSampler());
+    assertNotNull(this.system.getInternalDistributedSystemStats().getStatSampler()
+        .waitForSampleCollector(TIMEOUT));
 
     this.cache = new CacheFactory().create();
 
@@ -115,12 +116,13 @@ public class DistributedSystemStatsJUnitTest {
   }
 
   protected void waitForNotification() throws InterruptedException {
-    this.system.getStatSampler().waitForSample(TIMEOUT);
+    this.system.getInternalDistributedSystemStats().getStatSampler().waitForSample(TIMEOUT);
     Thread.sleep(SLEEP);
   }
 
   protected void sample() throws InterruptedException {
-    this.system.getStatSampler().getSampleCollector().sample(NanoTimer.getTime());
+    this.system.getInternalDistributedSystemStats().getStatSampler().getSampleCollector()
+        .sample(NanoTimer.getTime());
     Thread.sleep(SLEEP);
   }
 
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 53f1e4b..f15d996 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
@@ -23,8 +23,9 @@ import org.mockito.Mockito;
 
 import org.apache.geode.internal.NanoTimer;
 import org.apache.geode.internal.cache.wan.AbstractGatewaySender;
-import org.apache.geode.internal.cache.wan.GatewaySenderStats;
 import org.apache.geode.management.internal.beans.GatewaySenderMBeanBridge;
+import org.apache.geode.stats.common.internal.cache.wan.GatewaySenderStats;
+import org.apache.geode.stats.common.statistics.factory.StatsFactory;
 import org.apache.geode.test.junit.categories.JMXTest;
 
 @Category({JMXTest.class})
@@ -39,7 +40,7 @@ public class GatewayMBeanBridgeJUnitTest extends MBeanStatsTestCase {
   private AbstractGatewaySender sender;
 
   public void init() {
-    senderStats = new GatewaySenderStats(system, "test");
+    senderStats = StatsFactory.createStatsImpl(GatewaySenderStats.class, "test");
 
     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/GatewayReceiverStatsJUnitTest.java b/geode-core/src/integrationTest/java/org/apache/geode/management/bean/stats/GatewayReceiverStatsJUnitTest.java
index f87a054..4327d37 100644
--- a/geode-core/src/integrationTest/java/org/apache/geode/management/bean/stats/GatewayReceiverStatsJUnitTest.java
+++ b/geode-core/src/integrationTest/java/org/apache/geode/management/bean/stats/GatewayReceiverStatsJUnitTest.java
@@ -21,8 +21,9 @@ import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
 import org.apache.geode.cache.server.ServerLoad;
-import org.apache.geode.internal.cache.wan.GatewayReceiverStats;
 import org.apache.geode.management.internal.beans.GatewayReceiverMBeanBridge;
+import org.apache.geode.stats.common.internal.cache.wan.GatewayReceiverStats;
+import org.apache.geode.stats.common.statistics.factory.StatsFactory;
 import org.apache.geode.test.junit.categories.JMXTest;
 
 @Category({JMXTest.class})
@@ -33,7 +34,8 @@ public class GatewayReceiverStatsJUnitTest extends MBeanStatsTestCase {
   private GatewayReceiverStats receiverStats;
 
   public void init() {
-    receiverStats = GatewayReceiverStats.createGatewayReceiverStats("Test Sock Name");
+    receiverStats =
+        StatsFactory.createStatsImpl(GatewayReceiverStats.class, "Test Sock Name");
 
     bridge = new GatewayReceiverMBeanBridge();
     bridge.addGatewayReceiverStats(receiverStats);
@@ -58,7 +60,8 @@ public class GatewayReceiverStatsJUnitTest extends MBeanStatsTestCase {
     receiverStats.incProcessPutTime(startTime);
 
     ServerLoad load = new ServerLoad(1, 1, 1, 1);
-    receiverStats.setLoad(load);
+    receiverStats.setLoad(load.getConnectionLoad(), load.getLoadPerConnection(),
+        load.getSubscriptionConnectionLoad(), load.getLoadPerSubscriptionConnection());
 
     sample();
 
diff --git a/geode-core/src/integrationTest/java/org/apache/geode/management/bean/stats/MemberLevelStatsJUnitTest.java b/geode-core/src/integrationTest/java/org/apache/geode/management/bean/stats/MemberLevelStatsJUnitTest.java
index ff27c75..04c4c60 100644
--- a/geode-core/src/integrationTest/java/org/apache/geode/management/bean/stats/MemberLevelStatsJUnitTest.java
+++ b/geode-core/src/integrationTest/java/org/apache/geode/management/bean/stats/MemberLevelStatsJUnitTest.java
@@ -26,19 +26,21 @@ import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.junit.rules.TestName;
 
-import org.apache.geode.distributed.internal.DistributionStats;
-import org.apache.geode.distributed.internal.locks.DLockStats;
+import org.apache.geode.distributed.internal.DistributionStatsImpl;
 import org.apache.geode.internal.NanoTimer;
 import org.apache.geode.internal.OSProcess;
-import org.apache.geode.internal.cache.CachePerfStats;
-import org.apache.geode.internal.cache.DiskStoreStats;
 import org.apache.geode.internal.cache.GemFireCacheImpl;
-import org.apache.geode.internal.cache.PartitionedRegionStats;
-import org.apache.geode.internal.cache.execute.FunctionServiceStats;
-import org.apache.geode.internal.statistics.VMStatsContract;
+import org.apache.geode.internal.cache.execute.FunctionServiceStatsImpl;
+import org.apache.geode.internal.statistics.VMStats;
 import org.apache.geode.internal.stats50.VMStats50;
 import org.apache.geode.management.ManagementService;
 import org.apache.geode.management.internal.beans.MemberMBeanBridge;
+import org.apache.geode.stats.common.distributed.internal.DistributionStats;
+import org.apache.geode.stats.common.distributed.internal.locks.DLockStats;
+import org.apache.geode.stats.common.internal.cache.CachePerfStats;
+import org.apache.geode.stats.common.internal.cache.DiskStoreStats;
+import org.apache.geode.stats.common.internal.cache.PartitionedRegionStats;
+import org.apache.geode.stats.common.statistics.factory.StatsFactory;
 import org.apache.geode.test.junit.categories.JMXTest;
 
 @Category({JMXTest.class})
@@ -50,7 +52,7 @@ public class MemberLevelStatsJUnitTest extends MBeanStatsTestCase {
 
   private CachePerfStats cachePerfStats;
 
-  private FunctionServiceStats funcServiceStats;
+  private FunctionServiceStatsImpl funcServiceStats;
 
   private DistributionStats distributionStats;
 
@@ -65,12 +67,17 @@ public class MemberLevelStatsJUnitTest extends MBeanStatsTestCase {
   private static long testStartTime = NanoTimer.getTime();
 
   public void init() {
-    cachePerfStats = new CachePerfStats(system);
-    funcServiceStats = new FunctionServiceStats(system, "FunctionExecution");
+    cachePerfStats =
+        StatsFactory.createStatsImpl(CachePerfStats.class, null);
+    funcServiceStats =
+        new FunctionServiceStatsImpl(system.getStatisticsFactory(), "FunctionExecution");
     long statId = OSProcess.getId();
-    distributionStats = new DistributionStats(system, statId);
-    DistributionStats.enableClockStats = true;
-    dlockStats = new DLockStats(system, statId);
+    distributionStats =
+        StatsFactory.createStatsImpl(DistributionStats.class,
+            String.valueOf(statId));
+    DistributionStatsImpl.enableClockStats = true;
+    dlockStats = StatsFactory.createStatsImpl(DLockStats.class,
+        String.valueOf(statId));
 
     bridge = new MemberMBeanBridge();
     bridge.addCacheStats(cachePerfStats);
@@ -79,20 +86,24 @@ public class MemberLevelStatsJUnitTest extends MBeanStatsTestCase {
     bridge.addLockServiceStats(dlockStats);
 
 
-    VMStatsContract vmstats = system.getStatSampler().getVMStats();
+    VMStats vmstats =
+        system.getInternalDistributedSystemStats().getStatSampler().getVMStats();
     assertTrue(vmstats instanceof VMStats50);
 
     bridge.addSystemStats();
     bridge.addVMStats();
 
     for (int i = 0; i < 4; i++) {
-      DiskStoreStats stats = new DiskStoreStats(system, name.getMethodName() + i);
+      DiskStoreStats stats =
+          StatsFactory.createStatsImpl(DiskStoreStats.class,
+              name.getMethodName() + i);
       diskStatsList.add(stats);
       bridge.addDiskStoreStats(stats);
     }
 
     for (int i = 0; i < 4; i++) {
-      PartitionedRegionStats stats = new PartitionedRegionStats(system, name.getMethodName() + i);
+      PartitionedRegionStats stats = StatsFactory.createStatsImpl(PartitionedRegionStats.class,
+          name.getMethodName() + i);
       parRegionStatsList.add(stats);
       bridge.addPartionRegionStats(stats);
     }
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 fea30be..859d149 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
@@ -20,17 +20,17 @@ import static org.junit.Assert.assertTrue;
 import org.junit.Test;
 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.management.internal.beans.DiskRegionBridge;
 import org.apache.geode.management.internal.beans.PartitionedRegionBridge;
 import org.apache.geode.management.internal.beans.RegionMBeanBridge;
+import org.apache.geode.stats.common.internal.cache.CachePerfStats;
+import org.apache.geode.stats.common.internal.cache.DiskRegionStats;
+import org.apache.geode.stats.common.internal.cache.PartitionedRegionStats;
+import org.apache.geode.stats.common.statistics.factory.StatsFactory;
 import org.apache.geode.test.junit.categories.JMXTest;
 
 /**
  * Implementing RegionMXBean to ensure test coverage
- *
  */
 @Category({JMXTest.class})
 public class RegionStatsJUnitTest extends MBeanStatsTestCase {
@@ -48,9 +48,13 @@ public class RegionStatsJUnitTest extends MBeanStatsTestCase {
   private DiskRegionStats diskRegionStats;
 
   protected void init() {
-    cachePerfStats = new CachePerfStats(system);
-    partitionedRegionStats = new PartitionedRegionStats(system, "/tests");
-    diskRegionStats = new DiskRegionStats(system, "test-disk");
+    cachePerfStats = StatsFactory.createCachePerfStatsImpl(null);
+    partitionedRegionStats =
+        StatsFactory
+            .createStatsImpl(PartitionedRegionStats.class, "/tests");
+    diskRegionStats =
+        StatsFactory
+            .createStatsImpl(DiskRegionStats.class, "test-disk");
 
     bridge = new RegionMBeanBridge(cachePerfStats);
     parBridge = new PartitionedRegionBridge(partitionedRegionStats);
diff --git a/geode-core/src/integrationTest/java/org/apache/geode/management/internal/beans/ManagementAdapterTest.java b/geode-core/src/integrationTest/java/org/apache/geode/management/internal/beans/ManagementAdapterTest.java
index 66927f0..4999782 100644
--- a/geode-core/src/integrationTest/java/org/apache/geode/management/internal/beans/ManagementAdapterTest.java
+++ b/geode-core/src/integrationTest/java/org/apache/geode/management/internal/beans/ManagementAdapterTest.java
@@ -31,8 +31,9 @@ import org.junit.Test;
 import org.apache.geode.distributed.internal.InternalDistributedSystem;
 import org.apache.geode.distributed.internal.ResourceEvent;
 import org.apache.geode.internal.cache.DiskStoreImpl;
-import org.apache.geode.internal.cache.DiskStoreStats;
 import org.apache.geode.internal.cache.InternalCache;
+import org.apache.geode.stats.common.internal.cache.DiskStoreStats;
+import org.apache.geode.stats.common.statistics.factory.StatsFactory;
 import org.apache.geode.test.junit.rules.ConcurrencyRule;
 import org.apache.geode.test.junit.rules.ServerStarterRule;
 
@@ -52,8 +53,8 @@ public class ManagementAdapterTest {
   @Before
   public void before() {
     cache = serverRule.getCache();
-    doReturn(new DiskStoreStats(cache.getInternalDistributedSystem(), "disk-stats"))
-        .when(diskStore).getStats();
+    doReturn(StatsFactory.createStatsImpl(DiskStoreStats.class, "disk-stats")).when(diskStore)
+        .getStats();
     doReturn(new File[] {}).when(diskStore).getDiskDirs();
   }
 
diff --git a/geode-core/src/integrationTest/java/org/apache/geode/pdx/OffHeapByteSourceJUnitTest.java b/geode-core/src/integrationTest/java/org/apache/geode/pdx/OffHeapByteSourceJUnitTest.java
index a28c4bc..a30db02 100644
--- a/geode-core/src/integrationTest/java/org/apache/geode/pdx/OffHeapByteSourceJUnitTest.java
+++ b/geode-core/src/integrationTest/java/org/apache/geode/pdx/OffHeapByteSourceJUnitTest.java
@@ -18,7 +18,7 @@ import org.junit.After;
 import org.junit.Before;
 
 import org.apache.geode.internal.offheap.MemoryAllocatorImpl;
-import org.apache.geode.internal.offheap.NullOffHeapMemoryStats;
+import org.apache.geode.internal.offheap.NullOffHeapStorageStats;
 import org.apache.geode.internal.offheap.NullOutOfOffHeapMemoryListener;
 import org.apache.geode.internal.offheap.OffHeapStoredObject;
 import org.apache.geode.internal.offheap.SlabImpl;
@@ -32,7 +32,7 @@ public class OffHeapByteSourceJUnitTest extends ByteSourceJUnitTest {
   @Before
   public final void setUp() throws Exception {
     MemoryAllocatorImpl.createForUnitTest(new NullOutOfOffHeapMemoryListener(),
-        new NullOffHeapMemoryStats(), new SlabImpl[] {new SlabImpl(1024 * 1024)});
+        new NullOffHeapStorageStats(), new SlabImpl[] {new SlabImpl(1024 * 1024)});
   }
 
   @After
diff --git a/geode-core/src/main/java/org/apache/geode/admin/internal/CacheHealthEvaluator.java b/geode-core/src/main/java/org/apache/geode/admin/internal/CacheHealthEvaluator.java
index adf03ca..bda731c 100644
--- a/geode-core/src/main/java/org/apache/geode/admin/internal/CacheHealthEvaluator.java
+++ b/geode-core/src/main/java/org/apache/geode/admin/internal/CacheHealthEvaluator.java
@@ -26,11 +26,11 @@ import org.apache.geode.distributed.internal.DistributionManager;
 import org.apache.geode.distributed.internal.InternalDistributedSystem;
 import org.apache.geode.internal.OSProcess;
 import org.apache.geode.internal.cache.CacheLifecycleListener;
-import org.apache.geode.internal.cache.CachePerfStats;
 import org.apache.geode.internal.cache.GemFireCacheImpl;
 import org.apache.geode.internal.cache.InternalCache;
 import org.apache.geode.internal.i18n.LocalizedStrings;
 import org.apache.geode.internal.logging.LogService;
+import org.apache.geode.stats.common.internal.cache.CachePerfStats;
 
 /**
  * Contains the logic for evaluating the health of a GemFire {@code Cache} instance according to the
diff --git a/geode-core/src/main/java/org/apache/geode/admin/internal/MemberHealthEvaluator.java b/geode-core/src/main/java/org/apache/geode/admin/internal/MemberHealthEvaluator.java
index 57ba1b2..591e712 100644
--- a/geode-core/src/main/java/org/apache/geode/admin/internal/MemberHealthEvaluator.java
+++ b/geode-core/src/main/java/org/apache/geode/admin/internal/MemberHealthEvaluator.java
@@ -20,15 +20,15 @@ import org.apache.geode.CancelException;
 import org.apache.geode.admin.GemFireHealthConfig;
 import org.apache.geode.admin.MemberHealthConfig;
 import org.apache.geode.cache.CacheFactory;
-import org.apache.geode.distributed.internal.DMStats;
 import org.apache.geode.distributed.internal.DistributionManager;
 import org.apache.geode.distributed.internal.InternalDistributedSystem;
 import org.apache.geode.internal.OSProcess;
-import org.apache.geode.internal.cache.CachePerfStats;
 import org.apache.geode.internal.cache.InternalCache;
 import org.apache.geode.internal.i18n.LocalizedStrings;
 import org.apache.geode.internal.statistics.GemFireStatSampler;
 import org.apache.geode.internal.statistics.platform.ProcessStats;
+import org.apache.geode.stats.common.distributed.internal.DMStats;
+import org.apache.geode.stats.common.internal.cache.CachePerfStats;
 
 /**
  * Contains the logic for evaluating the health of a GemFire distributed system member according to
@@ -62,7 +62,7 @@ class MemberHealthEvaluator extends AbstractHealthEvaluator {
     this.config = config;
     InternalDistributedSystem system = dm.getSystem();
 
-    GemFireStatSampler sampler = system.getStatSampler();
+    GemFireStatSampler sampler = system.getInternalDistributedSystemStats().getStatSampler();
     if (sampler != null) {
       // Sampling is enabled
       this.processStats = sampler.getProcessStats();
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 2f317cf..068f4f3 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
@@ -14,98 +14,113 @@
  */
 package org.apache.geode.cache.asyncqueue.internal;
 
-import org.apache.geode.StatisticDescriptor;
-import org.apache.geode.StatisticsFactory;
-import org.apache.geode.StatisticsType;
-import org.apache.geode.StatisticsTypeFactory;
-import org.apache.geode.internal.cache.wan.GatewaySenderStats;
-import org.apache.geode.internal.statistics.StatisticsTypeFactoryImpl;
+import org.apache.geode.internal.cache.wan.GatewaySenderStatsImpl;
+import org.apache.geode.stats.common.statistics.StatisticDescriptor;
+import org.apache.geode.stats.common.statistics.StatisticsFactory;
+import org.apache.geode.stats.common.statistics.StatisticsType;
 
-public class AsyncEventQueueStats extends GatewaySenderStats {
 
-  public static final String typeName = "AsyncEventQueueStatistics";
+public class AsyncEventQueueStats extends GatewaySenderStatsImpl {
 
-  /** The <code>StatisticsType</code> of the statistics */
-  public static final StatisticsType type;
+  public static final String typeName = "AsyncEventQueueStatistics";
 
+  /**
+   * The <code>StatisticsType</code> of the statistics
+   */
+  private StatisticsType type;
 
-  static {
+  /**
+   * Returns the internal ID for {@link #getEventQueueSize()} statistic
+   */
+  public static String getEventQueueSizeId() {
+    return GatewaySenderStatsImpl.EVENT_QUEUE_SIZE;
+  }
 
-    StatisticsTypeFactory f = StatisticsTypeFactoryImpl.singleton();
+  /**
+   * Returns the internal ID for {@link #getTempEventQueueSize()} statistic
+   */
+  public static String getEventTempQueueSizeId() {
+    return GatewaySenderStatsImpl.TMP_EVENT_QUEUE_SIZE;
+  }
 
-    type = f.createType(typeName, "Stats for activity in the AsyncEventQueue",
+  @Override
+  public void initializeStats(StatisticsFactory factory) {
+    type = factory.createType(typeName, "Stats for activity in the AsyncEventQueue",
         new StatisticDescriptor[] {
-            f.createIntCounter(EVENTS_RECEIVED, "Number of events received by this queue.",
+            factory.createIntCounter(EVENTS_RECEIVED, "Number of events received by this queue.",
                 "operations"),
-            f.createIntCounter(EVENTS_QUEUED, "Number of events added to the event queue.",
+            factory.createIntCounter(EVENTS_QUEUED, "Number of events added to the event queue.",
                 "operations"),
-            f.createLongCounter(EVENT_QUEUE_TIME, "Total time spent queueing events.",
+            factory.createLongCounter(EVENT_QUEUE_TIME, "Total time spent queueing events.",
                 "nanoseconds"),
-            f.createIntGauge(EVENT_QUEUE_SIZE, "Size of the event queue.", "operations", false),
-            f.createIntGauge(SECONDARY_EVENT_QUEUE_SIZE, "Size of the secondary event queue.",
+            factory.createIntGauge(EVENT_QUEUE_SIZE, "Size of the event queue.", "operations",
+                false),
+            factory.createIntGauge(SECONDARY_EVENT_QUEUE_SIZE, "Size of the secondary event queue.",
                 "operations", false),
-            f.createIntGauge(EVENTS_PROCESSED_BY_PQRM,
+            factory.createIntGauge(EVENTS_PROCESSED_BY_PQRM,
                 "Total number of events processed by Parallel Queue Removal Message(PQRM).",
                 "operations", false),
-            f.createIntGauge(TMP_EVENT_QUEUE_SIZE, "Size of the temporary events queue.",
+            factory.createIntGauge(TMP_EVENT_QUEUE_SIZE, "Size of the temporary events queue.",
                 "operations", false),
-            f.createIntCounter(EVENTS_NOT_QUEUED_CONFLATED,
+            factory.createIntCounter(EVENTS_NOT_QUEUED_CONFLATED,
                 "Number of events received but not added to the event queue because the queue already contains an event with the event's key.",
                 "operations"),
-            f.createIntCounter(EVENTS_CONFLATED_FROM_BATCHES,
+            factory.createIntCounter(EVENTS_CONFLATED_FROM_BATCHES,
                 "Number of events conflated from batches.", "operations"),
-            f.createIntCounter(EVENTS_DISTRIBUTED,
+            factory.createIntCounter(EVENTS_DISTRIBUTED,
                 "Number of events removed from the event queue and sent.", "operations"),
-            f.createIntCounter(EVENTS_EXCEEDING_ALERT_THRESHOLD,
+            factory.createIntCounter(EVENTS_EXCEEDING_ALERT_THRESHOLD,
                 "Number of events exceeding the alert threshold.", "operations", false),
-            f.createLongCounter(BATCH_DISTRIBUTION_TIME,
+            factory.createLongCounter(BATCH_DISTRIBUTION_TIME,
                 "Total time spent distributing batches of events to receivers.", "nanoseconds"),
-            f.createIntCounter(BATCHES_DISTRIBUTED,
+            factory.createIntCounter(BATCHES_DISTRIBUTED,
                 "Number of batches of events removed from the event queue and sent.", "operations"),
-            f.createIntCounter(BATCHES_REDISTRIBUTED,
+            factory.createIntCounter(BATCHES_REDISTRIBUTED,
                 "Number of batches of events removed from the event queue and resent.",
                 "operations", false),
-            f.createIntCounter(UNPROCESSED_TOKENS_ADDED_BY_PRIMARY,
+            factory.createIntCounter(UNPROCESSED_TOKENS_ADDED_BY_PRIMARY,
                 "Number of tokens added to the secondary's unprocessed token map by the primary (though a listener).",
                 "tokens"),
-            f.createIntCounter(UNPROCESSED_EVENTS_ADDED_BY_SECONDARY,
+            factory.createIntCounter(UNPROCESSED_EVENTS_ADDED_BY_SECONDARY,
                 "Number of events added to the secondary's unprocessed event map by the secondary.",
                 "events"),
-            f.createIntCounter(UNPROCESSED_EVENTS_REMOVED_BY_PRIMARY,
+            factory.createIntCounter(UNPROCESSED_EVENTS_REMOVED_BY_PRIMARY,
                 "Number of events removed from the secondary's unprocessed event map by the primary (though a listener).",
                 "events"),
-            f.createIntCounter(UNPROCESSED_TOKENS_REMOVED_BY_SECONDARY,
+            factory.createIntCounter(UNPROCESSED_TOKENS_REMOVED_BY_SECONDARY,
                 "Number of tokens removed from the secondary's unprocessed token map by the secondary.",
                 "tokens"),
-            f.createIntCounter(UNPROCESSED_EVENTS_REMOVED_BY_TIMEOUT,
+            factory.createIntCounter(UNPROCESSED_EVENTS_REMOVED_BY_TIMEOUT,
                 "Number of events removed from the secondary's unprocessed event map by a timeout.",
                 "events"),
-            f.createIntCounter(UNPROCESSED_TOKENS_REMOVED_BY_TIMEOUT,
+            factory.createIntCounter(UNPROCESSED_TOKENS_REMOVED_BY_TIMEOUT,
                 "Number of tokens removed from the secondary's unprocessed token map by a timeout.",
                 "tokens"),
-            f.createIntGauge(UNPROCESSED_EVENT_MAP_SIZE,
+            factory.createIntGauge(UNPROCESSED_EVENT_MAP_SIZE,
                 "Current number of entries in the secondary's unprocessed event map.", "events",
                 false),
-            f.createIntGauge(UNPROCESSED_TOKEN_MAP_SIZE,
+            factory.createIntGauge(UNPROCESSED_TOKEN_MAP_SIZE,
                 "Current number of entries in the secondary's unprocessed token map.", "tokens",
                 false),
-            f.createIntGauge(CONFLATION_INDEXES_MAP_SIZE,
+            factory.createIntGauge(CONFLATION_INDEXES_MAP_SIZE,
                 "Current number of entries in the conflation indexes map.", "events"),
-            f.createIntCounter(NOT_QUEUED_EVENTS, "Number of events not added to queue.", "events"),
-            f.createIntCounter(EVENTS_DROPPED_DUE_TO_PRIMARY_SENDER_NOT_RUNNING,
+            factory.createIntCounter(NOT_QUEUED_EVENTS, "Number of events not added to queue.",
+                "events"),
+            factory.createIntCounter(EVENTS_DROPPED_DUE_TO_PRIMARY_SENDER_NOT_RUNNING,
                 "Number of events dropped because the primary gateway sender is not running.",
                 "events"),
-            f.createIntCounter(EVENTS_FILTERED,
+            factory.createIntCounter(EVENTS_FILTERED,
                 "Number of events filtered through GatewayEventFilter.", "events"),
-            f.createIntCounter(LOAD_BALANCES_COMPLETED, "Number of load balances completed",
+            factory.createIntCounter(LOAD_BALANCES_COMPLETED, "Number of load balances completed",
                 "operations"),
-            f.createIntGauge(LOAD_BALANCES_IN_PROGRESS, "Number of load balances in progress",
+            factory.createIntGauge(LOAD_BALANCES_IN_PROGRESS, "Number of load balances in progress",
                 "operations"),
-            f.createLongCounter(LOAD_BALANCE_TIME, "Total time spent load balancing this sender",
+            factory.createLongCounter(LOAD_BALANCE_TIME,
+                "Total time spent load balancing this sender",
                 "nanoseconds"),
-            f.createIntCounter(SYNCHRONIZATION_EVENTS_ENQUEUED,
+            factory.createIntCounter(SYNCHRONIZATION_EVENTS_ENQUEUED,
                 "Number of synchronization events added to the event queue.", "operations"),
-            f.createIntCounter(SYNCHRONIZATION_EVENTS_PROVIDED,
+            factory.createIntCounter(SYNCHRONIZATION_EVENTS_PROVIDED,
                 "Number of synchronization events provided to other members.", "operations"),});
 
     // Initialize id fields
@@ -146,11 +161,19 @@ public class AsyncEventQueueStats extends GatewaySenderStats {
   /**
    * Constructor.
    *
-   * @param f The <code>StatisticsFactory</code> which creates the <code>Statistics</code> instance
-   * @param asyncQueueId The id of the <code>AsyncEventQueue</code> used to generate the name of the
+   * @param factory The <code>StatisticsFactory</code> which creates the <code>Statistics</code>
+   *        instance
+   * @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 factory, String asyncQueueId) {
+    super();
+    initializeStats(factory);
+    this.stats = factory.createAtomicStatistics(type, "asyncEventQueueStats-" + asyncQueueId);
+  }
+
+  public StatisticsType getType() {
+    return type;
   }
 }
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 863cfdb..42e3c5f 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
@@ -50,8 +50,9 @@ public class ParallelAsyncEventQueueImpl extends AbstractGatewaySender {
     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(cache.getDistributedSystem().getStatisticsFactory(),
+              AsyncEventQueueImpl.getAsyncEventQueueIdFromSenderId(id));
     }
     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 05c2f77..a076d78 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
@@ -52,8 +52,9 @@ public class SerialAsyncEventQueueImpl extends AbstractGatewaySender {
     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(cache.getDistributedSystem().getStatisticsFactory(),
+              AsyncEventQueueImpl.getAsyncEventQueueIdFromSenderId(id));
     }
   }
 
diff --git a/geode-core/src/main/java/org/apache/geode/cache/client/internal/AbstractOp.java b/geode-core/src/main/java/org/apache/geode/cache/client/internal/AbstractOp.java
index 3c24f83..7f6575b 100644
--- a/geode-core/src/main/java/org/apache/geode/cache/client/internal/AbstractOp.java
+++ b/geode-core/src/main/java/org/apache/geode/cache/client/internal/AbstractOp.java
@@ -34,6 +34,7 @@ import org.apache.geode.internal.cache.tier.sockets.Part;
 import org.apache.geode.internal.cache.tier.sockets.ServerConnection;
 import org.apache.geode.internal.logging.LogService;
 import org.apache.geode.internal.logging.log4j.LogMarker;
+import org.apache.geode.stats.common.cache.client.internal.ConnectionStats;
 
 /**
  * Represents an operation that can be performed in a client by sending a message to a server.
diff --git a/geode-core/src/main/java/org/apache/geode/cache/client/internal/AddPDXEnumOp.java b/geode-core/src/main/java/org/apache/geode/cache/client/internal/AddPDXEnumOp.java
index c926358..972cf6a 100644
--- a/geode-core/src/main/java/org/apache/geode/cache/client/internal/AddPDXEnumOp.java
+++ b/geode-core/src/main/java/org/apache/geode/cache/client/internal/AddPDXEnumOp.java
@@ -17,6 +17,7 @@ package org.apache.geode.cache.client.internal;
 import org.apache.geode.internal.cache.tier.MessageType;
 import org.apache.geode.internal.cache.tier.sockets.Message;
 import org.apache.geode.pdx.internal.EnumInfo;
+import org.apache.geode.stats.common.cache.client.internal.ConnectionStats;
 
 /**
  * Push a PDX Enum id to other servers.
diff --git a/geode-core/src/main/java/org/apache/geode/cache/client/internal/AddPDXTypeOp.java b/geode-core/src/main/java/org/apache/geode/cache/client/internal/AddPDXTypeOp.java
index de1e5cb..03ce2f9 100644
--- a/geode-core/src/main/java/org/apache/geode/cache/client/internal/AddPDXTypeOp.java
+++ b/geode-core/src/main/java/org/apache/geode/cache/client/internal/AddPDXTypeOp.java
@@ -17,6 +17,7 @@ package org.apache.geode.cache.client.internal;
 import org.apache.geode.internal.cache.tier.MessageType;
 import org.apache.geode.internal.cache.tier.sockets.Message;
 import org.apache.geode.pdx.internal.PdxType;
+import org.apache.geode.stats.common.cache.client.internal.ConnectionStats;
 
 /**
  * Add a PdxType to a server.
diff --git a/geode-core/src/main/java/org/apache/geode/cache/client/internal/AuthenticateUserOp.java b/geode-core/src/main/java/org/apache/geode/cache/client/internal/AuthenticateUserOp.java
index 35ef898..bdec502 100644
--- a/geode-core/src/main/java/org/apache/geode/cache/client/internal/AuthenticateUserOp.java
+++ b/geode-core/src/main/java/org/apache/geode/cache/client/internal/AuthenticateUserOp.java
@@ -40,6 +40,7 @@ import org.apache.geode.internal.logging.InternalLogWriter;
 import org.apache.geode.security.AuthenticationFailedException;
 import org.apache.geode.security.AuthenticationRequiredException;
 import org.apache.geode.security.NotAuthorizedException;
+import org.apache.geode.stats.common.cache.client.internal.ConnectionStats;
 
 /**
  * Authenticates this client (or a user) on a server. This op ideally should get executed
diff --git a/geode-core/src/main/java/org/apache/geode/cache/client/internal/ClearOp.java b/geode-core/src/main/java/org/apache/geode/cache/client/internal/ClearOp.java
index 0c1e059..b106868 100644
--- a/geode-core/src/main/java/org/apache/geode/cache/client/internal/ClearOp.java
+++ b/geode-core/src/main/java/org/apache/geode/cache/client/internal/ClearOp.java
@@ -17,6 +17,7 @@ package org.apache.geode.cache.client.internal;
 import org.apache.geode.internal.cache.EventID;
 import org.apache.geode.internal.cache.tier.MessageType;
 import org.apache.geode.internal.cache.tier.sockets.Message;
+import org.apache.geode.stats.common.cache.client.internal.ConnectionStats;
 
 /**
  * Does a region clear (or create) on a server
diff --git a/geode-core/src/main/java/org/apache/geode/cache/client/internal/CloseConnectionOp.java b/geode-core/src/main/java/org/apache/geode/cache/client/internal/CloseConnectionOp.java
index 987ea74..cc8b11d 100755
--- a/geode-core/src/main/java/org/apache/geode/cache/client/internal/CloseConnectionOp.java
+++ b/geode-core/src/main/java/org/apache/geode/cache/client/internal/CloseConnectionOp.java
@@ -18,6 +18,7 @@ import java.io.EOFException;
 
 import org.apache.geode.internal.cache.tier.MessageType;
 import org.apache.geode.internal.cache.tier.sockets.Message;
+import org.apache.geode.stats.common.cache.client.internal.ConnectionStats;
 
 /**
  * Tell a server that a connection is being closed
diff --git a/geode-core/src/main/java/org/apache/geode/cache/client/internal/CommitOp.java b/geode-core/src/main/java/org/apache/geode/cache/client/internal/CommitOp.java
index c961848..23277e9 100644
--- a/geode-core/src/main/java/org/apache/geode/cache/client/internal/CommitOp.java
+++ b/geode-core/src/main/java/org/apache/geode/cache/client/internal/CommitOp.java
@@ -17,6 +17,7 @@ package org.apache.geode.cache.client.internal;
 import org.apache.geode.internal.cache.TXCommitMessage;
 import org.apache.geode.internal.cache.tier.MessageType;
 import org.apache.geode.internal.cache.tier.sockets.Message;
+import org.apache.geode.stats.common.cache.client.internal.ConnectionStats;
 
 /**
  * Does a commit on a server
diff --git a/geode-core/src/main/java/org/apache/geode/cache/client/internal/Connection.java b/geode-core/src/main/java/org/apache/geode/cache/client/internal/Connection.java
index 46794df..5b4d392 100644
--- a/geode-core/src/main/java/org/apache/geode/cache/client/internal/Connection.java
+++ b/geode-core/src/main/java/org/apache/geode/cache/client/internal/Connection.java
@@ -22,6 +22,7 @@ import java.nio.ByteBuffer;
 
 import org.apache.geode.distributed.internal.ServerLocation;
 import org.apache.geode.internal.cache.tier.sockets.ServerQueueStatus;
+import org.apache.geode.stats.common.cache.client.internal.ConnectionStats;
 
 /**
  * Represents a connection from a client to a server. Instances are created, kept, and used by
diff --git a/geode-core/src/main/java/org/apache/geode/cache/client/internal/ConnectionImpl.java b/geode-core/src/main/java/org/apache/geode/cache/client/internal/ConnectionImpl.java
index 26de256..5d0aa27 100644
--- a/geode-core/src/main/java/org/apache/geode/cache/client/internal/ConnectionImpl.java
+++ b/geode-core/src/main/java/org/apache/geode/cache/client/internal/ConnectionImpl.java
@@ -44,6 +44,7 @@ import org.apache.geode.internal.i18n.LocalizedStrings;
 import org.apache.geode.internal.logging.LogService;
 import org.apache.geode.internal.logging.log4j.LocalizedMessage;
 import org.apache.geode.internal.net.SocketCreator;
+import org.apache.geode.stats.common.cache.client.internal.ConnectionStats;
 
 /**
  * A single client to server connection.
diff --git a/geode-core/src/main/java/org/apache/geode/cache/client/internal/ConnectionStats.java b/geode-core/src/main/java/org/apache/geode/cache/client/internal/ConnectionStats.java
deleted file mode 100644
index 20b00f1..0000000
--- a/geode-core/src/main/java/org/apache/geode/cache/client/internal/ConnectionStats.java
+++ /dev/null
@@ -1,3787 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
- * agreements. See the NOTICE file distributed with this work for additional information regarding
- * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License. You may obtain a
- * copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software distributed under the License
- * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
- * or implied. See the License for the specific language governing permissions and limitations under
- * the License.
- */
-package org.apache.geode.cache.client.internal;
-
-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.distributed.internal.DistributionStats;
-import org.apache.geode.internal.cache.PoolStats;
-import org.apache.geode.internal.cache.tier.sockets.MessageStats;
-import org.apache.geode.internal.statistics.StatisticsTypeFactoryImpl;
-
-/**
- * Stats for a client to server {@link Connection}
- *
- * @since GemFire 5.7
- */
-public class ConnectionStats implements MessageStats {
-  // static fields
-  private static final StatisticsType type;
-  private static final StatisticsType sendType;
-
-  ///////////////////////////////////////////////////////////////////////
-  /*
-   * private static final int opInProgressId; private static final int opSendInProgressId; private
-   * static final int opSendFailedId; private static final int opSendId; private static final int
-   * opSendDurationId; private static final int opTimedOutId; private static final int opFailedId;
-   * private static final int opId; private static final int opDurationId;
-   */
-  ///////////////////////////////////////////////////////////////////////
-
-  private static final int getInProgressId;
-  private static final int getSendInProgressId;
-  private static final int getSendFailedId;
-  private static final int getSendId;
-  private static final int getSendDurationId;
-  private static final int getTimedOutId;
-  private static final int getFailedId;
-  private static final int getId;
-  private static final int getDurationId;
-
-  private static final int putInProgressId;
-  private static final int putSendInProgressId;
-  private static final int putSendFailedId;
-  private static final int putSendId;
-  private static final int putSendDurationId;
-  private static final int putTimedOutId;
-  private static final int putFailedId;
-  private static final int putId;
-  private static final int putDurationId;
-
-  private static final int destroyInProgressId;
-  private static final int destroySendInProgressId;
-  private static final int destroySendFailedId;
-  private static final int destroySendId;
-  private static final int destroySendDurationId;
-  private static final int destroyTimedOutId;
-  private static final int destroyFailedId;
-  private static final int destroyId;
-  private static final int destroyDurationId;
-
-  private static final int destroyRegionInProgressId;
-  private static final int destroyRegionSendInProgressId;
-  private static final int destroyRegionSendFailedId;
-  private static final int destroyRegionSendId;
-  private static final int destroyRegionSendDurationId;
-  private static final int destroyRegionTimedOutId;
-  private static final int destroyRegionFailedId;
-  private static final int destroyRegionId;
-  private static final int destroyRegionDurationId;
-
-  private static final int clearInProgressId;
-  private static final int clearSendInProgressId;
-  private static final int clearSendFailedId;
-  private static final int clearSendId;
-  private static final int clearSendDurationId;
-  private static final int clearTimedOutId;
-  private static final int clearFailedId;
-  private static final int clearId;
-  private static final int clearDurationId;
-
-  private static final int containsKeyInProgressId;
-  private static final int containsKeySendInProgressId;
-  private static final int containsKeySendFailedId;
-  private static final int containsKeySendId;
-  private static final int containsKeySendDurationId;
-  private static final int containsKeyTimedOutId;
-  private static final int containsKeyFailedId;
-  private static final int containsKeyId;
-  private static final int containsKeyDurationId;
-
-  private static final int keySetInProgressId;
-  private static final int keySetSendInProgressId;
-  private static final int keySetSendFailedId;
-  private static final int keySetSendId;
-  private static final int keySetSendDurationId;
-  private static final int keySetTimedOutId;
-  private static final int keySetFailedId;
-  private static final int keySetId;
-  private static final int keySetDurationId;
-
-  private static final int commitInProgressId;
-  private static final int commitSendInProgressId;
-  private static final int commitSendFailedId;
-  private static final int commitSendId;
-  private static final int commitSendDurationId;
-
-  private static final int commitFailedId;
-  private static final int commitTimedOutId;
-  private static final int commitId;
-  private static final int commitDurationId;
-
-  private static final int rollbackInProgressId;
-  private static final int rollbackSendInProgressId;
-  private static final int rollbackSendFailedId;
-  private static final int rollbackSendId;
-  private static final int rollbackSendDurationId;
-
-  private static final int rollbackFailedId;
-  private static final int rollbackTimedOutId;
-  private static final int rollbackId;
-  private static final int rollbackDurationId;
-
-  private static final int getEntryInProgressId;
-  private static final int getEntrySendInProgressId;
-  private static final int getEntrySendFailedId;
-  private static final int getEntrySendId;
-  private static final int getEntrySendDurationId;
-
-  private static final int getEntryFailedId;
-  private static final int getEntryTimedOutId;
-  private static final int getEntryId;
-  private static final int getEntryDurationId;
-
-  private static final int txSynchronizationInProgressId;
-  private static final int txSynchronizationSendInProgressId;
-  private static final int txSynchronizationSendFailedId;
-  private static final int txSynchronizationSendId;
-  private static final int txSynchronizationSendDurationId;
-
-  private static final int txSynchronizationFailedId;
-  private static final int txSynchronizationTimedOutId;
-  private static final int txSynchronizationId;
-  private static final int txSynchronizationDurationId;
-
-  private static final int txFailoverInProgressId;
-  private static final int txFailoverSendInProgressId;
-  private static final int txFailoverSendFailedId;
-  private static final int txFailoverSendId;
-  private static final int txFailoverSendDurationId;
-
-  private static final int txFailoverFailedId;
-  private static final int txFailoverTimedOutId;
-  private static final int txFailoverId;
-  private static final int txFailoverDurationId;
-
-  private static final int sizeInProgressId;
-  private static final int sizeSendInProgressId;
-  private static final int sizeSendFailedId;
-  private static final int sizeSendId;
-  private static final int sizeSendDurationId;
-
-  private static final int sizeFailedId;
-  private static final int sizeTimedOutId;
-  private static final int sizeId;
-  private static final int sizeDurationId;
-
-  private static final int invalidateInProgressId;
-  private static final int invalidateSendInProgressId;
-  private static final int invalidateSendFailedId;
-  private static final int invalidateSendId;
-  private static final int invalidateSendDurationId;
-
-  private static final int invalidateFailedId;
-  private static final int invalidateTimedOutId;
-  private static final int invalidateId;
-  private static final int invalidateDurationId;
-
-
-  private static final int registerInterestInProgressId;
-  private static final int registerInterestSendInProgressId;
-  private static final int registerInterestSendFailedId;
-  private static final int registerInterestSendId;
-  private static final int registerInterestSendDurationId;
-  private static final int registerInterestTimedOutId;
-  private static final int registerInterestFailedId;
-  private static final int registerInterestId;
-  private static final int registerInterestDurationId;
-
-  private static final int unregisterInterestInProgressId;
-  private static final int unregisterInterestSendInProgressId;
-  private static final int unregisterInterestSendFailedId;
-  private static final int unregisterInterestSendId;
-  private static final int unregisterInterestSendDurationId;
-  private static final int unregisterInterestTimedOutId;
-  private static final int unregisterInterestFailedId;
-  private static final int unregisterInterestId;
-  private static final int unregisterInterestDurationId;
-
-  private static final int queryInProgressId;
-  private static final int querySendInProgressId;
-  private static final int querySendFailedId;
-  private static final int querySendId;
-  private static final int querySendDurationId;
-  private static final int queryTimedOutId;
-  private static final int queryFailedId;
-  private static final int queryId;
-  private static final int queryDurationId;
-
-  private static final int createCQInProgressId;
-  private static final int createCQSendInProgressId;
-  private static final int createCQSendFailedId;
-  private static final int createCQSendId;
-  private static final int createCQSendDurationId;
-  private static final int createCQTimedOutId;
-  private static final int createCQFailedId;
-  private static final int createCQId;
-  private static final int createCQDurationId;
-  private static final int stopCQInProgressId;
-  private static final int stopCQSendInProgressId;
-  private static final int stopCQSendFailedId;
-  private static final int stopCQSendId;
-  private static final int stopCQSendDurationId;
-  private static final int stopCQTimedOutId;
-  private static final int stopCQFailedId;
-  private static final int stopCQId;
-  private static final int stopCQDurationId;
-  private static final int closeCQInProgressId;
-  private static final int closeCQSendInProgressId;
-  private static final int closeCQSendFailedId;
-  private static final int closeCQSendId;
-  private static final int closeCQSendDurationId;
-  private static final int closeCQTimedOutId;
-  private static final int closeCQFailedId;
-  private static final int closeCQId;
-  private static final int closeCQDurationId;
-  private static final int gatewayBatchInProgressId;
-  private static final int gatewayBatchSendInProgressId;
-  private static final int gatewayBatchSendFailedId;
-  private static final int gatewayBatchSendId;
-  private static final int gatewayBatchSendDurationId;
-  private static final int gatewayBatchTimedOutId;
-  private static final int gatewayBatchFailedId;
-  private static final int gatewayBatchId;
-  private static final int gatewayBatchDurationId;
-  private static final int getDurableCQsInProgressId;
-  private static final int getDurableCQsSendsInProgressId;
-  private static final int getDurableCQsSendFailedId;
-  private static final int getDurableCQsSendId;
-  private static final int getDurableCQsSendDurationId;
-  private static final int getDurableCQsTimedOutId;
-  private static final int getDurableCQsFailedId;
-  private static final int getDurableCQsId;
-  private static final int getDurableCQsDurationId;
-
-  private static final int readyForEventsInProgressId;
-  private static final int readyForEventsSendInProgressId;
-  private static final int readyForEventsSendFailedId;
-  private static final int readyForEventsSendId;
-  private static final int readyForEventsSendDurationId;
-  private static final int readyForEventsTimedOutId;
-  private static final int readyForEventsFailedId;
-  private static final int readyForEventsId;
-  private static final int readyForEventsDurationId;
-
-  private static final int makePrimaryInProgressId;
-  private static final int makePrimarySendInProgressId;
-  private static final int makePrimarySendFailedId;
-  private static final int makePrimarySendId;
-  private static final int makePrimarySendDurationId;
-  private static final int makePrimaryTimedOutId;
-  private static final int makePrimaryFailedId;
-  private static final int makePrimaryId;
-  private static final int makePrimaryDurationId;
-
-  private static final int closeConInProgressId;
-  private static final int closeConSendInProgressId;
-  private static final int closeConSendFailedId;
-  private static final int closeConSendId;
-  private static final int closeConSendDurationId;
-  private static final int closeConTimedOutId;
-  private static final int closeConFailedId;
-  private static final int closeConId;
-  private static final int closeConDurationId;
-
-  private static final int primaryAckInProgressId;
-  private static final int primaryAckSendInProgressId;
-  private static final int primaryAckSendFailedId;
-  private static final int primaryAckSendId;
-  private static final int primaryAckSendDurationId;
-  private static final int primaryAckTimedOutId;
-  private static final int primaryAckFailedId;
-  private static final int primaryAckId;
-  private static final int primaryAckDurationId;
-
-  private static final int pingInProgressId;
-  private static final int pingSendInProgressId;
-  private static final int pingSendFailedId;
-  private static final int pingSendId;
-  private static final int pingSendDurationId;
-  private static final int pingTimedOutId;
-  private static final int pingFailedId;
-  private static final int pingId;
-  private static final int pingDurationId;
-
-  private static final int registerInstantiatorsInProgressId;
-  private static final int registerInstantiatorsSendInProgressId;
-  private static final int registerInstantiatorsSendFailedId;
-  private static final int registerInstantiatorsSendId;
-  private static final int registerInstantiatorsSendDurationId;
-  private static final int registerInstantiatorsTimedOutId;
-  private static final int registerInstantiatorsFailedId;
-  private static final int registerInstantiatorsId;
-  private static final int registerInstantiatorsDurationId;
-
-  private static final int registerDataSerializersInProgressId;
-  private static final int registerDataSerializersSendInProgressId;
-  private static final int registerDataSerializersSendFailedId;
-  private static final int registerDataSerializersSendId;
-  private static final int registerDataSerializersSendDurationId;
-  private static final int registerDataSerializersTimedOutId;
-  private static final int registerDataSerializersFailedId;
-  private static final int registerDataSerializersId;
-  private static final int registerDataSerializersDurationId;
-
-  private static final int putAllInProgressId;
-  private static final int putAllSendInProgressId;
-  private static final int putAllSendFailedId;
-  private static final int putAllSendId;
-  private static final int putAllSendDurationId;
-  private static final int putAllTimedOutId;
-  private static final int putAllFailedId;
-  private static final int putAllId;
-  private static final int putAllDurationId;
-
-  private static final int removeAllInProgressId;
-  private static final int removeAllSendInProgressId;
-  private static final int removeAllSendFailedId;
-  private static final int removeAllSendId;
-  private static final int removeAllSendDurationId;
-  private static final int removeAllTimedOutId;
-  private static final int removeAllFailedId;
-  private static final int removeAllId;
-  private static final int removeAllDurationId;
-
-  private static final int getAllInProgressId;
-  private static final int getAllSendInProgressId;
-  private static final int getAllSendFailedId;
-  private static final int getAllSendId;
-  private static final int getAllSendDurationId;
-  private static final int getAllTimedOutId;
-  private static final int getAllFailedId;
-  private static final int getAllId;
-  private static final int getAllDurationId;
-
-  private static final int connectionsId;
-  private static final int connectsId;
-  private static final int disconnectsId;
-  private static final int messagesBeingReceivedId;
-  private static final int messageBytesBeingReceivedId;
-  private static final int receivedBytesId;
-  private static final int sentBytesId;
-
-  private static final int executeFunctionInProgressId;
-  private static final int executeFunctionSendInProgressId;
-  private static final int executeFunctionSendFailedId;
-  private static final int executeFunctionSendId;
-  private static final int executeFunctionSendDurationId;
-  private static final int executeFunctionTimedOutId;
-  private static final int executeFunctionFailedId;
-  private static final int executeFunctionId;
-  private static final int executeFunctionDurationId;
-
-  private static final int getClientPRMetadataInProgressId;
-  private static final int getClientPRMetadataSendInProgressId;
-  private static final int getClientPRMetadataSendFailedId;
-  private static final int getClientPRMetadataSendId;
-  private static final int getClientPRMetadataSendDurationId;
-  private static final int getClientPRMetadataTimedOutId;
-  private static final int getClientPRMetadataFailedId;
-  private static final int getClientPRMetadataId;
-  private static final int getClientPRMetadataDurationId;
-
-  private static final int getClientPartitionAttributesInProgressId;
-  private static final int getClientPartitionAttributesSendInProgressId;
-  private static final int getClientPartitionAttributesSendFailedId;
-  private static final int getClientPartitionAttributesSendId;
-  private static final int getClientPartitionAttributesSendDurationId;
-  private static final int getClientPartitionAttributesTimedOutId;
-  private static final int getClientPartitionAttributesFailedId;
-  private static final int getClientPartitionAttributesId;
-  private static final int getClientPartitionAttributesDurationId;
-
-  private static final int getPDXIdForTypeInProgressId;
-  private static final int getPDXIdForTypeSendInProgressId;
-  private static final int getPDXIdForTypeSendFailedId;
-  private static final int getPDXIdForTypeSendId;
-  private static final int getPDXIdForTypeSendDurationId;
-  private static final int getPDXIdForTypeTimedOutId;
-  private static final int getPDXIdForTypeFailedId;
-  private static final int getPDXIdForTypeId;
-  private static final int getPDXIdForTypeDurationId;
-
-  private static final int getPDXTypeByIdInProgressId;
-  private static final int getPDXTypeByIdSendInProgressId;
-  private static final int getPDXTypeByIdSendFailedId;
-  private static final int getPDXTypeByIdSendId;
-  private static final int getPDXTypeByIdSendDurationId;
-  private static final int getPDXTypeByIdTimedOutId;
-  private static final int getPDXTypeByIdFailedId;
-  private static final int getPDXTypeByIdId;
-  private static final int getPDXTypeByIdDurationId;
-
-  private static final int addPdxTypeInProgressId;
-  private static final int addPdxTypeSendInProgressId;
-  private static final int addPdxTypeSendFailedId;
-  private static final int addPdxTypeSendId;
-  private static final int addPdxTypeSendDurationId;
-  private static final int addPdxTypeTimedOutId;
-  private static final int addPdxTypeFailedId;
-  private static final int addPdxTypeId;
-  private static final int addPdxTypeDurationId;
-
-
-  // An array of all of the ids that represent operation statistics. This
-  // is used by the getOps method to aggregate the individual stats
-  // into a total value for all operations.
-  private static int[] opIds;
-
-  static {
-    try {
-      StatisticsTypeFactory f = StatisticsTypeFactoryImpl.singleton();
-      type = f.createType("ClientStats", "Statistics about client to server communication",
-          new StatisticDescriptor[] {
-              ///////////////////////////////////////////////////////////////////////
-              /*
-               * f.createIntGauge("opsInProgress", "Current number of ops being executed", "ops"),
-               * f.createIntCounter("ops", "Total number of ops completed successfully", "ops"),
-               * f.createIntCounter("opFailures", "Total number of op attempts that have failed",
-               * "ops"), f.createIntCounter("opTimeouts",
-               * "Total number of op attempts that have timed out", "ops"),
-               * f.createLongCounter("opTime",
-               * "Total amount of time, in nanoseconds spent doing ops", "nanoseconds"),
-               */
-              ///////////////////////////////////////////////////////////////////////
-              f.createIntGauge("getsInProgress", "Current number of gets being executed", "gets"),
-              f.createIntCounter("gets", "Total number of gets completed successfully", "gets"),
-              f.createIntCounter("getFailures", "Total number of get attempts that have failed",
-                  "gets"),
-              f.createIntCounter("getTimeouts", "Total number of get attempts that have timed out",
-                  "gets"),
-              f.createLongCounter("getTime",
-                  "Total amount of time, in nanoseconds spent doing gets", "nanoseconds"),
-              f.createIntGauge("putsInProgress", "Current number of puts being executed", "puts"),
-              f.createIntCounter("puts", "Total number of puts completed successfully", "puts"),
-              f.createIntCounter("putFailures", "Total number of put attempts that have failed",
-                  "puts"),
-              f.createIntCounter("putTimeouts", "Total number of put attempts that have timed out",
-                  "puts"),
-              f.createLongCounter("putTime",
-                  "Total amount of time, in nanoseconds spent doing puts", "nanoseconds"),
-              f.createIntGauge("destroysInProgress", "Current number of destroys being executed",
-                  "destroys"),
-              f.createIntCounter("destroys", "Total number of destroys completed successfully",
-                  "destroys"),
-              f.createIntCounter("destroyFailures",
-                  "Total number of destroy attempts that have failed", "destroys"),
-              f.createIntCounter("destroyTimeouts",
-                  "Total number of destroy attempts that have timed out", "destroys"),
-              f.createLongCounter("destroyTime",
-                  "Total amount of time, in nanoseconds spent doing destroys", "nanoseconds"),
-              f.createIntGauge("destroyRegionsInProgress",
-                  "Current number of destroyRegions being executed", "destroyRegions"),
-              f.createIntCounter("destroyRegions",
-                  "Total number of destroyRegions completed successfully", "destroyRegions"),
-              f.createIntCounter("destroyRegionFailures",
-                  "Total number of destroyRegion attempts that have failed", "destroyRegions"),
-              f.createIntCounter("destroyRegionTimeouts",
-                  "Total number of destroyRegion attempts that have timed out", "destroyRegions"),
-              f.createLongCounter("destroyRegionTime",
-                  "Total amount of time, in nanoseconds spent doing destroyRegions", "nanoseconds"),
-              f.createIntGauge("clearsInProgress", "Current number of clears being executed",
-                  "clears"),
-              f.createIntCounter("clears", "Total number of clears completed successfully",
-                  "clears"),
-              f.createIntCounter("clearFailures", "Total number of clear attempts that have failed",
-                  "clears"),
-              f.createIntCounter("clearTimeouts",
-                  "Total number of clear attempts that have timed out", "clears"),
-              f.createLongCounter("clearTime",
-                  "Total amount of time, in nanoseconds spent doing clears", "nanoseconds"),
-              f.createIntGauge("containsKeysInProgress",
-                  "Current number of containsKeys being executed", "containsKeys"),
-              f.createIntCounter("containsKeys",
-                  "Total number of containsKeys completed successfully", "containsKeys"),
-              f.createIntCounter("containsKeyFailures",
-                  "Total number of containsKey attempts that have failed", "containsKeys"),
-              f.createIntCounter("containsKeyTimeouts",
-                  "Total number of containsKey attempts that have timed out", "containsKeys"),
-              f.createLongCounter("containsKeyTime",
-                  "Total amount of time, in nanoseconds spent doing containsKeys", "nanoseconds"),
-              f.createIntGauge("keySetsInProgress", "Current number of keySets being executed",
-                  "keySets"),
-              f.createIntCounter("keySets", "Total number of keySets completed successfully",
-                  "keySets"),
-              f.createIntCounter("keySetFailures",
-                  "Total number of keySet attempts that have failed", "keySets"),
-              f.createIntCounter("keySetTimeouts",
-                  "Total number of keySet attempts that have timed out", "keySets"),
-              f.createLongCounter("keySetTime",
-                  "Total amount of time, in nanoseconds spent doing keySets", "nanoseconds"),
-
-              f.createIntGauge("commitsInProgress", "Current number of commits being executed",
-                  "commits"),
-              f.createIntCounter("commits", "Total number of commits completed successfully",
-                  "commits"),
-              f.createIntCounter("commitFailures",
-                  "Total number of commit attempts that have failed", "commits"),
-              f.createIntCounter("commitTimeouts",
-                  "Total number of commit attempts that have timed out", "commits"),
-              f.createLongCounter("commitTime",
-                  "Total amount of time, in nanoseconds spent doing commits", "nanoseconds"),
-
-              f.createIntGauge("rollbacksInProgress", "Current number of rollbacks being executed",
-                  "rollbacks"),
-              f.createIntCounter("rollbacks", "Total number of rollbacks completed successfully",
-                  "rollbacks"),
-              f.createIntCounter("rollbackFailures",
-                  "Total number of rollback attempts that have failed", "rollbacks"),
-              f.createIntCounter("rollbackTimeouts",
-                  "Total number of rollback attempts that have timed out", "rollbacks"),
-              f.createLongCounter("rollbackTime",
-                  "Total amount of time, in nanoseconds spent doing rollbacks", "nanoseconds"),
-
-              f.createIntGauge("getEntrysInProgress",
-                  "Current number of getEntry messages being executed", "messages"),
-              f.createIntCounter("getEntrys",
-                  "Total number of getEntry messages completed successfully", "messages"),
-              f.createIntCounter("getEntryFailures",
-                  "Total number of getEntry attempts that have failed", "attempts"),
-              f.createIntCounter("getEntryTimeouts",
-                  "Total number of getEntry attempts that have timed out", "attempts"),
-              f.createLongCounter("getEntryTime",
-                  "Total amount of time, in nanoseconds spent doing getEntry processings",
-                  "nanoseconds"),
-
-              f.createIntGauge("jtaSynchronizationsInProgress",
-                  "Current number of jtaSynchronizations being executed", "sizes"),
-              f.createIntCounter("jtaSynchronizations",
-                  "Total number of jtaSynchronizations completed successfully",
-                  "jtaSynchronizations"),
-              f.createIntCounter("jtaSynchronizationFailures",
-                  "Total number of jtaSynchronization attempts that have failed",
-                  "jtaSynchronizations"),
-              f.createIntCounter("jtaSynchronizationTimeouts",
-                  "Total number of jtaSynchronization attempts that have timed out",
-                  "jtaSynchronizations"),
-              f.createLongCounter("jtaSynchronizationTime",
-                  "Total amount of time, in nanoseconds spent doing jtaSynchronizations",
-                  "nanoseconds"),
-
-              f.createIntGauge("txFailoversInProgress",
-                  "Current number of txFailovers being executed", "txFailovers"),
-              f.createIntCounter("txFailovers",
-                  "Total number of txFailovers completed successfully", "txFailovers"),
-              f.createIntCounter("txFailoverFailures",
-                  "Total number of txFailover attempts that have failed", "txFailovers"),
-              f.createIntCounter("txFailoverTimeouts",
-                  "Total number of txFailover attempts that have timed out", "sizes"),
-              f.createLongCounter("txFailoverTime",
-                  "Total amount of time, in nanoseconds spent doing txFailovers", "nanoseconds"),
-
-              f.createIntGauge("sizesInProgress", "Current number of sizes being executed",
-                  "sizes"),
-              f.createIntCounter("sizes", "Total number of sizes completed successfully", "sizes"),
-              f.createIntCounter("sizeFailures", "Total number of size attempts that have failed",
-                  "sizes"),
-              f.createIntCounter("sizeTimeouts",
-                  "Total number of size attempts that have timed out", "sizes"),
-              f.createLongCounter("sizeTime",
-                  "Total amount of time, in nanoseconds spent doing sizes", "nanoseconds"),
-
-              f.createIntGauge("invalidatesInProgress",
-                  "Current number of invalidates being executed", "invalidates"),
-              f.createIntCounter("invalidates",
-                  "Total number of invalidates completed successfully", "invalidates"),
-              f.createIntCounter("invalidateFailures",
-                  "Total number of invalidate attempts that have failed", "invalidates"),
-              f.createIntCounter("invalidateTimeouts",
-                  "Total number of invalidate attempts that have timed out", "invalidates"),
-              f.createLongCounter("invalidateTime",
-                  "Total amount of time, in nanoseconds spent doing invalidates", "nanoseconds"),
-
-
-              f.createIntGauge("registerInterestsInProgress",
-                  "Current number of registerInterests being executed", "registerInterests"),
-              f.createIntCounter("registerInterests",
-                  "Total number of registerInterests completed successfully", "registerInterests"),
-              f.createIntCounter("registerInterestFailures",
-                  "Total number of registerInterest attempts that have failed",
-                  "registerInterests"),
-              f.createIntCounter("registerInterestTimeouts",
-                  "Total number of registerInterest attempts that have timed out",
-                  "registerInterests"),
-              f.createLongCounter("registerInterestTime",
-                  "Total amount of time, in nanoseconds spent doing registerInterests",
-                  "nanoseconds"),
-              f.createIntGauge("unregisterInterestsInProgress",
-                  "Current number of unregisterInterests being executed", "unregisterInterests"),
-              f.createIntCounter("unregisterInterests",
-                  "Total number of unregisterInterests completed successfully",
-                  "unregisterInterests"),
-              f.createIntCounter("unregisterInterestFailures",
-                  "Total number of unregisterInterest attempts that have failed",
-                  "unregisterInterests"),
-              f.createIntCounter("unregisterInterestTimeouts",
-                  "Total number of unregisterInterest attempts that have timed out",
-                  "unregisterInterests"),
-              f.createLongCounter("unregisterInterestTime",
-                  "Total amount of time, in nanoseconds spent doing unregisterInterests",
-                  "nanoseconds"),
-              f.createIntGauge("querysInProgress", "Current number of querys being executed",
-                  "querys"),
-              f.createIntCounter("querys", "Total number of querys completed successfully",
-                  "querys"),
-              f.createIntCounter("queryFailures", "Total number of query attempts that have failed",
-                  "querys"),
-              f.createIntCounter("queryTimeouts",
-                  "Total number of query attempts that have timed out", "querys"),
-              f.createLongCounter("queryTime",
-                  "Total amount of time, in nanoseconds spent doing querys", "nanoseconds"),
-              f.createIntGauge("createCQsInProgress", "Current number of createCQs being executed",
-                  "createCQs"),
-              f.createIntCounter("createCQs", "Total number of createCQs completed successfully",
-                  "createCQs"),
-              f.createIntCounter("createCQFailures",
-                  "Total number of createCQ attempts that have failed", "createCQs"),
-              f.createIntCounter("createCQTimeouts",
-                  "Total number of createCQ attempts that have timed out", "createCQs"),
-              f.createLongCounter("createCQTime",
-                  "Total amount of time, in nanoseconds spent doing createCQs", "nanoseconds"),
-              f.createIntGauge("stopCQsInProgress", "Current number of stopCQs being executed",
-                  "stopCQs"),
-              f.createIntCounter("stopCQs", "Total number of stopCQs completed successfully",
-                  "stopCQs"),
-              f.createIntCounter("stopCQFailures",
-                  "Total number of stopCQ attempts that have failed", "stopCQs"),
-              f.createIntCounter("stopCQTimeouts",
-                  "Total number of stopCQ attempts that have timed out", "stopCQs"),
-              f.createLongCounter("stopCQTime",
-                  "Total amount of time, in nanoseconds spent doing stopCQs", "nanoseconds"),
-              f.createIntGauge("closeCQsInProgress", "Current number of closeCQs being executed",
-                  "closeCQs"),
-              f.createIntCounter("closeCQs", "Total number of closeCQs completed successfully",
-                  "closeCQs"),
-              f.createIntCounter("closeCQFailures",
-                  "Total number of closeCQ attempts that have failed", "closeCQs"),
-              f.createIntCounter("closeCQTimeouts",
-                  "Total number of closeCQ attempts that have timed out", "closeCQs"),
-              f.createLongCounter("closeCQTime",
-                  "Total amount of time, in nanoseconds spent doing closeCQs", "nanoseconds"),
-              f.createIntGauge("gatewayBatchsInProgress",
-                  "Current number of gatewayBatchs being executed", "gatewayBatchs"),
-              f.createIntCounter("gatewayBatchs",
-                  "Total number of gatewayBatchs completed successfully", "gatewayBatchs"),
-              f.createIntCounter("gatewayBatchFailures",
-                  "Total number of gatewayBatch attempts that have failed", "gatewayBatchs"),
-              f.createIntCounter("gatewayBatchTimeouts",
-                  "Total number of gatewayBatch attempts that have timed out", "gatewayBatchs"),
-              f.createLongCounter("gatewayBatchTime",
-                  "Total amount of time, in nanoseconds spent doing gatewayBatchs", "nanoseconds"),
-              f.createIntGauge("getDurableCQsInProgress",
-                  "Current number of getDurableCQs being executed", "getDurableCQs"),
-              f.createIntCounter("getDurableCQs",
-                  "Total number of getDurableCQs completed successfully", "getDurableCQs"),
-              f.createIntCounter("getDurableCQsFailures",
-                  "Total number of getDurableCQs attempts that have failed", "getDurableCQs"),
-              f.createIntCounter("getDurableCQsTimeouts",
-                  "Total number of getDurableCQs attempts that have timed out", "getDurableCQs"),
-              f.createLongCounter("getDurableCQsTime",
-                  "Total amount of time, in nanoseconds spent doing getDurableCQs", "nanoseconds"),
-              f.createIntGauge("readyForEventsInProgress",
-                  "Current number of readyForEvents being executed", "readyForEvents"),
-              f.createIntCounter("readyForEvents",
-                  "Total number of readyForEvents completed successfully", "readyForEvents"),
-              f.createIntCounter("readyForEventsFailures",
-                  "Total number of readyForEvents attempts that have failed", "readyForEvents"),
-              f.createIntCounter("readyForEventsTimeouts",
-                  "Total number of readyForEvents attempts that have timed out", "readyForEvents"),
-              f.createLongCounter("readyForEventsTime",
-                  "Total amount of time, in nanoseconds spent doing readyForEvents", "nanoseconds"),
-              f.createIntGauge("makePrimarysInProgress",
-                  "Current number of makePrimarys being executed", "makePrimarys"),
-              f.createIntCounter("makePrimarys",
-                  "Total number of makePrimarys completed successfully", "makePrimarys"),
-              f.createIntCounter("makePrimaryFailures",
-                  "Total number of makePrimary attempts that have failed", "makePrimarys"),
-              f.createIntCounter("makePrimaryTimeouts",
-                  "Total number of makePrimary attempts that have timed out", "makePrimarys"),
-              f.createLongCounter("makePrimaryTime",
-                  "Total amount of time, in nanoseconds spent doing makePrimarys", "nanoseconds"),
-
-              f.createIntGauge("closeConsInProgress", "Current number of closeCons being executed",
-                  "closeCons"),
-              f.createIntCounter("closeCons", "Total number of closeCons completed successfully",
-                  "closeCons"),
-              f.createIntCounter("closeConFailures",
-                  "Total number of closeCon attempts that have failed", "closeCons"),
-              f.createIntCounter("closeConTimeouts",
-                  "Total number of closeCon attempts that have timed out", "closeCons"),
-              f.createLongCounter("closeConTime",
-                  "Total amount of time, in nanoseconds spent doing closeCons", "nanoseconds"),
-
-              f.createIntGauge("primaryAcksInProgress",
-                  "Current number of primaryAcks being executed", "primaryAcks"),
-              f.createIntCounter("primaryAcks",
-                  "Total number of primaryAcks completed successfully", "primaryAcks"),
-              f.createIntCounter("primaryAckFailures",
-                  "Total number of primaryAck attempts that have failed", "primaryAcks"),
-              f.createIntCounter("primaryAckTimeouts",
-                  "Total number of primaryAck attempts that have timed out", "primaryAcks"),
-              f.createLongCounter("primaryAckTime",
-                  "Total amount of time, in nanoseconds spent doing primaryAcks", "nanoseconds"),
-
-              f.createIntGauge("pingsInProgress", "Current number of pings being executed",
-                  "pings"),
-              f.createIntCounter("pings", "Total number of pings completed successfully", "pings"),
-              f.createIntCounter("pingFailures", "Total number of ping attempts that have failed",
-                  "pings"),
-              f.createIntCounter("pingTimeouts",
-                  "Total number of ping attempts that have timed out", "pings"),
-              f.createLongCounter("pingTime",
-                  "Total amount of time, in nanoseconds spent doing pings", "nanoseconds"),
-
-              f.createIntGauge("registerInstantiatorsInProgress",
-                  "Current number of registerInstantiators being executed",
-                  "registerInstantiators"),
-              f.createIntCounter("registerInstantiators",
-                  "Total number of registerInstantiators completed successfully",
-                  "registerInstantiators"),
-              f.createIntCounter("registerInstantiatorsFailures",
-                  "Total number of registerInstantiators attempts that have failed",
-                  "registerInstantiators"),
-              f.createIntCounter("registerInstantiatorsTimeouts",
-                  "Total number of registerInstantiators attempts that have timed out",
-                  "registerInstantiators"),
-              f.createLongCounter("registerInstantiatorsTime",
-                  "Total amount of time, in nanoseconds spent doing registerInstantiators",
-                  "nanoseconds"),
-
-              f.createIntGauge("registerDataSerializersInProgress",
-                  "Current number of registerDataSerializers being executed",
-                  "registerDataSerializers"),
-              f.createIntCounter("registerDataSerializers",
-                  "Total number of registerDataSerializers completed successfully",
-                  "registerDataSerializers"),
-              f.createIntCounter("registerDataSerializersFailures",
-                  "Total number of registerDataSerializers attempts that have failed",
-                  "registerDataSerializers"),
-              f.createIntCounter("registerDataSerializersTimeouts",
-                  "Total number of registerDataSerializers attempts that have timed out",
-                  "registerDataSerializers"),
-              f.createLongCounter("registerDataSerializersTime",
-                  "Total amount of time, in nanoseconds spent doing registerDataSerializers",
-                  "nanoseconds"),
-
-              f.createIntGauge("connections", "Current number of connections", "connections"),
-              f.createIntCounter("connects", "Total number of times a connection has been created.",
-                  "connects"),
-              f.createIntCounter("disconnects",
-                  "Total number of times a connection has been destroyed.", "disconnects"),
-              f.createIntGauge("putAllsInProgress", "Current number of putAlls being executed",
-                  "putAlls"),
-              f.createIntCounter("putAlls", "Total number of putAlls completed successfully",
-                  "putAlls"),
-              f.createIntCounter("putAllFailures",
-                  "Total number of putAll attempts that have failed", "putAlls"),
-              f.createIntCounter("putAllTimeouts",
-                  "Total number of putAll attempts that have timed out", "putAlls"),
-              f.createLongCounter("putAllTime",
-                  "Total amount of time, in nanoseconds spent doing putAlls", "nanoseconds"),
-              f.createIntGauge("removeAllsInProgress",
-                  "Current number of removeAlls being executed", "removeAlls"),
-              f.createIntCounter("removeAlls", "Total number of removeAlls completed successfully",
-                  "removeAlls"),
-              f.createIntCounter("removeAllFailures",
-                  "Total number of removeAll attempts that have failed", "removeAlls"),
-              f.createIntCounter("removeAllTimeouts",
-                  "Total number of removeAll attempts that have timed out", "removeAlls"),
-              f.createLongCounter("removeAllTime",
-                  "Total amount of time, in nanoseconds spent doing removeAlls", "nanoseconds"),
-              f.createIntGauge("getAllsInProgress", "Current number of getAlls being executed",
-                  "getAlls"),
-              f.createIntCounter("getAlls", "Total number of getAlls completed successfully",
-                  "getAlls"),
-              f.createIntCounter("getAllFailures",
-                  "Total number of getAll attempts that have failed", "getAlls"),
-              f.createIntCounter("getAllTimeouts",
-                  "Total number of getAll attempts that have timed out", "getAlls"),
-              f.createLongCounter("getAllTime",
-                  "Total amount of time, in nanoseconds spent doing getAlls", "nanoseconds"),
-              f.createLongCounter("receivedBytes",
-                  "Total number of bytes received (as responses) from server over a client-to-server connection.",
-                  "bytes"),
-              f.createLongCounter("sentBytes",
-                  "Total number of bytes sent to server over a client-to-server connection.",
-                  "bytes"),
-              f.createIntGauge("messagesBeingReceived",
-                  "Current number of message being received off the network or being processed after reception over a client-to-server connection.",
-                  "messages"),
-              f.createLongGauge("messageBytesBeingReceived",
-                  "Current number of bytes consumed by messages being received or processed over a client-to-server connection.",
-                  "bytes"),
-
-              f.createIntGauge("executeFunctionsInProgress",
-                  "Current number of Functions being executed", "executeFunctions"),
-              f.createIntCounter("executeFunctions",
-                  "Total number of Functions completed successfully", "executeFunctions"),
-              f.createIntCounter("executeFunctionFailures",
-                  "Total number of Function attempts that have failed", "executeFunctions"),
-              f.createIntCounter("executeFunctionTimeouts",
-                  "Total number of Function attempts that have timed out", "executeFunctions"),
-              f.createLongCounter("executeFunctionTime",
-                  "Total amount of time, in nanoseconds spent doing Functions", "nanoseconds"),
-
-              f.createIntGauge("asyncExecuteFunctionsInProgress",
-                  "Current number of Functions being executed asynchronously",
-                  "asyncExecuteFunctions"),
-              f.createIntCounter("asyncExecuteFunctions",
-                  "Total number of asynchronous Functions completed successfully",
-                  "asyncExecuteFunctions"),
-              f.createIntCounter("asyncExecuteFunctionFailures",
-                  "Total number of asynchronous Function attempts that have failed",
-                  "asyncExecuteFunctions"),
-              f.createIntCounter("asyncExecuteFunctionTimeouts",
-                  "Total number of asynchronous Function attempts that have timed out",
-                  "asyncExecuteFunctions"),
-              f.createLongCounter("asyncExecuteFunctionTime",
-                  "Total amount of time, in nanoseconds spent doing asynchronous Functions",
-                  "nanoseconds"),
-
-              f.createIntGauge("getClientPRMetadataInProgress",
-                  "Current number of getClientPRMetadata operations being executed",
-                  "getClientPRMetadata"),
-              f.createIntCounter("getClientPRMetadataFailures",
-                  "Total number of getClientPRMetadata operation attempts that have failed",
-                  "getClientPRMetadata"),
-              f.createIntCounter("getClientPRMetadataSuccessful",
-                  "Total number of getClientPRMetadata operations completed successfully",
-                  "getClientPRMetadata"),
-              f.createIntCounter("getClientPRMetadataTimeouts",
-                  "Total number of getClientPRMetadata operation attempts that have timed out",
-                  "getClientPRMetadata"),
-              f.createLongCounter("getClientPRMetadataTime",
-                  "Total amount of time, in nanoseconds spent doing getClientPRMetadata successfully/unsuccessfully",
-                  "nanoseconds"),
-
-              f.createIntGauge("getClientPartitionAttributesInProgress",
-                  "Current number of getClientPartitionAttributes operations being executed",
-                  "getClientPartitionAttributes"),
-              f.createIntCounter("getClientPartitionAttributesFailures",
-                  "Total number of getClientPartitionAttributes operation attempts that have failed",
-                  "getClientPartitionAttributes"),
-              f.createIntCounter("getClientPartitionAttributesSuccessful",
-                  "Total number of getClientPartitionAttributes operations completed successfully",
-                  "getClientPartitionAttributes"),
-              f.createIntCounter("getClientPartitionAttributesTimeouts",
-                  "Total number of getClientPartitionAttributes operation attempts that have timed out",
-                  "getClientPartitionAttributes"),
-              f.createLongCounter("getClientPartitionAttributesTime",
-                  "Total amount of time, in nanoseconds spent doing getClientPartitionAttributes successfully/unsuccessfully.",
-                  "nanoseconds"),
-
-              f.createIntGauge("getPDXTypeByIdInProgress",
-                  "Current number of getPDXTypeById operations being executed", "getPDXTypeById"),
-              f.createIntCounter("getPDXTypeByIdFailures",
-                  "Total number of getPDXTypeById operation attempts that have failed",
-                  "getPDXTypeById"),
-              f.createIntCounter("getPDXTypeByIdSuccessful",
-                  "Total number of getPDXTypeById operations completed successfully",
-                  "getPDXTypeById"),
-              f.createIntCounter("getPDXTypeByIdTimeouts",
-                  "Total number of getPDXTypeById operation attempts that have timed out",
-                  "getPDXTypeById"),
-              f.createLongCounter("getPDXTypeByIdTime",
-                  "Total amount of time, in nanoseconds spent doing getPDXTypeById successfully/unsuccessfully.",
-                  "nanoseconds"),
-
-              f.createIntGauge("getPDXIdForTypeInProgress",
-                  "Current number of getPDXIdForType operations being executed", "getPDXIdForType"),
-              f.createIntCounter("getPDXIdForTypeFailures",
-                  "Total number of getPDXIdForType operation attempts that have failed",
-                  "getPDXIdForType"),
-              f.createIntCounter("getPDXIdForTypeSuccessful",
-                  "Total number of getPDXIdForType operations completed successfully",
-                  "getPDXIdForType"),
-              f.createIntCounter("getPDXIdForTypeTimeouts",
-                  "Total number of getPDXIdForType operation attempts that have timed out",
-                  "getPDXIdForType"),
-              f.createLongCounter("getPDXIdForTypeTime",
-                  "Total amount of time, in nanoseconds spent doing getPDXIdForType successfully/unsuccessfully.",
-                  "nanoseconds"),
-
-              f.createIntGauge("addPdxTypeInProgress",
-                  "Current number of addPdxType operations being executed", "addPdxType"),
-              f.createIntCounter("addPdxTypeFailures",
-                  "Total number of addPdxType operation attempts that have failed", "addPdxType"),
-              f.createIntCounter("addPdxTypeSuccessful",
-                  "Total number of addPdxType operations completed successfully", "addPdxType"),
-              f.createIntCounter("addPdxTypeTimeouts",
-                  "Total number of addPdxType operation attempts that have timed out",
-                  "addPdxType"),
-              f.createLongCounter("addPdxTypeTime",
-                  "Total amount of time, in nanoseconds spent doing addPdxType successfully/unsuccessfully.",
-                  "nanoseconds"),});
-
-      sendType = f.createType("ClientSendStats", "Statistics about client to server communication",
-          new StatisticDescriptor[] {
-              ///////////////////////////////////////////////////////////////////////
-              /*
-               * f.createIntGauge("opSendsInProgress", "Current number of op sends being executed",
-               * "sends"), f.createIntCounter("opSends",
-               * "Total number of op sends that have completed successfully", "sends"),
-               * f.createIntCounter("opSendFailures", "Total number of op sends that have failed",
-               * "sends"), f.createLongCounter("opSendTime",
-               * "Total amount of time, in nanoseconds spent doing op sends", "nanoseconds"),
-               */
-              ///////////////////////////////////////////////////////////////////////
-              f.createIntGauge("getSendsInProgress", "Current number of get sends being executed",
-                  "sends"),
-              f.createIntCounter("getSends",
-                  "Total number of get sends that have completed successfully", "sends"),
-              f.createIntCounter("getSendFailures", "Total number of get sends that have failed",
-                  "sends"),
-              f.createLongCounter("getSendTime",
-                  "Total amount of time, in nanoseconds spent doing get sends", "nanoseconds"),
-              f.createIntGauge("putSendsInProgress", "Current number of put sends being executed",
-                  "sends"),
-              f.createIntCounter("putSends",
-                  "Total number of put sends that have completed successfully", "sends"),
-              f.createIntCounter("putSendFailures", "Total number of put sends that have failed",
-                  "sends"),
-              f.createLongCounter("putSendTime",
-                  "Total amount of time, in nanoseconds spent doing put sends", "nanoseconds"),
-              f.createIntGauge("destroySendsInProgress",
-                  "Current number of destroy sends being executed", "sends"),
-              f.createIntCounter("destroySends",
-                  "Total number of destroy sends that have completed successfully", "sends"),
-              f.createIntCounter("destroySendFailures",
-                  "Total number of destroy sends that have failed", "sends"),
-              f.createLongCounter("destroySendTime",
-                  "Total amount of time, in nanoseconds spent doing destroy sends", "nanoseconds"),
-              f.createIntGauge("destroyRegionSendsInProgress",
-                  "Current number of destroyRegion sends being executed", "sends"),
-              f.createIntCounter("destroyRegionSends",
-                  "Total number of destroyRegion sends that have completed successfully", "sends"),
-              f.createIntCounter("destroyRegionSendFailures",
-                  "Total number of destroyRegion sends that have failed", "sends"),
-              f.createLongCounter("destroyRegionSendTime",
-                  "Total amount of time, in nanoseconds spent doing destroyRegion sends",
-                  "nanoseconds"),
-              f.createIntGauge("clearSendsInProgress",
-                  "Current number of clear sends being executed", "sends"),
-              f.createIntCounter("clearSends",
-                  "Total number of clear sends that have completed successfully", "sends"),
-              f.createIntCounter("clearSendFailures",
-                  "Total number of clear sends that have failed", "sends"),
-              f.createLongCounter("clearSendTime",
-                  "Total amount of time, in nanoseconds spent doing clear sends", "nanoseconds"),
-              f.createIntGauge("containsKeySendsInProgress",
-                  "Current number of containsKey sends being executed", "sends"),
-              f.createIntCounter("containsKeySends",
-                  "Total number of containsKey sends that have completed successfully", "sends"),
-              f.createIntCounter("containsKeySendFailures",
-                  "Total number of containsKey sends that have failed", "sends"),
-              f.createLongCounter("containsKeySendTime",
-                  "Total amount of time, in nanoseconds spent doing containsKey sends",
-                  "nanoseconds"),
-              f.createIntGauge("keySetSendsInProgress",
-                  "Current number of keySet sends being executed", "sends"),
-              f.createIntCounter("keySetSends",
-                  "Total number of keySet sends that have completed successfully", "sends"),
-              f.createIntCounter("keySetSendFailures",
-                  "Total number of keySet sends that have failed", "sends"),
-              f.createLongCounter("keySetSendTime",
-                  "Total amount of time, in nanoseconds spent doing keySet sends", "nanoseconds"),
-
-              f.createIntGauge("commitSendsInProgress",
-                  "Current number of commit sends being executed", "sends"),
-              f.createIntCounter("commitSendFailures",
-                  "Total number of commit sends that have failed", "sends"),
-              f.createIntCounter("commitSends", "Total number of commit sends that have failed",
-                  "sends"),
-              f.createLongCounter("commitSendTime",
-                  "Total amount of time, in nanoseconds spent doing commits", "nanoseconds"),
-              f.createIntGauge("rollbackSendsInProgress",
-                  "Current number of rollback sends being executed", "sends"),
-              f.createIntCounter("rollbackSendFailures",
-                  "Total number of rollback sends that have failed", "sends"),
-              f.createIntCounter("rollbackSends", "Total number of rollback sends that have failed",
-                  "sends"),
-              f.createLongCounter("rollbackSendTime",
-                  "Total amount of time, in nanoseconds spent doing rollbacks", "nanoseconds"),
-              f.createIntGauge("getEntrySendsInProgress",
-                  "Current number of getEntry sends being executed", "sends"),
-              f.createIntCounter("getEntrySendFailures",
-                  "Total number of getEntry sends that have failed", "sends"),
-              f.createIntCounter("getEntrySends", "Total number of getEntry sends that have failed",
-                  "sends"),
-              f.createLongCounter("getEntrySendTime",
-                  "Total amount of time, in nanoseconds spent sending getEntry messages",
-                  "nanoseconds"),
-              f.createIntGauge("jtaSynchronizationSendsInProgress",
-                  "Current number of jtaSynchronization sends being executed", "sends"),
-              f.createIntCounter("jtaSynchronizationSendFailures",
-                  "Total number of jtaSynchronization sends that have failed", "sends"),
-              f.createIntCounter("jtaSynchronizationSends",
-                  "Total number of jtaSynchronization sends that have failed", "sends"),
-              f.createLongCounter("jtaSynchronizationSendTime",
-                  "Total amount of time, in nanoseconds spent doing jtaSynchronizations",
-                  "nanoseconds"),
-              f.createIntGauge("txFailoverSendsInProgress",
-                  "Current number of txFailover sends being executed", "sends"),
-              f.createIntCounter("txFailoverSendFailures",
-                  "Total number of txFailover sends that have failed", "sends"),
-              f.createIntCounter("txFailoverSends",
-                  "Total number of txFailover sends that have failed", "sends"),
-              f.createLongCounter("txFailoverSendTime",
-                  "Total amount of time, in nanoseconds spent doing txFailovers", "nanoseconds"),
-              f.createIntGauge("sizeSendsInProgress", "Current number of size sends being executed",
-                  "sends"),
-              f.createIntCounter("sizeSendFailures", "Total number of size sends that have failed",
-                  "sends"),
-              f.createIntCounter("sizeSends", "Total number of size sends that have failed",
-                  "sends"),
-              f.createLongCounter("sizeSendTime",
-                  "Total amount of time, in nanoseconds spent doing sizes", "nanoseconds"),
-              f.createIntGauge("invalidateSendsInProgress",
-                  "Current number of invalidate sends being executed", "sends"),
-              f.createIntCounter("invalidateSendFailures",
-                  "Total number of invalidate sends that have failed", "sends"),
-              f.createIntCounter("invalidateSends",
-                  "Total number of invalidate sends that have failed", "sends"),
-              f.createLongCounter("invalidateSendTime",
-                  "Total amount of time, in nanoseconds spent doing invalidates", "nanoseconds"),
-              f.createIntGauge("registerInterestSendsInProgress",
-                  "Current number of registerInterest sends being executed", "sends"),
-              f.createIntCounter("registerInterestSends",
-                  "Total number of registerInterest sends that have completed successfully",
-                  "sends"),
-              f.createIntCounter("registerInterestSendFailures",
-                  "Total number of registerInterest sends that have failed", "sends"),
-              f.createLongCounter("registerInterestSendTime",
-                  "Total amount of time, in nanoseconds spent doing registerInterest sends",
-                  "nanoseconds"),
-              f.createIntGauge("unregisterInterestSendsInProgress",
-                  "Current number of unregisterInterest sends being executed", "sends"),
-              f.createIntCounter("unregisterInterestSends",
-                  "Total number of unregisterInterest sends that have completed successfully",
-                  "sends"),
-              f.createIntCounter("unregisterInterestSendFailures",
-                  "Total number of unregisterInterest sends that have failed", "sends"),
-              f.createLongCounter("unregisterInterestSendTime",
-                  "Total amount of time, in nanoseconds spent doing unregisterInterest sends",
-                  "nanoseconds"),
-              f.createIntGauge("querySendsInProgress",
-                  "Current number of query sends being executed", "sends"),
-              f.createIntCounter("querySends",
-                  "Total number of query sends that have completed successfully", "sends"),
-              f.createIntCounter("querySendFailures",
-                  "Total number of query sends that have failed", "sends"),
-              f.createLongCounter("querySendTime",
-                  "Total amount of time, in nanoseconds spent doing query sends", "nanoseconds"),
-              f.createIntGauge("createCQSendsInProgress",
-                  "Current number of createCQ sends being executed", "sends"),
-              f.createIntCounter("createCQSends",
-                  "Total number of createCQ sends that have completed successfully", "sends"),
-              f.createIntCounter("createCQSendFailures",
-                  "Total number of createCQ sends that have failed", "sends"),
-              f.createLongCounter("createCQSendTime",
-                  "Total amount of time, in nanoseconds spent doing createCQ sends", "nanoseconds"),
-              f.createIntGauge("stopCQSendsInProgress",
-                  "Current number of stopCQ sends being executed", "sends"),
-              f.createIntCounter("stopCQSends",
-                  "Total number of stopCQ sends that have completed successfully", "sends"),
-              f.createIntCounter("stopCQSendFailures",
-                  "Total number of stopCQ sends that have failed", "sends"),
-              f.createLongCounter("stopCQSendTime",
-                  "Total amount of time, in nanoseconds spent doing stopCQ sends", "nanoseconds"),
-              f.createIntGauge("closeCQSendsInProgress",
-                  "Current number of closeCQ sends being executed", "sends"),
-              f.createIntCounter("closeCQSends",
-                  "Total number of closeCQ sends that have completed successfully", "sends"),
-              f.createIntCounter("closeCQSendFailures",
-                  "Total number of closeCQ sends that have failed", "sends"),
-              f.createLongCounter("closeCQSendTime",
-                  "Total amount of time, in nanoseconds spent doing closeCQ sends", "nanoseconds"),
-              f.createIntGauge("gatewayBatchSendsInProgress",
-                  "Current number of gatewayBatch sends being executed", "sends"),
-              f.createIntCounter("gatewayBatchSends",
-                  "Total number of gatewayBatch sends that have completed successfully", "sends"),
-              f.createIntCounter("gatewayBatchSendFailures",
-                  "Total number of gatewayBatch sends that have failed", "sends"),
-              f.createLongCounter("gatewayBatchSendTime",
-                  "Total amount of time, in nanoseconds spent doing gatewayBatch sends",
-                  "nanoseconds"),
-              f.createIntGauge("getDurableCQsSendsInProgressId",
-                  "Current number of getDurableCQs sends being executed", "sends"),
-              f.createIntCounter("getDurableCQsSends",
-                  "Total number of getDurableCQs sends that have completed successfully", "sends"),
-              f.createIntCounter("getDurableCQsSendFailures",
-                  "Total number of getDurableCQs sends that have failed", "sends"),
-              f.createLongCounter("getDurableCQsSendTime",
-                  "Total amount of time, in nanoseconds spent doing getDurableCQs sends",
-                  "nanoseconds"),
-              f.createIntGauge("readyForEventsSendsInProgress",
-                  "Current number of readyForEvents sends being executed", "sends"),
-              f.createIntCounter("readyForEventsSends",
-                  "Total number of readyForEvents sends that have completed successfully", "sends"),
-              f.createIntCounter("readyForEventsSendFailures",
-                  "Total number of readyForEvents sends that have failed", "sends"),
-              f.createLongCounter("readyForEventsSendTime",
-                  "Total amount of time, in nanoseconds spent doing readyForEvents sends",
-                  "nanoseconds"),
-              f.createIntGauge("makePrimarySendsInProgress",
-                  "Current number of makePrimary sends being executed", "sends"),
-              f.createIntCounter("makePrimarySends",
-                  "Total number of makePrimary sends that have completed successfully", "sends"),
-              f.createIntCounter("makePrimarySendFailures",
-                  "Total number of makePrimary sends that have failed", "sends"),
-              f.createLongCounter("makePrimarySendTime",
-                  "Total amount of time, in nanoseconds spent doing makePrimary sends",
-                  "nanoseconds"),
-              f.createIntGauge("closeConSendsInProgress",
-                  "Current number of closeCon sends being executed", "sends"),
-              f.createIntCounter("closeConSends",
-                  "Total number of closeCon sends that have completed successfully", "sends"),
-              f.createIntCounter("closeConSendFailures",
-                  "Total number of closeCon sends that have failed", "sends"),
-              f.createLongCounter("closeConSendTime",
-                  "Total amount of time, in nanoseconds spent doing closeCon sends", "nanoseconds"),
-              f.createIntGauge("primaryAckSendsInProgress",
-                  "Current number of primaryAck sends being executed", "sends"),
-              f.createIntCounter("primaryAckSends",
-                  "Total number of primaryAck sends that have completed successfully", "sends"),
-              f.createIntCounter("primaryAckSendFailures",
-                  "Total number of primaryAck sends that have failed", "sends"),
-              f.createLongCounter("primaryAckSendTime",
-                  "Total amount of time, in nanoseconds spent doing primaryAck sends",
-                  "nanoseconds"),
-              f.createIntGauge("pingSendsInProgress", "Current number of ping sends being executed",
-                  "sends"),
-              f.createIntCounter("pingSends",
-                  "Total number of ping sends that have completed successfully", "sends"),
-              f.createIntCounter("pingSendFailures", "Total number of ping sends that have failed",
-                  "sends"),
-              f.createLongCounter("pingSendTime",
-                  "Total amount of time, in nanoseconds spent doing ping sends", "nanoseconds"),
-              f.createIntGauge("registerInstantiatorsSendsInProgress",
-                  "Current number of registerInstantiators sends being executed", "sends"),
-              f.createIntCounter("registerInstantiatorsSends",
-                  "Total number of registerInstantiators sends that have completed successfully",
-                  "sends"),
-              f.createIntCounter("registerInstantiatorsSendFailures",
-                  "Total number of registerInstantiators sends that have failed", "sends"),
-              f.createLongCounter("registerInstantiatorsSendTime",
-                  "Total amount of time, in nanoseconds spent doing registerInstantiators sends",
-                  "nanoseconds"),
-              f.createIntGauge("registerDataSerializersSendInProgress",
-                  "Current number of registerDataSerializers sends being executed", "sends"),
-              f.createIntCounter("registerDataSerializersSends",
-                  "Total number of registerDataSerializers sends that have completed successfully",
-                  "sends"),
-              f.createIntCounter("registerDataSerializersSendFailures",
-                  "Total number of registerDataSerializers sends that have failed", "sends"),
-              f.createLongCounter("registerDataSerializersSendTime",
-                  "Total amount of time, in nanoseconds spent doing registerDataSerializers sends",
-                  "nanoseconds"),
-              f.createIntGauge("putAllSendsInProgress",
-                  "Current number of putAll sends being executed", "sends"),
-              f.createIntCounter("putAllSends",
-                  "Total number of putAll sends that have completed successfully", "sends"),
-              f.createIntCounter("putAllSendFailures",
-                  "Total number of putAll sends that have failed", "sends"),
-              f.createLongCounter("putAllSendTime",
-                  "Total amount of time, in nanoseconds spent doing putAll sends", "nanoseconds"),
-              f.createIntGauge("removeAllSendsInProgress",
-                  "Current number of removeAll sends being executed", "sends"),
-              f.createIntCounter("removeAllSends",
-                  "Total number of removeAll sends that have completed successfully", "sends"),
-              f.createIntCounter("removeAllSendFailures",
-                  "Total number of removeAll sends that have failed", "sends"),
-              f.createLongCounter("removeAllSendTime",
-                  "Total amount of time, in nanoseconds spent doing removeAll sends",
-                  "nanoseconds"),
-              f.createIntGauge("getAllSendsInProgress",
-                  "Current number of getAll sends being executed", "sends"),
-              f.createIntCounter("getAllSends",
-                  "Total number of getAll sends that have completed successfully", "sends"),
-              f.createIntCounter("getAllSendFailures",
-                  "Total number of getAll sends that have failed", "sends"),
-              f.createLongCounter("getAllSendTime",
-                  "Total amount of time, in nanoseconds spent doing getAll sends", "nanoseconds"),
-              f.createIntGauge("executeFunctionSendsInProgress",
-                  "Current number of Function sends being executed", "sends"),
-              f.createIntCounter("executeFunctionSends",
-                  "Total number of Function sends that have completed successfully", "sends"),
-              f.createIntCounter("executeFunctionSendFailures",
-                  "Total number of Function sends that have failed", "sends"),
-              f.createLongCounter("executeFunctionSendTime",
-                  "Total amount of time, in nanoseconds spent doing Function sends", "nanoseconds"),
-              f.createIntGauge("asyncExecuteFunctionSendsInProgress",
-                  "Current number of Function sends being executed asynchronously", "sends"),
-              f.createIntCounter("asyncExecuteFunctionSends",
-                  "Total number of asynchronous Function sends that have completed successfully",
-                  "sends"),
-              f.createIntCounter("asyncExecuteFunctionSendFailures",
-                  "Total number of asynchronous Function sends that have failed", "sends"),
-              f.createLongCounter("asyncExecuteFunctionSendTime",
-                  "Total amount of time, in nanoseconds spent doing asynchronous Function sends",
-                  "nanoseconds"),
-              f.createIntGauge("getClientPRMetadataSendsInProgress",
-                  "Current number of getClientPRMetadata operation's request messages being send from the client to server",
-                  "sends"),
-              f.createIntCounter("getClientPRMetadataSendFailures",
-                  "Total number of getClientPRMetadata operation's request messages not sent successfully from the client to server",
-                  "sends"),
-              f.createIntCounter("getClientPRMetadataSendsSuccessful",
-                  "Total number of getClientPRMetadata operation's request messages sent successfully from the client to server",
-                  "sends"),
-              f.createLongCounter("getClientPRMetadataSendTime",
-                  "Total amount of time, in nanoseconds spent sending getClientPRMetadata operation's request messages successfully/unsuccessfully from the client to server",
-                  "nanoseconds"),
-              f.createIntGauge("getClientPartitionAttributesSendsInProgress",
-                  "Current number of getClientPartitionAttributes operation's request messages being send from the client to server",
-                  "sends"),
-              f.createIntCounter("getClientPartitionAttributesSendFailures",
-                  "Total number of getClientPartitionAttributes operation's request messages not sent successfully from the client to server",
-                  "sends"),
-              f.createIntCounter("getClientPartitionAttributesSendsSuccessful",
-                  "Total number of getClientPartitionAttributes operation's request messages sent successfully from the client to server",
-                  "sends"),
-              f.createLongCounter("getClientPartitionAttributesSendTime",
-                  "Total amount of time, in nanoseconds spent sending getClientPartitionAttributes operation's request messages successfully/unsuccessfully from the client to server",
-                  "nanoseconds"),
-              f.createIntGauge("getPDXTypeByIdSendsInProgress",
-                  "Current number of getPDXTypeById operation's request messages being send from the client to server",
-                  "sends"),
-              f.createIntCounter("getPDXTypeByIdSendFailures",
-                  "Total number of getPDXTypeById operation's request messages not sent successfully from the client to server",
-                  "sends"),
-              f.createIntCounter("getPDXTypeByIdSendsSuccessful",
-                  "Total number of getPDXTypeById operation's request messages sent successfully from the client to server",
-                  "sends"),
-              f.createLongCounter("getPDXTypeByIdSendTime",
-                  "Total amount of time, in nanoseconds spent sending getPDXTypeById operation's request messages successfully/unsuccessfully from the client to server",
-                  "nanoseconds"),
-              f.createIntGauge("getPDXIdForTypeSendsInProgress",
-                  "Current number of getPDXIdForType operation's request messages being send from the client to server",
-                  "sends"),
-              f.createIntCounter("getPDXIdForTypeSendFailures",
-                  "Total number of getPDXIdForType operation's request messages not sent successfully from the client to server",
-                  "sends"),
-              f.createIntCounter("getPDXIdForTypeSendsSuccessful",
-                  "Total number of getPDXIdForType operation's request messages sent successfully from the client to server",
-                  "sends"),
-              f.createLongCounter("getPDXIdForTypeSendTime",
-                  "Total amount of time, in nanoseconds spent sending getPDXIdForType operation's request messages successfully/unsuccessfully from the client to server",
-                  "nanoseconds"),
-              f.createIntGauge("addPdxTypeSendsInProgress",
-                  "Current number of addPdxType operation's request messages being send from the client to server",
-                  "sends"),
-              f.createIntCounter("addPdxTypeSendFailures",
-                  "Total number of addPdxType operation's request messages not sent successfully from the client to server",
-                  "sends"),
-              f.createIntCounter("addPdxTypeSendsSuccessful",
-                  "Total number of addPdxType operation's request messages sent successfully from the client to server",
-                  "sends"),
-              f.createLongCounter("addPdxTypeSendTime",
-                  "Total amount of time, in nanoseconds spent sending addPdxType operation's request messages successfully/unsuccessfully from the client to server",
-                  "nanoseconds"),});
-      ///////////////////////////////////////////////////////////////////////
-      /*
-       * opInProgressId = type.nameToId("opsInProgress"); opSendInProgressId =
-       * sendType.nameToId("opSendsInProgress"); opSendFailedId =
-       * sendType.nameToId("opSendFailures"); opSendId = sendType.nameToId("opSends");
-       * opSendDurationId = sendType.nameToId("opSendTime"); opTimedOutId =
-       * type.nameToId("opTimeouts"); opFailedId = type.nameToId("opFailures"); opId =
-       * type.nameToId("ops"); opDurationId = type.nameToId("opTime");
-       */
-      ///////////////////////////////////////////////////////////////////////
-      getInProgressId = type.nameToId("getsInProgress");
-      getSendInProgressId = sendType.nameToId("getSendsInProgress");
-      getSendFailedId = sendType.nameToId("getSendFailures");
-      getSendId = sendType.nameToId("getSends");
-      getSendDurationId = sendType.nameToId("getSendTime");
-      getTimedOutId = type.nameToId("getTimeouts");
-      getFailedId = type.nameToId("getFailures");
-      getId = type.nameToId("gets");
-      getDurationId = type.nameToId("getTime");
-      putInProgressId = type.nameToId("putsInProgress");
-      putSendInProgressId = sendType.nameToId("putSendsInProgress");
-      putSendFailedId = sendType.nameToId("putSendFailures");
-      putSendId = sendType.nameToId("putSends");
-      putSendDurationId = sendType.nameToId("putSendTime");
-      putTimedOutId = type.nameToId("putTimeouts");
-      putFailedId = type.nameToId("putFailures");
-      putId = type.nameToId("puts");
-      putDurationId = type.nameToId("putTime");
-      destroyInProgressId = type.nameToId("destroysInProgress");
-      destroySendInProgressId = sendType.nameToId("destroySendsInProgress");
-      destroySendFailedId = sendType.nameToId("destroySendFailures");
-      destroySendId = sendType.nameToId("destroySends");
-      destroySendDurationId = sendType.nameToId("destroySendTime");
-      destroyTimedOutId = type.nameToId("destroyTimeouts");
-      destroyFailedId = type.nameToId("destroyFailures");
-      destroyId = type.nameToId("destroys");
-      destroyDurationId = type.nameToId("destroyTime");
-      destroyRegionInProgressId = type.nameToId("destroyRegionsInProgress");
-      destroyRegionSendInProgressId = sendType.nameToId("destroyRegionSendsInProgress");
-      destroyRegionSendFailedId = sendType.nameToId("destroyRegionSendFailures");
-      destroyRegionSendId = sendType.nameToId("destroyRegionSends");
-      destroyRegionSendDurationId = sendType.nameToId("destroyRegionSendTime");
-      destroyRegionTimedOutId = type.nameToId("destroyRegionTimeouts");
-      destroyRegionFailedId = type.nameToId("destroyRegionFailures");
-      destroyRegionId = type.nameToId("destroyRegions");
-      destroyRegionDurationId = type.nameToId("destroyRegionTime");
-      clearInProgressId = type.nameToId("clearsInProgress");
-      clearSendInProgressId = sendType.nameToId("clearSendsInProgress");
-      clearSendFailedId = sendType.nameToId("clearSendFailures");
-      clearSendId = sendType.nameToId("clearSends");
-      clearSendDurationId = sendType.nameToId("clearSendTime");
-      clearTimedOutId = type.nameToId("clearTimeouts");
-      clearFailedId = type.nameToId("clearFailures");
-      clearId = type.nameToId("clears");
-      clearDurationId = type.nameToId("clearTime");
-      containsKeyInProgressId = type.nameToId("containsKeysInProgress");
-      containsKeySendInProgressId = sendType.nameToId("containsKeySendsInProgress");
-      containsKeySendFailedId = sendType.nameToId("containsKeySendFailures");
-      containsKeySendId = sendType.nameToId("containsKeySends");
-      containsKeySendDurationId = sendType.nameToId("containsKeySendTime");
-      containsKeyTimedOutId = type.nameToId("containsKeyTimeouts");
-      containsKeyFailedId = type.nameToId("containsKeyFailures");
-      containsKeyId = type.nameToId("containsKeys");
-      containsKeyDurationId = type.nameToId("containsKeyTime");
-
-      keySetInProgressId = type.nameToId("keySetsInProgress");
-      keySetSendInProgressId = sendType.nameToId("keySetSendsInProgress");
-      keySetSendFailedId = sendType.nameToId("keySetSendFailures");
-      keySetSendId = sendType.nameToId("keySetSends");
-      keySetSendDurationId = sendType.nameToId("keySetSendTime");
-      keySetTimedOutId = type.nameToId("keySetTimeouts");
-      keySetFailedId = type.nameToId("keySetFailures");
-      keySetId = type.nameToId("keySets");
-      keySetDurationId = type.nameToId("keySetTime");
-
-      commitInProgressId = type.nameToId("commitsInProgress");
-      commitSendInProgressId = sendType.nameToId("commitSendsInProgress");
-      commitSendFailedId = sendType.nameToId("commitSendFailures");
-      commitSendId = sendType.nameToId("commitSends");
-      commitSendDurationId = sendType.nameToId("commitSendTime");
-      commitDurationId = type.nameToId("commitTime");
-      commitTimedOutId = type.nameToId("commitTimeouts");
-      commitFailedId = type.nameToId("commitFailures");
-      commitId = type.nameToId("commits");
-
-      rollbackInProgressId = type.nameToId("rollbacksInProgress");
-      rollbackSendInProgressId = sendType.nameToId("rollbackSendsInProgress");
-      rollbackSendFailedId = sendType.nameToId("rollbackSendFailures");
-      rollbackSendId = sendType.nameToId("rollbackSends");
-      rollbackSendDurationId = sendType.nameToId("rollbackSendTime");
-      rollbackDurationId = type.nameToId("rollbackTime");
-      rollbackTimedOutId = type.nameToId("rollbackTimeouts");
-      rollbackFailedId = type.nameToId("rollbackFailures");
-      rollbackId = type.nameToId("rollbacks");
-
-      getEntryInProgressId = type.nameToId("getEntrysInProgress");
-      getEntrySendInProgressId = sendType.nameToId("getEntrySendsInProgress");
-      getEntrySendFailedId = sendType.nameToId("getEntrySendFailures");
-      getEntrySendId = sendType.nameToId("getEntrySends");
-      getEntrySendDurationId = sendType.nameToId("getEntrySendTime");
-      getEntryDurationId = type.nameToId("getEntryTime");
-      getEntryTimedOutId = type.nameToId("getEntryTimeouts");
-      getEntryFailedId = type.nameToId("getEntryFailures");
-      getEntryId = type.nameToId("getEntrys");
-
-      txSynchronizationInProgressId = type.nameToId("jtaSynchronizationsInProgress");
-      txSynchronizationSendInProgressId = sendType.nameToId("jtaSynchronizationSendsInProgress");
-      txSynchronizationSendFailedId = sendType.nameToId("jtaSynchronizationSendFailures");
-      txSynchronizationSendId = sendType.nameToId("jtaSynchronizationSends");
-      txSynchronizationSendDurationId = sendType.nameToId("jtaSynchronizationSendTime");
-      txSynchronizationDurationId = type.nameToId("jtaSynchronizationTime");
-      txSynchronizationTimedOutId = type.nameToId("jtaSynchronizationTimeouts");
-      txSynchronizationFailedId = type.nameToId("jtaSynchronizationFailures");
-      txSynchronizationId = type.nameToId("jtaSynchronizations");
-
-      txFailoverInProgressId = type.nameToId("txFailoversInProgress");
-      txFailoverSendInProgressId = sendType.nameToId("txFailoverSendsInProgress");
-      txFailoverSendFailedId = sendType.nameToId("txFailoverSendFailures");
-      txFailoverSendId = sendType.nameToId("txFailoverSends");
-      txFailoverSendDurationId = sendType.nameToId("txFailoverSendTime");
-      txFailoverDurationId = type.nameToId("txFailoverTime");
-      txFailoverTimedOutId = type.nameToId("txFailoverTimeouts");
-      txFailoverFailedId = type.nameToId("txFailoverFailures");
-      txFailoverId = type.nameToId("txFailovers");
-
-      sizeInProgressId = type.nameToId("sizesInProgress");
-      sizeSendInProgressId = sendType.nameToId("sizeSendsInProgress");
-      sizeSendFailedId = sendType.nameToId("sizeSendFailures");
-      sizeSendId = sendType.nameToId("sizeSends");
-      sizeSendDurationId = sendType.nameToId("sizeSendTime");
-      sizeDurationId = type.nameToId("sizeTime");
-      sizeTimedOutId = type.nameToId("sizeTimeouts");
-      sizeFailedId = type.nameToId("sizeFailures");
-      sizeId = type.nameToId("sizes");
-
-
-      invalidateInProgressId = type.nameToId("invalidatesInProgress");
-      invalidateSendInProgressId = sendType.nameToId("invalidateSendsInProgress");
-      invalidateSendFailedId = sendType.nameToId("invalidateSendFailures");
-      invalidateSendId = sendType.nameToId("invalidateSends");
-      invalidateSendDurationId = sendType.nameToId("invalidateSendTime");
-      invalidateDurationId = type.nameToId("invalidateTime");
-      invalidateTimedOutId = type.nameToId("invalidateTimeouts");
-      invalidateFailedId = type.nameToId("invalidateFailures");
-      invalidateId = type.nameToId("invalidates");
-
-
-      registerInterestInProgressId = type.nameToId("registerInterestsInProgress");
-      registerInterestSendInProgressId = sendType.nameToId("registerInterestSendsInProgress");
-      registerInterestSendFailedId = sendType.nameToId("registerInterestSendFailures");
-      registerInterestSendId = sendType.nameToId("registerInterestSends");
-      registerInterestSendDurationId = sendType.nameToId("registerInterestSendTime");
-      registerInterestTimedOutId = type.nameToId("registerInterestTimeouts");
-      registerInterestFailedId = type.nameToId("registerInterestFailures");
-      registerInterestId = type.nameToId("registerInterests");
-      registerInterestDurationId = type.nameToId("registerInterestTime");
-      unregisterInterestInProgressId = type.nameToId("unregisterInterestsInProgress");
-      unregisterInterestSendInProgressId = sendType.nameToId("unregisterInterestSendsInProgress");
-      unregisterInterestSendFailedId = sendType.nameToId("unregisterInterestSendFailures");
-      unregisterInterestSendId = sendType.nameToId("unregisterInterestSends");
-      unregisterInterestSendDurationId = sendType.nameToId("unregisterInterestSendTime");
-      unregisterInterestTimedOutId = type.nameToId("unregisterInterestTimeouts");
-      unregisterInterestFailedId = type.nameToId("unregisterInterestFailures");
-      unregisterInterestId = type.nameToId("unregisterInterests");
-      unregisterInterestDurationId = type.nameToId("unregisterInterestTime");
-      queryInProgressId = type.nameToId("querysInProgress");
-      querySendInProgressId = sendType.nameToId("querySendsInProgress");
-      querySendFailedId = sendType.nameToId("querySendFailures");
-      querySendId = sendType.nameToId("querySends");
-      querySendDurationId = sendType.nameToId("querySendTime");
-      queryTimedOutId = type.nameToId("queryTimeouts");
-      queryFailedId = type.nameToId("queryFailures");
-      queryId = type.nameToId("querys");
-      queryDurationId = type.nameToId("queryTime");
-      createCQInProgressId = type.nameToId("createCQsInProgress");
-      createCQSendInProgressId = sendType.nameToId("createCQSendsInProgress");
-      createCQSendFailedId = sendType.nameToId("createCQSendFailures");
-      createCQSendId = sendType.nameToId("createCQSends");
-      createCQSendDurationId = sendType.nameToId("createCQSendTime");
-      createCQTimedOutId = type.nameToId("createCQTimeouts");
-      createCQFailedId = type.nameToId("createCQFailures");
-      createCQId = type.nameToId("createCQs");
-      createCQDurationId = type.nameToId("createCQTime");
-      stopCQInProgressId = type.nameToId("stopCQsInProgress");
-      stopCQSendInProgressId = sendType.nameToId("stopCQSendsInProgress");
-      stopCQSendFailedId = sendType.nameToId("stopCQSendFailures");
-      stopCQSendId = sendType.nameToId("stopCQSends");
-      stopCQSendDurationId = sendType.nameToId("stopCQSendTime");
-      stopCQTimedOutId = type.nameToId("stopCQTimeouts");
-      stopCQFailedId = type.nameToId("stopCQFailures");
-      stopCQId = type.nameToId("stopCQs");
-      stopCQDurationId = type.nameToId("stopCQTime");
-      closeCQInProgressId = type.nameToId("closeCQsInProgress");
-      closeCQSendInProgressId = sendType.nameToId("closeCQSendsInProgress");
-      closeCQSendFailedId = sendType.nameToId("closeCQSendFailures");
-      closeCQSendId = sendType.nameToId("closeCQSends");
-      closeCQSendDurationId = sendType.nameToId("closeCQSendTime");
-      closeCQTimedOutId = type.nameToId("closeCQTimeouts");
-      closeCQFailedId = type.nameToId("closeCQFailures");
-      closeCQId = type.nameToId("closeCQs");
-      closeCQDurationId = type.nameToId("closeCQTime");
-      gatewayBatchInProgressId = type.nameToId("gatewayBatchsInProgress");
-      gatewayBatchSendInProgressId = sendType.nameToId("gatewayBatchSendsInProgress");
-      gatewayBatchSendFailedId = sendType.nameToId("gatewayBatchSendFailures");
-      gatewayBatchSendId = sendType.nameToId("gatewayBatchSends");
-      gatewayBatchSendDurationId = sendType.nameToId("gatewayBatchSendTime");
-      gatewayBatchTimedOutId = type.nameToId("gatewayBatchTimeouts");
-      gatewayBatchFailedId = type.nameToId("gatewayBatchFailures");
-      gatewayBatchId = type.nameToId("gatewayBatchs");
-      gatewayBatchDurationId = type.nameToId("gatewayBatchTime");
-      getDurableCQsInProgressId = type.nameToId("getDurableCQsInProgress");
-      getDurableCQsSendsInProgressId = sendType.nameToId("getDurableCQsSendsInProgressId");
-      getDurableCQsSendFailedId = sendType.nameToId("getDurableCQsSendFailures");
-      getDurableCQsSendId = sendType.nameToId("getDurableCQsSends");
-      getDurableCQsSendDurationId = sendType.nameToId("getDurableCQsSendTime");
-      getDurableCQsTimedOutId = type.nameToId("getDurableCQsTimeouts");
-      getDurableCQsFailedId = type.nameToId("getDurableCQsFailures");
-      getDurableCQsId = type.nameToId("getDurableCQs");
-      getDurableCQsDurationId = type.nameToId("getDurableCQsTime");
-      readyForEventsInProgressId = type.nameToId("readyForEventsInProgress");
-      readyForEventsSendInProgressId = sendType.nameToId("readyForEventsSendsInProgress");
-      readyForEventsSendFailedId = sendType.nameToId("readyForEventsSendFailures");
-      readyForEventsSendId = sendType.nameToId("readyForEventsSends");
-      readyForEventsSendDurationId = sendType.nameToId("readyForEventsSendTime");
-      readyForEventsTimedOutId = type.nameToId("readyForEventsTimeouts");
-      readyForEventsFailedId = type.nameToId("readyForEventsFailures");
-      readyForEventsId = type.nameToId("readyForEvents");
-      readyForEventsDurationId = type.nameToId("readyForEventsTime");
-      makePrimaryInProgressId = type.nameToId("makePrimarysInProgress");
-      makePrimarySendInProgressId = sendType.nameToId("makePrimarySendsInProgress");
-      makePrimarySendFailedId = sendType.nameToId("makePrimarySendFailures");
-      makePrimarySendId = sendType.nameToId("makePrimarySends");
-      makePrimarySendDurationId = sendType.nameToId("makePrimarySendTime");
-      makePrimaryTimedOutId = type.nameToId("makePrimaryTimeouts");
-      makePrimaryFailedId = type.nameToId("makePrimaryFailures");
-      makePrimaryId = type.nameToId("makePrimarys");
-      makePrimaryDurationId = type.nameToId("makePrimaryTime");
-
-      closeConInProgressId = type.nameToId("closeConsInProgress");
-      closeConSendInProgressId = sendType.nameToId("closeConSendsInProgress");
-      closeConSendFailedId = sendType.nameToId("closeConSendFailures");
-      closeConSendId = sendType.nameToId("closeConSends");
-      closeConSendDurationId = sendType.nameToId("closeConSendTime");
-      closeConTimedOutId = type.nameToId("closeConTimeouts");
-      closeConFailedId = type.nameToId("closeConFailures");
-      closeConId = type.nameToId("closeCons");
-      closeConDurationId = type.nameToId("closeConTime");
-
-      primaryAckInProgressId = type.nameToId("primaryAcksInProgress");
-      primaryAckSendInProgressId = sendType.nameToId("primaryAckSendsInProgress");
-      primaryAckSendFailedId = sendType.nameToId("primaryAckSendFailures");
-      primaryAckSendId = sendType.nameToId("primaryAckSends");
-      primaryAckSendDurationId = sendType.nameToId("primaryAckSendTime");
-      primaryAckTimedOutId = type.nameToId("primaryAckTimeouts");
-      primaryAckFailedId = type.nameToId("primaryAckFailures");
-      primaryAckId = type.nameToId("primaryAcks");
-      primaryAckDurationId = type.nameToId("primaryAckTime");
-
-      pingInProgressId = type.nameToId("pingsInProgress");
-      pingSendInProgressId = sendType.nameToId("pingSendsInProgress");
-      pingSendFailedId = sendType.nameToId("pingSendFailures");
-      pingSendId = sendType.nameToId("pingSends");
-      pingSendDurationId = sendType.nameToId("pingSendTime");
-      pingTimedOutId = type.nameToId("pingTimeouts");
-      pingFailedId = type.nameToId("pingFailures");
-      pingId = type.nameToId("pings");
-      pingDurationId = type.nameToId("pingTime");
-
-      registerInstantiatorsInProgressId = type.nameToId("registerInstantiatorsInProgress");
-      registerInstantiatorsSendInProgressId =
-          sendType.nameToId("registerInstantiatorsSendsInProgress");
-      registerInstantiatorsSendFailedId = sendType.nameToId("registerInstantiatorsSendFailures");
-      registerInstantiatorsSendId = sendType.nameToId("registerInstantiatorsSends");
-      registerInstantiatorsSendDurationId = sendType.nameToId("registerInstantiatorsSendTime");
-      registerInstantiatorsTimedOutId = type.nameToId("registerInstantiatorsTimeouts");
-      registerInstantiatorsFailedId = type.nameToId("registerInstantiatorsFailures");
-      registerInstantiatorsId = type.nameToId("registerInstantiators");
-      registerInstantiatorsDurationId = type.nameToId("registerInstantiatorsTime");
-
-      registerDataSerializersInProgressId = type.nameToId("registerDataSerializersInProgress");
-      registerDataSerializersSendInProgressId =
-          sendType.nameToId("registerDataSerializersSendInProgress");
-      registerDataSerializersSendFailedId =
-          sendType.nameToId("registerDataSerializersSendFailures");
-      registerDataSerializersSendId = sendType.nameToId("registerDataSerializersSends");
-      registerDataSerializersSendDurationId = sendType.nameToId("registerDataSerializersSendTime");
-      registerDataSerializersTimedOutId = type.nameToId("registerDataSerializersTimeouts");
-      registerDataSerializersFailedId = type.nameToId("registerDataSerializersFailures");
-      registerDataSerializersId = type.nameToId("registerDataSerializers");
-      registerDataSerializersDurationId = type.nameToId("registerDataSerializersTime");
-
-      putAllInProgressId = type.nameToId("putAllsInProgress");
-      putAllSendInProgressId = sendType.nameToId("putAllSendsInProgress");
-      putAllSendFailedId = sendType.nameToId("putAllSendFailures");
-      putAllSendId = sendType.nameToId("putAllSends");
-      putAllSendDurationId = sendType.nameToId("putAllSendTime");
-      putAllTimedOutId = type.nameToId("putAllTimeouts");
-      putAllFailedId = type.nameToId("putAllFailures");
-      putAllId = type.nameToId("putAlls");
-      putAllDurationId = type.nameToId("putAllTime");
-
-      removeAllInProgressId = type.nameToId("removeAllsInProgress");
-      removeAllSendInProgressId = sendType.nameToId("removeAllSendsInProgress");
-      removeAllSendFailedId = sendType.nameToId("removeAllSendFailures");
-      removeAllSendId = sendType.nameToId("removeAllSends");
-      removeAllSendDurationId = sendType.nameToId("removeAllSendTime");
-      removeAllTimedOutId = type.nameToId("removeAllTimeouts");
-      removeAllFailedId = type.nameToId("removeAllFailures");
-      removeAllId = type.nameToId("removeAlls");
-      removeAllDurationId = type.nameToId("removeAllTime");
-
-      getAllInProgressId = type.nameToId("getAllsInProgress");
-      getAllSendInProgressId = sendType.nameToId("getAllSendsInProgress");
-      getAllSendFailedId = sendType.nameToId("getAllSendFailures");
-      getAllSendId = sendType.nameToId("getAllSends");
-      getAllSendDurationId = sendType.nameToId("getAllSendTime");
-      getAllTimedOutId = type.nameToId("getAllTimeouts");
-      getAllFailedId = type.nameToId("getAllFailures");
-      getAllId = type.nameToId("getAlls");
-      getAllDurationId = type.nameToId("getAllTime");
-
-      connectionsId = type.nameToId("connections");
-      connectsId = type.nameToId("connects");
-      disconnectsId = type.nameToId("disconnects");
-
-      receivedBytesId = type.nameToId("receivedBytes");
-      sentBytesId = type.nameToId("sentBytes");
-      messagesBeingReceivedId = type.nameToId("messagesBeingReceived");
-      messageBytesBeingReceivedId = type.nameToId("messageBytesBeingReceived");
-
-      executeFunctionInProgressId = type.nameToId("executeFunctionsInProgress");
-      executeFunctionSendInProgressId = sendType.nameToId("executeFunctionSendsInProgress");
-      executeFunctionSendFailedId = sendType.nameToId("executeFunctionSendFailures");
-      executeFunctionSendId = sendType.nameToId("executeFunctionSends");
-      executeFunctionSendDurationId = sendType.nameToId("executeFunctionSendTime");
-      executeFunctionTimedOutId = type.nameToId("executeFunctionTimeouts");
-      executeFunctionFailedId = type.nameToId("executeFunctionFailures");
-      executeFunctionId = type.nameToId("executeFunctions");
-      executeFunctionDurationId = type.nameToId("executeFunctionTime");
-
-      getClientPRMetadataInProgressId = type.nameToId("getClientPRMetadataInProgress");
-      getClientPRMetadataSendInProgressId = sendType.nameToId("getClientPRMetadataSendsInProgress");
-      getClientPRMetadataSendFailedId = sendType.nameToId("getClientPRMetadataSendFailures");
-      getClientPRMetadataSendId = sendType.nameToId("getClientPRMetadataSendsSuccessful");
-      getClientPRMetadataSendDurationId = sendType.nameToId("getClientPRMetadataSendTime");
-      getClientPRMetadataTimedOutId = type.nameToId("getClientPRMetadataTimeouts");
-      getClientPRMetadataFailedId = type.nameToId("getClientPRMetadataFailures");
-      getClientPRMetadataId = type.nameToId("getClientPRMetadataSuccessful");
-      getClientPRMetadataDurationId = type.nameToId("getClientPRMetadataTime");
-
-      getClientPartitionAttributesInProgressId =
-          type.nameToId("getClientPartitionAttributesInProgress");
-      getClientPartitionAttributesSendInProgressId =
-          sendType.nameToId("getClientPartitionAttributesSendsInProgress");
-      getClientPartitionAttributesSendFailedId =
-          sendType.nameToId("getClientPartitionAttributesSendFailures");
-      getClientPartitionAttributesSendId =
-          sendType.nameToId("getClientPartitionAttributesSendsSuccessful");
-      getClientPartitionAttributesSendDurationId =
-          sendType.nameToId("getClientPartitionAttributesSendTime");
-      getClientPartitionAttributesTimedOutId =
-          type.nameToId("getClientPartitionAttributesTimeouts");
-      getClientPartitionAttributesFailedId = type.nameToId("getClientPartitionAttributesFailures");
-      getClientPartitionAttributesId = type.nameToId("getClientPartitionAttributesSuccessful");
-      getClientPartitionAttributesDurationId = type.nameToId("getClientPartitionAttributesTime");
-
-      getPDXTypeByIdInProgressId = type.nameToId("getPDXTypeByIdInProgress");
-      getPDXTypeByIdSendInProgressId = sendType.nameToId("getPDXTypeByIdSendsInProgress");
-      getPDXTypeByIdSendFailedId = sendType.nameToId("getPDXTypeByIdSendFailures");
-      getPDXTypeByIdSendId = sendType.nameToId("getPDXTypeByIdSendsSuccessful");
-      getPDXTypeByIdSendDurationId = sendType.nameToId("getPDXTypeByIdSendTime");
-      getPDXTypeByIdTimedOutId = type.nameToId("getPDXTypeByIdTimeouts");
-      getPDXTypeByIdFailedId = type.nameToId("getPDXTypeByIdFailures");
-      getPDXTypeByIdId = type.nameToId("getPDXTypeByIdSuccessful");
-      getPDXTypeByIdDurationId = type.nameToId("getPDXTypeByIdTime");
-
-      getPDXIdForTypeInProgressId = type.nameToId("getPDXIdForTypeInProgress");
-      getPDXIdForTypeSendInProgressId = sendType.nameToId("getPDXIdForTypeSendsInProgress");
-      getPDXIdForTypeSendFailedId = sendType.nameToId("getPDXIdForTypeSendFailures");
-      getPDXIdForTypeSendId = sendType.nameToId("getPDXIdForTypeSendsSuccessful");
-      getPDXIdForTypeSendDurationId = sendType.nameToId("getPDXIdForTypeSendTime");
-      getPDXIdForTypeTimedOutId = type.nameToId("getPDXIdForTypeTimeouts");
-      getPDXIdForTypeFailedId = type.nameToId("getPDXIdForTypeFailures");
-      getPDXIdForTypeId = type.nameToId("getPDXIdForTypeSuccessful");
-      getPDXIdForTypeDurationId = type.nameToId("getPDXIdForTypeTime");
-
-      addPdxTypeInProgressId = type.nameToId("addPdxTypeInProgress");
-      addPdxTypeSendInProgressId = sendType.nameToId("addPdxTypeSendsInProgress");
-      addPdxTypeSendFailedId = sendType.nameToId("addPdxTypeSendFailures");
-      addPdxTypeSendId = sendType.nameToId("addPdxTypeSendsSuccessful");
-      addPdxTypeSendDurationId = sendType.nameToId("addPdxTypeSendTime");
-      addPdxTypeTimedOutId = type.nameToId("addPdxTypeTimeouts");
-      addPdxTypeFailedId = type.nameToId("addPdxTypeFailures");
-      addPdxTypeId = type.nameToId("addPdxTypeSuccessful");
-      addPdxTypeDurationId = type.nameToId("addPdxTypeTime");
-
-
-      opIds = new int[] {getId, putId, destroyId, destroyRegionId, clearId, containsKeyId, keySetId,
-          registerInterestId, unregisterInterestId, queryId, createCQId, stopCQId, closeCQId,
-          gatewayBatchId, readyForEventsId, makePrimaryId, closeConId, primaryAckId, pingId,
-          putAllId, removeAllId, getAllId, registerInstantiatorsId, executeFunctionId,
-          getClientPRMetadataId, getClientPartitionAttributesId, getPDXTypeByIdId,
-          getPDXIdForTypeId, addPdxTypeId};
-    } catch (RuntimeException t) {
-      t.printStackTrace();
-      throw t;
-    }
-  }
-
-  private static long getStatTime() {
-    return DistributionStats.getStatTime();
-  }
-
-  // instance fields
-  private final Statistics stats;
-  private final Statistics sendStats;
-  private final PoolStats poolStats;
-
-  public ConnectionStats(StatisticsFactory factory, String name,
-      PoolStats poolStats/* , GatewayStats gatewayStats */) {
-    this.stats = factory.createAtomicStatistics(type, "ClientStats-" + name);
-    this.sendStats = factory.createAtomicStatistics(sendType, "ClientSendStats-" + name);
-    this.poolStats = poolStats;
-  }
-
-  /**
-   * Records that the specified get is starting
-   * <p>
-   * Note: for every call of this method the caller must also call {@link #endGetSend} and
-   * {@link #endGet}.
-   *
-   * @return the start time of this get
-   */
-  public long startGet() {
-    this.stats.incInt(getInProgressId, 1);
-    this.sendStats.incInt(getSendInProgressId, 1);
-    startClientOp();
-    return getStatTime();
-  }
-
-  /**
-   * Records that the send part of the get has completed
-   *
-   * @param startTime the value returned by {@link #startGet}.
-   * @param failed true if the send of the get failed
-   */
-  public void endGetSend(long startTime, boolean failed) {
-    long duration = getStatTime() - startTime;
-    endClientOpSend(duration, failed);
-    this.sendStats.incInt(getSendInProgressId, -1);
-    int endGetSendId;
-    if (failed) {
-      endGetSendId = getSendFailedId;
-    } else {
-      endGetSendId = getSendId;
-    }
-    this.sendStats.incInt(endGetSendId, 1);
-    this.stats.incLong(getSendDurationId, duration);
-  }
-
-  /**
-   * Records that the specified get has ended
-   *
-   * @param startTime the value returned by {@link #startGet}.
-   * @param timedOut true if get timed out
-   * @param failed true if get failed
-   */
-  public void endGet(long startTime, boolean timedOut, boolean failed) {
-    long duration = getStatTime() - startTime;
-    endClientOp(duration, timedOut, failed);
-    this.stats.incInt(getInProgressId, -1);
-    int endGetId;
-    if (timedOut) {
-      endGetId = getTimedOutId;
-    } else if (failed) {
-      endGetId = getFailedId;
-    } else {
-      endGetId = getId;
-    }
-    this.stats.incInt(endGetId, 1);
-    this.stats.incLong(getDurationId, duration);
-  }
-
-  public int getGets() {
-    return this.stats.getInt(getId);
-  }
-
-  public long getGetDuration() {
-    return this.stats.getLong(getDurationId);
-  }
-
-  /**
-   * Records that the specified put is starting
-   * <p>
-   * Note: for every call of this method the caller must also call {@link #endPutSend} and
-   * {@link #endPut}.
-   *
-   * @return the start time of this put
-   */
-  public long startPut() {
-    this.stats.incInt(putInProgressId, 1);
-    this.sendStats.incInt(putSendInProgressId, 1);
-    startClientOp();
-    return getStatTime();
-  }
-
-  /**
-   * Records that the send part of the put has completed
-   *
-   * @param startTime the value returned by {@link #startPut}.
-   * @param failed true if the send of the put failed
-   */
-  public void endPutSend(long startTime, boolean failed) {
-    long duration = getStatTime() - startTime;
-    endClientOpSend(duration, failed);
-    this.sendStats.incInt(putSendInProgressId, -1);
-    int endPutSendId;
-    if (failed) {
-      endPutSendId = putSendFailedId;
-    } else {
-      endPutSendId = putSendId;
-    }
-    this.sendStats.incInt(endPutSendId, 1);
-    this.stats.incLong(putSendDurationId, duration);
-  }
-
-  /**
-   * Records that the specified put has ended
-   *
-   * @param startTime the value returned by {@link #startPut}.
-   * @param timedOut true if put timed out
-   * @param failed true if put failed
-   */
-  public void endPut(long startTime, boolean timedOut, boolean failed) {
-    long duration = getStatTime() - startTime;
-    endClientOp(duration, timedOut, failed);
-    this.stats.incInt(putInProgressId, -1);
-    int endPutId;
-    if (timedOut) {
-      endPutId = putTimedOutId;
-    } else if (failed) {
-      endPutId = putFailedId;
-    } else {
-      endPutId = putId;
-    }
-    this.stats.incInt(endPutId, 1);
-    this.stats.incLong(putDurationId, duration);
-  }
-
-  public int getPuts() {
-    return this.stats.getInt(putId);
-  }
-
-  public long getPutDuration() {
-    return this.stats.getLong(putDurationId);
-  }
-
-  /**
-   * Records that the specified destroy is starting
-   * <p>
-   * Note: for every call of this method the caller must also call {@link #endDestroySend} and
-   * {@link #endDestroy}.
-   *
-   * @return the start time of this destroy
-   */
-  public long startDestroy() {
-    this.stats.incInt(destroyInProgressId, 1);
-    this.sendStats.incInt(destroySendInProgressId, 1);
-    startClientOp();
-    return getStatTime();
-  }
-
-  /**
-   * Records that the send part of the destroy has completed
-   *
-   * @param startTime the value returned by {@link #startDestroy}.
-   * @param failed true if the send of the destroy failed
-   */
-  public void endDestroySend(long startTime, boolean failed) {
-    long duration = getStatTime() - startTime;
-    endClientOpSend(duration, failed);
-    this.sendStats.incInt(destroySendInProgressId, -1);
-    int endDestroySendId;
-    if (failed) {
-      endDestroySendId = destroySendFailedId;
-    } else {
-      endDestroySendId = destroySendId;
-    }
-    this.sendStats.incInt(endDestroySendId, 1);
-    this.stats.incLong(destroySendDurationId, duration);
-  }
-
-  /**
-   * Records that the specified destroy has ended
-   *
-   * @param startTime the value returned by {@link #startDestroy}.
-   * @param timedOut true if destroy timed out
-   * @param failed true if destroy failed
-   */
-  public void endDestroy(long startTime, boolean timedOut, boolean failed) {
-    long duration = getStatTime() - startTime;
-    endClientOp(duration, timedOut, failed);
-    this.stats.incInt(destroyInProgressId, -1);
-    int endDestroyId;
-    if (timedOut) {
-      endDestroyId = destroyTimedOutId;
-    } else if (failed) {
-      endDestroyId = destroyFailedId;
-    } else {
-      endDestroyId = destroyId;
-    }
-    this.stats.incInt(endDestroyId, 1);
-    this.stats.incLong(destroyDurationId, duration);
-  }
-
-  /**
-   * Records that the specified destroyRegion is starting
-   * <p>
-   * Note: for every call of this method the caller must also call {@link #endDestroyRegionSend} and
-   * {@link #endDestroyRegion}.
-   *
-   * @return the start time of this destroyRegion
-   */
-  public long startDestroyRegion() {
-    this.stats.incInt(destroyRegionInProgressId, 1);
-    this.sendStats.incInt(destroyRegionSendInProgressId, 1);
-    startClientOp();
-    return getStatTime();
-  }
-
-  /**
-   * Records that the send part of the destroyRegion has completed
-   *
-   * @param startTime the value returned by {@link #startDestroyRegion}.
-   * @param failed true if the send of the destroyRegion failed
-   */
-  public void endDestroyRegionSend(long startTime, boolean failed) {
-    long duration = getStatTime() - startTime;
-    endClientOpSend(duration, failed);
-    this.sendStats.incInt(destroyRegionSendInProgressId, -1);
-    int endDestroyRegionSendId;
-    if (failed) {
-      endDestroyRegionSendId = destroyRegionSendFailedId;
-    } else {
-      endDestroyRegionSendId = destroyRegionSendId;
-    }
-    this.sendStats.incInt(endDestroyRegionSendId, 1);
-    this.stats.incLong(destroyRegionSendDurationId, duration);
-  }
-
-  /**
-   * Records that the specified destroyRegion has ended
-   *
-   * @param startTime the value returned by {@link #startDestroyRegion}.
-   * @param timedOut true if destroyRegion timed out
-   * @param failed true if destroyRegion failed
-   */
-  public void endDestroyRegion(long startTime, boolean timedOut, boolean failed) {
-    long duration = getStatTime() - startTime;
-    endClientOp(duration, timedOut, failed);
-    this.stats.incInt(destroyRegionInProgressId, -1);
-    int endDestroyRegionId;
-    if (timedOut) {
-      endDestroyRegionId = destroyRegionTimedOutId;
-    } else if (failed) {
-      endDestroyRegionId = destroyRegionFailedId;
-    } else {
-      endDestroyRegionId = destroyRegionId;
-    }
-    this.stats.incInt(endDestroyRegionId, 1);
-    this.stats.incLong(destroyRegionDurationId, duration);
-  }
-
-  /**
-   * Records that the specified clear is starting
-   * <p>
-   * Note: for every call of this method the caller must also call {@link #endClearSend} and
-   * {@link #endClear}.
-   *
-   * @return the start time of this clear
-   */
-  public long startClear() {
-    this.stats.incInt(clearInProgressId, 1);
-    this.sendStats.incInt(clearSendInProgressId, 1);
-    startClientOp();
-    return getStatTime();
-  }
-
-  /**
-   * Records that the send part of the clear has completed
-   *
-   * @param startTime the value returned by {@link #startClear}.
-   * @param failed true if the send of the clear failed
-   */
-  public void endClearSend(long startTime, boolean failed) {
-    long duration = getStatTime() - startTime;
-    endClientOpSend(duration, failed);
-    this.sendStats.incInt(clearSendInProgressId, -1);
-    int endClearSendId;
-    if (failed) {
-      endClearSendId = clearSendFailedId;
-    } else {
-      endClearSendId = clearSendId;
-    }
-    this.sendStats.incInt(endClearSendId, 1);
-    this.stats.incLong(clearSendDurationId, duration);
-  }
-
-  /**
-   * Records that the specified clear has ended
-   *
-   * @param startTime the value returned by {@link #startClear}.
-   * @param timedOut true if clear timed out
-   * @param failed true if clear failed
-   */
-  public void endClear(long startTime, boolean timedOut, boolean failed) {
-    long duration = getStatTime() - startTime;
-    endClientOp(duration, timedOut, failed);
-    this.stats.incInt(clearInProgressId, -1);
-    int endClearId;
-    if (timedOut) {
-      endClearId = clearTimedOutId;
-    } else if (failed) {
-      endClearId = clearFailedId;
-    } else {
-      endClearId = clearId;
-    }
-    this.stats.incInt(endClearId, 1);
-    this.stats.incLong(clearDurationId, duration);
-  }
-
-  /**
-   * Records that the specified containsKey is starting
-   * <p>
-   * Note: for every call of this method the caller must also call {@link #endContainsKeySend} and
-   * {@link #endContainsKey}.
-   *
-   * @return the start time of this containsKey
-   */
-  public long startContainsKey() {
-    this.stats.incInt(containsKeyInProgressId, 1);
-    this.sendStats.incInt(containsKeySendInProgressId, 1);
-    startClientOp();
-    return getStatTime();
-  }
-
-  /**
-   * Records that the send part of the containsKey has completed
-   *
-   * @param startTime the value returned by {@link #startContainsKey}.
-   * @param failed true if the send of the containsKey failed
-   */
-  public void endContainsKeySend(long startTime, boolean failed) {
-    long duration = getStatTime() - startTime;
-    endClientOpSend(duration, failed);
-    this.sendStats.incInt(containsKeySendInProgressId, -1);
-    int endContainsKeySendId;
-    if (failed) {
-      endContainsKeySendId = containsKeySendFailedId;
-    } else {
-      endContainsKeySendId = containsKeySendId;
-    }
-    this.sendStats.incInt(endContainsKeySendId, 1);
-    this.stats.incLong(containsKeySendDurationId, duration);
-  }
-
-  /**
-   * Records that the specified containsKey has ended
-   *
-   * @param startTime the value returned by {@link #startContainsKey}.
-   * @param timedOut true if containsKey timed out
-   * @param failed true if containsKey failed
-   */
-  public void endContainsKey(long startTime, boolean timedOut, boolean failed) {
-    long duration = getStatTime() - startTime;
-    endClientOp(duration, timedOut, failed);
-    this.stats.incInt(containsKeyInProgressId, -1);
-    int endContainsKeyId;
-    if (timedOut) {
-      endContainsKeyId = containsKeyTimedOutId;
-    } else if (failed) {
-      endContainsKeyId = containsKeyFailedId;
-    } else {
-      endContainsKeyId = containsKeyId;
-    }
-    this.stats.incInt(endContainsKeyId, 1);
-    this.stats.incLong(containsKeyDurationId, duration);
-  }
-
-  /**
-   * Records that the specified keySet is starting
-   * <p>
-   * Note: for every call of this method the caller must also call {@link #endKeySetSend} and
-   * {@link #endKeySet}.
-   *
-   * @return the start time of this keySet
-   */
-  public long startKeySet() {
-    this.stats.incInt(keySetInProgressId, 1);
-    this.sendStats.incInt(keySetSendInProgressId, 1);
-    startClientOp();
-    return getStatTime();
-  }
-
-  /**
-   * Records that the send part of the keySet has completed
-   *
-   * @param startTime the value returned by {@link #startKeySet}.
-   * @param failed true if the send of the keySet failed
-   */
-  public void endKeySetSend(long startTime, boolean failed) {
-    long duration = getStatTime() - startTime;
-    endClientOpSend(duration, failed);
-    this.sendStats.incInt(keySetSendInProgressId, -1);
-    int endKeySetSendId;
-    if (failed) {
-      endKeySetSendId = keySetSendFailedId;
-    } else {
-      endKeySetSendId = keySetSendId;
-    }
-    this.sendStats.incInt(endKeySetSendId, 1);
-    this.stats.incLong(keySetSendDurationId, duration);
-  }
-
-  /**
-   * Records that the specified keySet has ended
-   *
-   * @param startTime the value returned by {@link #startKeySet}.
-   * @param timedOut true if keySet timed out
-   * @param failed true if keySet failed
-   */
-  public void endKeySet(long startTime, boolean timedOut, boolean failed) {
-    long duration = getStatTime() - startTime;
-    endClientOp(duration, timedOut, failed);
-    this.stats.incInt(keySetInProgressId, -1);
-    int endKeySetId;
-    if (timedOut) {
-      endKeySetId = keySetTimedOutId;
-    } else if (failed) {
-      endKeySetId = keySetFailedId;
-    } else {
-      endKeySetId = keySetId;
-    }
-    this.stats.incInt(endKeySetId, 1);
-    this.stats.incLong(keySetDurationId, duration);
-  }
-
-  /**
-   * Records that the specified registerInterest is starting
-   * <p>
-   * Note: for every call of this method the caller must also call {@link #endRegisterInterestSend}
-   * and {@link #endRegisterInterest}.
-   *
-   * @return the start time of this registerInterest
-   */
-  public long startRegisterInterest() {
-    this.stats.incInt(registerInterestInProgressId, 1);
-    this.sendStats.incInt(registerInterestSendInProgressId, 1);
-    startClientOp();
-    return getStatTime();
-  }
-
-  /**
-   * Records that the send part of the registerInterest has completed
-   *
-   * @param startTime the value returned by {@link #startRegisterInterest}.
-   * @param failed true if the send of the registerInterest failed
-   */
-  public void endRegisterInterestSend(long startTime, boolean failed) {
-    long duration = getStatTime() - startTime;
-    endClientOpSend(duration, failed);
-    this.sendStats.incInt(registerInterestSendInProgressId, -1);
-    int endRegisterInterestSendId;
-    if (failed) {
-      endRegisterInterestSendId = registerInterestSendFailedId;
-    } else {
-      endRegisterInterestSendId = registerInterestSendId;
-    }
-    this.sendStats.incInt(endRegisterInterestSendId, 1);
-    this.stats.incLong(registerInterestSendDurationId, duration);
-  }
-
-  /**
-   * Records that the specified registerInterest has ended
-   *
-   * @param startTime the value returned by {@link #startRegisterInterest}.
-   * @param timedOut true if registerInterest timed out
-   * @param failed true if registerInterest failed
-   */
-  public void endRegisterInterest(long startTime, boolean timedOut, boolean failed) {
-    long duration = getStatTime() - startTime;
-    endClientOp(duration, timedOut, failed);
-    this.stats.incInt(registerInterestInProgressId, -1);
-    int endRegisterInterestId;
-    if (timedOut) {
-      endRegisterInterestId = registerInterestTimedOutId;
-    } else if (failed) {
-      endRegisterInterestId = registerInterestFailedId;
-    } else {
-      endRegisterInterestId = registerInterestId;
-    }
-    this.stats.incInt(endRegisterInterestId, 1);
-    this.stats.incLong(registerInterestDurationId, duration);
-  }
-
-  /**
-   * Records that the specified unregisterInterest is starting
-   * <p>
-   * Note: for every call of this method the caller must also call
-   * {@link #endUnregisterInterestSend} and {@link #endUnregisterInterest}.
-   *
-   * @return the start time of this unregisterInterest
-   */
-  public long startUnregisterInterest() {
-    this.stats.incInt(unregisterInterestInProgressId, 1);
-    this.sendStats.incInt(unregisterInterestSendInProgressId, 1);
-    startClientOp();
-    return getStatTime();
-  }
-
-  /**
-   * Records that the send part of the unregisterInterest has completed
-   *
-   * @param startTime the value returned by {@link #startUnregisterInterest}.
-   * @param failed true if the send of the unregisterInterest failed
-   */
-  public void endUnregisterInterestSend(long startTime, boolean failed) {
-    long duration = getStatTime() - startTime;
-    endClientOpSend(duration, failed);
-    this.sendStats.incInt(unregisterInterestSendInProgressId, -1);
-    int endUnregisterInterestSendId;
-    if (failed) {
-      endUnregisterInterestSendId = unregisterInterestSendFailedId;
-    } else {
-      endUnregisterInterestSendId = unregisterInterestSendId;
-    }
-    this.sendStats.incInt(endUnregisterInterestSendId, 1);
-    this.stats.incLong(unregisterInterestSendDurationId, duration);
-  }
-
-  /**
-   * Records that the specified unregisterInterest has ended
-   *
-   * @param startTime the value returned by {@link #startUnregisterInterest}.
-   * @param timedOut true if unregisterInterest timed out
-   * @param failed true if unregisterInterest failed
-   */
-  public void endUnregisterInterest(long startTime, boolean timedOut, boolean failed) {
-    long duration = getStatTime() - startTime;
-    endClientOp(duration, timedOut, failed);
-    this.stats.incInt(unregisterInterestInProgressId, -1);
-    int endUnregisterInterestId;
-    if (timedOut) {
-      endUnregisterInterestId = unregisterInterestTimedOutId;
-    } else if (failed) {
-      endUnregisterInterestId = unregisterInterestFailedId;
-    } else {
-      endUnregisterInterestId = unregisterInterestId;
-    }
-    this.stats.incInt(endUnregisterInterestId, 1);
-    this.stats.incLong(unregisterInterestDurationId, duration);
-  }
-
-  /**
-   * Records that the specified query is starting
-   * <p>
-   * Note: for every call of this method the caller must also call {@link #endQuerySend} and
-   * {@link #endQuery}.
-   *
-   * @return the start time of this query
-   */
-  public long startQuery() {
-    this.stats.incInt(queryInProgressId, 1);
-    this.sendStats.incInt(querySendInProgressId, 1);
-    startClientOp();
-    return getStatTime();
-  }
-
-  /**
-   * Records that the send part of the query has completed
-   *
-   * @param startTime the value returned by {@link #startQuery}.
-   * @param failed true if the send of the query failed
-   */
-  public void endQuerySend(long startTime, boolean failed) {
-    long duration = getStatTime() - startTime;
-    endClientOpSend(duration, failed);
-    this.sendStats.incInt(querySendInProgressId, -1);
-    int endQuerySendId;
-    if (failed) {
-      endQuerySendId = querySendFailedId;
-    } else {
-      endQuerySendId = querySendId;
-    }
-    this.sendStats.incInt(endQuerySendId, 1);
-    this.stats.incLong(querySendDurationId, duration);
-  }
-
-  /**
-   * Records that the specified query has ended
-   *
-   * @param startTime the value returned by {@link #startQuery}.
-   * @param timedOut true if query timed out
-   * @param failed true if query failed
-   */
-  public void endQuery(long startTime, boolean timedOut, boolean failed) {
-    long duration = getStatTime() - startTime;
-    endClientOp(duration, timedOut, failed);
-    this.stats.incInt(queryInProgressId, -1);
-    int endQueryId;
-    if (timedOut) {
-      endQueryId = queryTimedOutId;
-    } else if (failed) {
-      endQueryId = queryFailedId;
-    } else {
-      endQueryId = queryId;
-    }
-    this.stats.incInt(endQueryId, 1);
-    this.stats.incLong(queryDurationId, duration);
-  }
-
-  /**
-   * Records that the specified createCQ is starting
-   * <p>
-   * Note: for every call of this method the caller must also call {@link #endCreateCQSend} and
-   * {@link #endCreateCQ}.
-   *
-   * @return the start time of this createCQ
-   */
-  public long startCreateCQ() {
-    this.stats.incInt(createCQInProgressId, 1);
-    this.sendStats.incInt(createCQSendInProgressId, 1);
-    startClientOp();
-    return getStatTime();
-  }
-
-  /**
-   * Records that the send part of the createCQ has completed
-   *
-   * @param startTime the value returned by {@link #startCreateCQ}.
-   * @param failed true if the send of the createCQ failed
-   */
-  public void endCreateCQSend(long startTime, boolean failed) {
-    long duration = getStatTime() - startTime;
-    endClientOpSend(duration, failed);
-    this.sendStats.incInt(createCQSendInProgressId, -1);
-    int endCreateCQSendId;
-    if (failed) {
-      endCreateCQSendId = createCQSendFailedId;
-    } else {
-      endCreateCQSendId = createCQSendId;
-    }
-    this.sendStats.incInt(endCreateCQSendId, 1);
-    this.stats.incLong(createCQSendDurationId, duration);
-  }
-
-  /**
-   * Records that the specified createCQ has ended
-   *
-   * @param startTime the value returned by {@link #startCreateCQ}.
-   * @param timedOut true if createCQ timed out
-   * @param failed true if createCQ failed
-   */
-  public void endCreateCQ(long startTime, boolean timedOut, boolean failed) {
-    long duration = getStatTime() - startTime;
-    endClientOp(duration, timedOut, failed);
-    this.stats.incInt(createCQInProgressId, -1);
-    int endCreateCQId;
-    if (timedOut) {
-      endCreateCQId = createCQTimedOutId;
-    } else if (failed) {
-      endCreateCQId = createCQFailedId;
-    } else {
-      endCreateCQId = createCQId;
-    }
-    this.stats.incInt(endCreateCQId, 1);
-    this.stats.incLong(createCQDurationId, duration);
-  }
-
-  /**
-   * Records that the specified stopCQ is starting
-   * <p>
-   * Note: for every call of this method the caller must also call {@link #endStopCQSend} and
-   * {@link #endStopCQ}.
-   *
-   * @return the start time of this stopCQ
-   */
-  public long startStopCQ() {
-    this.stats.incInt(stopCQInProgressId, 1);
-    this.sendStats.incInt(stopCQSendInProgressId, 1);
-    startClientOp();
-    return getStatTime();
-  }
-
-  /**
-   * Records that the send part of the stopCQ has completed
-   *
-   * @param startTime the value returned by {@link #startStopCQ}.
-   * @param failed true if the send of the stopCQ failed
-   */
-  public void endStopCQSend(long startTime, boolean failed) {
-    long duration = getStatTime() - startTime;
-    endClientOpSend(duration, failed);
-    this.sendStats.incInt(stopCQSendInProgressId, -1);
-    int endStopCQSendId;
-    if (failed) {
-      endStopCQSendId = stopCQSendFailedId;
-    } else {
-      endStopCQSendId = stopCQSendId;
-    }
-    this.sendStats.incInt(endStopCQSendId, 1);
-    this.stats.incLong(stopCQSendDurationId, duration);
-  }
-
-  /**
-   * Records that the specified stopCQ has ended
-   *
-   * @param startTime the value returned by {@link #startStopCQ}.
-   * @param timedOut true if stopCQ timed out
-   * @param failed true if stopCQ failed
-   */
-  public void endStopCQ(long startTime, boolean timedOut, boolean failed) {
-    long duration = getStatTime() - startTime;
-    endClientOp(duration, timedOut, failed);
-    this.stats.incInt(stopCQInProgressId, -1);
-    int endStopCQId;
-    if (timedOut) {
-      endStopCQId = stopCQTimedOutId;
-    } else if (failed) {
-      endStopCQId = stopCQFailedId;
-    } else {
-      endStopCQId = stopCQId;
-    }
-    this.stats.incInt(endStopCQId, 1);
-    this.stats.incLong(stopCQDurationId, duration);
-  }
-
-  /**
-   * Records that the specified closeCQ is starting
-   * <p>
-   * Note: for every call of this method the caller must also call {@link #endCloseCQSend} and
-   * {@link #endCloseCQ}.
-   *
-   * @return the start time of this closeCQ
-   */
-  public long startCloseCQ() {
-    this.stats.incInt(closeCQInProgressId, 1);
-    this.sendStats.incInt(closeCQSendInProgressId, 1);
-    startClientOp();
-    return getStatTime();
-  }
-
-  /**
-   * Records that the send part of the closeCQ has completed
-   *
-   * @param startTime the value returned by {@link #startCloseCQ}.
-   * @param failed true if the send of the closeCQ failed
-   */
-  public void endCloseCQSend(long startTime, boolean failed) {
-    long duration = getStatTime() - startTime;
-    endClientOpSend(duration, failed);
-    this.sendStats.incInt(closeCQSendInProgressId, -1);
-    int endCloseCQSendId;
-    if (failed) {
-      endCloseCQSendId = closeCQSendFailedId;
-    } else {
-      endCloseCQSendId = closeCQSendId;
-    }
-    this.sendStats.incInt(endCloseCQSendId, 1);
-    this.stats.incLong(closeCQSendDurationId, duration);
-  }
-
-  /**
-   * Records that the specified closeCQ has ended
-   *
-   * @param startTime the value returned by {@link #startCloseCQ}.
-   * @param timedOut true if closeCQ timed out
-   * @param failed true if closeCQ failed
-   */
-  public void endCloseCQ(long startTime, boolean timedOut, boolean failed) {
-    long duration = getStatTime() - startTime;
-    endClientOp(duration, timedOut, failed);
-    this.stats.incInt(closeCQInProgressId, -1);
-    int endCloseCQId;
-    if (timedOut) {
-      endCloseCQId = closeCQTimedOutId;
-    } else if (failed) {
-      endCloseCQId = closeCQFailedId;
-    } else {
-      endCloseCQId = closeCQId;
-    }
-    this.stats.incInt(endCloseCQId, 1);
-    this.stats.incLong(closeCQDurationId, duration);
-  }
-
-  /**
-   * Records that the specified stopCQ is starting
-   * <p>
-   * Note: for every call of this method the caller must also call {@link #endStopCQSend} and
-   * {@link #endStopCQ}.
-   *
-   * @return the start time of this stopCQ
-   */
-  public long startGetDurableCQs() {
-    this.stats.incInt(getDurableCQsInProgressId, 1);
-    this.sendStats.incInt(getDurableCQsSendsInProgressId, 1);
-    startClientOp();
-    return getStatTime();
-  }
-
-  /**
-   * Records that the send part of the stopCQ has completed
-   *
-   * @param startTime the value returned by {@link #startStopCQ}.
-   * @param failed true if the send of the stopCQ failed
-   */
-  public void endGetDurableCQsSend(long startTime, boolean failed) {
-    long duration = getStatTime() - startTime;
-    endClientOpSend(duration, failed);
-    this.sendStats.incInt(getDurableCQsSendsInProgressId, -1);
-    int endGetDurableCQsSendId;
-    if (failed) {
-      endGetDurableCQsSendId = getDurableCQsSendFailedId;
-    } else {
-      endGetDurableCQsSendId = getDurableCQsSendId;
-    }
-    this.sendStats.incInt(endGetDurableCQsSendId, 1);
-    this.stats.incLong(getDurableCQsSendDurationId, duration);
-  }
-
-  /**
-   * Records that the specified stopCQ has ended
-   *
-   * @param startTime the value returned by {@link #startStopCQ}.
-   * @param timedOut true if stopCQ timed out
-   * @param failed true if stopCQ failed
-   */
-  public void endGetDurableCQs(long startTime, boolean timedOut, boolean failed) {
-    long duration = getStatTime() - startTime;
-    endClientOp(duration, timedOut, failed);
-    this.stats.incInt(getDurableCQsInProgressId, -1);
-    int endGetDurableCQsId;
-    if (timedOut) {
-      endGetDurableCQsId = getDurableCQsTimedOutId;
-    } else if (failed) {
-      endGetDurableCQsId = getDurableCQsFailedId;
-    } else {
-      endGetDurableCQsId = getDurableCQsId;
-    }
-    this.stats.incInt(endGetDurableCQsId, 1);
-    this.stats.incLong(getDurableCQsDurationId, duration);
-  }
-
-  /**
-   * Records that the specified gatewayBatch is starting
-   * <p>
-   * Note: for every call of this method the caller must also call {@link #endGatewayBatchSend} and
-   * {@link #endGatewayBatch}.
-   *
-   * @return the start time of this gatewayBatch
-   */
-  public long startGatewayBatch() {
-    this.stats.incInt(gatewayBatchInProgressId, 1);
-    this.sendStats.incInt(gatewayBatchSendInProgressId, 1);
-    startClientOp();
-    return getStatTime();
-  }
-
-  /**
-   * Records that the send part of the gatewayBatch has completed
-   *
-   * @param startTime the value returned by {@link #startGatewayBatch}.
-   * @param failed true if the send of the gatewayBatch failed
-   */
-  public void endGatewayBatchSend(long startTime, boolean failed) {
-    long duration = getStatTime() - startTime;
-    endClientOpSend(duration, failed);
-    this.sendStats.incInt(gatewayBatchSendInProgressId, -1);
-    int endGatewayBatchSendId;
-    if (failed) {
-      endGatewayBatchSendId = gatewayBatchSendFailedId;
-    } else {
-      endGatewayBatchSendId = gatewayBatchSendId;
-    }
-    this.sendStats.incInt(endGatewayBatchSendId, 1);
-    this.stats.incLong(gatewayBatchSendDurationId, duration);
-  }
-
-  /**
-   * Records that the specified gatewayBatch has ended
-   *
-   * @param startTime the value returned by {@link #startGatewayBatch}.
-   * @param timedOut true if gatewayBatch timed out
-   * @param failed true if gatewayBatch failed
-   */
-  public void endGatewayBatch(long startTime, boolean timedOut, boolean failed) {
-    long duration = getStatTime() - startTime;
-    endClientOp(duration, timedOut, failed);
-    this.stats.incInt(gatewayBatchInProgressId, -1);
-    int endGatewayBatchId;
-    if (timedOut) {
-      endGatewayBatchId = gatewayBatchTimedOutId;
-    } else if (failed) {
-      endGatewayBatchId = gatewayBatchFailedId;
-    } else {
-      endGatewayBatchId = gatewayBatchId;
-    }
-    this.stats.incInt(endGatewayBatchId, 1);
-    this.stats.incLong(gatewayBatchDurationId, duration);
-  }
-
-  /**
-   * Records that the specified readyForEvents is starting
-   * <p>
-   * Note: for every call of this method the caller must also call {@link #endReadyForEventsSend}
-   * and {@link #endReadyForEvents}.
-   *
-   * @return the start time of this readyForEvents
-   */
-  public long startReadyForEvents() {
-    this.stats.incInt(readyForEventsInProgressId, 1);
-    this.sendStats.incInt(readyForEventsSendInProgressId, 1);
-    startClientOp();
-    return getStatTime();
-  }
-
-  /**
-   * Records that the send part of the readyForEvents has completed
-   *
-   * @param startTime the value returned by {@link #startReadyForEvents}.
-   * @param failed true if the send of the readyForEvents failed
-   */
-  public void endReadyForEventsSend(long startTime, boolean failed) {
-    long duration = getStatTime() - startTime;
-    endClientOpSend(duration, failed);
-    this.sendStats.incInt(readyForEventsSendInProgressId, -1);
-    int endReadyForEventsSendId;
-    if (failed) {
-      endReadyForEventsSendId = readyForEventsSendFailedId;
-    } else {
-      endReadyForEventsSendId = readyForEventsSendId;
-    }
-    this.sendStats.incInt(endReadyForEventsSendId, 1);
-    this.stats.incLong(readyForEventsSendDurationId, duration);
-  }
-
-  /**
-   * Records that the specified readyForEvents has ended
-   *
-   * @param startTime the value returned by {@link #startReadyForEvents}.
-   * @param timedOut true if readyForEvents timed out
-   * @param failed true if readyForEvents failed
-   */
-  public void endReadyForEvents(long startTime, boolean timedOut, boolean failed) {
-    long duration = getStatTime() - startTime;
-    endClientOp(duration, timedOut, failed);
-    this.stats.incInt(readyForEventsInProgressId, -1);
-    int endReadyForEventsId;
-    if (timedOut) {
-      endReadyForEventsId = readyForEventsTimedOutId;
-    } else if (failed) {
-      endReadyForEventsId = readyForEventsFailedId;
-    } else {
-      endReadyForEventsId = readyForEventsId;
-    }
-    this.stats.incInt(endReadyForEventsId, 1);
-    this.stats.incLong(readyForEventsDurationId, duration);
-  }
-
-  /**
-   * Records that the specified makePrimary is starting
-   * <p>
-   * Note: for every call of this method the caller must also call {@link #endMakePrimarySend} and
-   * {@link #endMakePrimary}.
-   *
-   * @return the start time of this makePrimary
-   */
-  public long startMakePrimary() {
-    this.stats.incInt(makePrimaryInProgressId, 1);
-    this.sendStats.incInt(makePrimarySendInProgressId, 1);
-    startClientOp();
-    return getStatTime();
-  }
-
-  /**
-   * Records that the send part of the makePrimary has completed
-   *
-   * @param startTime the value returned by {@link #startMakePrimary}.
-   * @param failed true if the send of the makePrimary failed
-   */
-  public void endMakePrimarySend(long startTime, boolean failed) {
-    long duration = getStatTime() - startTime;
-    endClientOpSend(duration, failed);
-    this.sendStats.incInt(makePrimarySendInProgressId, -1);
-    int endMakePrimarySendId;
-    if (failed) {
-      endMakePrimarySendId = makePrimarySendFailedId;
-    } else {
-      endMakePrimarySendId = makePrimarySendId;
-    }
-    this.sendStats.incInt(endMakePrimarySendId, 1);
-    this.stats.incLong(makePrimarySendDurationId, duration);
-  }
-
-  /**
-   * Records that the specified makePrimary has ended
-   *
-   * @param startTime the value returned by {@link #startMakePrimary}.
-   * @param timedOut true if makePrimary timed out
-   * @param failed true if makePrimary failed
-   */
-  public void endMakePrimary(long startTime, boolean timedOut, boolean failed) {
-    long duration = getStatTime() - startTime;
-    endClientOp(duration, timedOut, failed);
-    this.stats.incInt(makePrimaryInProgressId, -1);
-    int endMakePrimaryId;
-    if (timedOut) {
-      endMakePrimaryId = makePrimaryTimedOutId;
-    } else if (failed) {
-      endMakePrimaryId = makePrimaryFailedId;
-    } else {
-      endMakePrimaryId = makePrimaryId;
-    }
-    this.stats.incInt(endMakePrimaryId, 1);
-    this.stats.incLong(makePrimaryDurationId, duration);
-  }
-
-  /**
-   * Records that the specified closeCon is starting
-   * <p>
-   * Note: for every call of this method the caller must also call {@link #endCloseConSend} and
-   * {@link #endCloseCon}.
-   *
-   * @return the start time of this closeCon
-   */
-  public long startCloseCon() {
-    this.stats.incInt(closeConInProgressId, 1);
-    this.sendStats.incInt(closeConSendInProgressId, 1);
-    startClientOp();
-    return getStatTime();
-  }
-
-  /**
-   * Records that the send part of the closeCon has completed
-   *
-   * @param startTime the value returned by {@link #startCloseCon}.
-   * @param failed true if the send of the closeCon failed
-   */
-  public void endCloseConSend(long startTime, boolean failed) {
-    long duration = getStatTime() - startTime;
-    endClientOpSend(duration, failed);
-    this.sendStats.incInt(closeConSendInProgressId, -1);
-    int endCloseConSendId;
-    if (failed) {
-      endCloseConSendId = closeConSendFailedId;
-    } else {
-      endCloseConSendId = closeConSendId;
-    }
-    this.sendStats.incInt(endCloseConSendId, 1);
-    this.stats.incLong(closeConSendDurationId, duration);
-  }
-
-  /**
-   * Records that the specified closeCon has ended
-   *
-   * @param startTime the value returned by {@link #startCloseCon}.
-   * @param timedOut true if closeCon timed out
-   * @param failed true if closeCon failed
-   */
-  public void endCloseCon(long startTime, boolean timedOut, boolean failed) {
-    long duration = getStatTime() - startTime;
-    endClientOp(duration, timedOut, failed);
-    this.stats.incInt(closeConInProgressId, -1);
-    int endCloseConId;
-    if (timedOut) {
-      endCloseConId = closeConTimedOutId;
-    } else if (failed) {
-      endCloseConId = closeConFailedId;
-    } else {
-      endCloseConId = closeConId;
-    }
-    this.stats.incInt(endCloseConId, 1);
-    this.stats.incLong(closeConDurationId, duration);
-  }
-
-  /**
-   * Records that the specified primaryAck is starting
-   * <p>
-   * Note: for every call of this method the caller must also call {@link #endPrimaryAckSend} and
-   * {@link #endPrimaryAck}.
-   *
-   * @return the start time of this primaryAck
-   */
-  public long startPrimaryAck() {
-    this.stats.incInt(primaryAckInProgressId, 1);
-    this.sendStats.incInt(primaryAckSendInProgressId, 1);
-    startClientOp();
-    return getStatTime();
-  }
-
-  /**
-   * Records that the send part of the primaryAck has completed
-   *
-   * @param startTime the value returned by {@link #startPrimaryAck}.
-   * @param failed true if the send of the primaryAck failed
-   */
-  public void endPrimaryAckSend(long startTime, boolean failed) {
-    long duration = getStatTime() - startTime;
-    endClientOpSend(duration, failed);
-    this.sendStats.incInt(primaryAckSendInProgressId, -1);
-    int endPrimaryAckSendId;
-    if (failed) {
-      endPrimaryAckSendId = primaryAckSendFailedId;
-    } else {
-      endPrimaryAckSendId = primaryAckSendId;
-    }
-    this.sendStats.incInt(endPrimaryAckSendId, 1);
-    this.stats.incLong(primaryAckSendDurationId, duration);
-  }
-
-  /**
-   * Records that the specified primaryAck has ended
-   *
-   * @param startTime the value returned by {@link #startPrimaryAck}.
-   * @param timedOut true if primaryAck timed out
-   * @param failed true if primaryAck failed
-   */
-  public void endPrimaryAck(long startTime, boolean timedOut, boolean failed) {
-    long duration = getStatTime() - startTime;
-    endClientOp(duration, timedOut, failed);
-    this.stats.incInt(primaryAckInProgressId, -1);
-    int endPrimaryAckId;
-    if (timedOut) {
-      endPrimaryAckId = primaryAckTimedOutId;
-    } else if (failed) {
-      endPrimaryAckId = primaryAckFailedId;
-    } else {
-      endPrimaryAckId = primaryAckId;
-    }
-    this.stats.incInt(endPrimaryAckId, 1);
-    this.stats.incLong(primaryAckDurationId, duration);
-  }
-
-  /**
-   * Records that the specified ping is starting
-   * <p>
-   * Note: for every call of this method the caller must also call {@link #endPingSend} and
-   * {@link #endPing}.
-   *
-   * @return the start time of this ping
-   */
-  public long startPing() {
-    this.stats.incInt(pingInProgressId, 1);
-    this.sendStats.incInt(pingSendInProgressId, 1);
-    startClientOp();
-    return getStatTime();
-  }
-
-  /**
-   * Records that the send part of the ping has completed
-   *
-   * @param startTime the value returned by {@link #startPing}.
-   * @param failed true if the send of the ping failed
-   */
-  public void endPingSend(long startTime, boolean failed) {
-    long duration = getStatTime() - startTime;
-    endClientOpSend(duration, failed);
-    this.sendStats.incInt(pingSendInProgressId, -1);
-    int endPingSendId;
-    if (failed) {
-      endPingSendId = pingSendFailedId;
-    } else {
-      endPingSendId = pingSendId;
-    }
-    this.sendStats.incInt(endPingSendId, 1);
-    this.stats.incLong(pingSendDurationId, duration);
-  }
-
-  /**
-   * Records that the specified ping has ended
-   *
-   * @param startTime the value returned by {@link #startPing}.
-   * @param timedOut true if ping timed out
-   * @param failed true if ping failed
-   */
-  public void endPing(long startTime, boolean timedOut, boolean failed) {
-    long duration = getStatTime() - startTime;
-    endClientOp(duration, timedOut, failed);
-    this.stats.incInt(pingInProgressId, -1);
-    int endPingId;
-    if (timedOut) {
-      endPingId = pingTimedOutId;
-    } else if (failed) {
-      endPingId = pingFailedId;
-    } else {
-      endPingId = pingId;
-    }
-    this.stats.incInt(endPingId, 1);
-    this.stats.incLong(pingDurationId, duration);
-  }
-
-  /**
-   * Records that the specified registerInstantiators is starting
-   * <p>
-   * Note: for every call of this method the caller must also call
-   * {@link #endRegisterInstantiatorsSend} and {@link #endRegisterInstantiators}.
-   *
-   * @return the start time of this registerInstantiators
-   */
-  public long startRegisterInstantiators() {
-    this.stats.incInt(registerInstantiatorsInProgressId, 1);
-    this.sendStats.incInt(registerInstantiatorsSendInProgressId, 1);
-    startClientOp();
-    return getStatTime();
-  }
-
-  public long startRegisterDataSerializers() {
-    this.stats.incInt(registerDataSerializersInProgressId, 1);
-    this.sendStats.incInt(registerDataSerializersSendInProgressId, 1);
-    startClientOp();
-    return getStatTime();
-  }
-
-  /**
-   * Records that the send part of the registerInstantiators has completed
-   *
-   * @param startTime the value returned by {@link #startRegisterInstantiators}.
-   * @param failed true if the send of the registerInstantiators failed
-   */
-  public void endRegisterInstantiatorsSend(long startTime, boolean failed) {
-    long duration = getStatTime() - startTime;
-    endClientOpSend(duration, failed);
-    this.sendStats.incInt(registerInstantiatorsSendInProgressId, -1);
-    int endRegisterInstantiatorsSendId;
-    if (failed) {
-      endRegisterInstantiatorsSendId = registerInstantiatorsSendFailedId;
-    } else {
-      endRegisterInstantiatorsSendId = registerInstantiatorsSendId;
-    }
-    this.sendStats.incInt(endRegisterInstantiatorsSendId, 1);
-    this.stats.incLong(registerInstantiatorsSendDurationId, duration);
-  }
-
-  public void endRegisterDataSerializersSend(long startTime, boolean failed) {
-    long duration = getStatTime() - startTime;
-    endClientOpSend(duration, failed);
-    this.sendStats.incInt(registerDataSerializersSendInProgressId, -1);
-    int endDataSerializersSendId;
-    if (failed) {
-      endDataSerializersSendId = registerDataSerializersSendFailedId;
-    } else {
-      endDataSerializersSendId = registerDataSerializersSendId;
-    }
-    this.sendStats.incInt(endDataSerializersSendId, 1);
-    this.stats.incLong(registerDataSerializersSendDurationId, duration);
-  }
-
-  /**
-   * Records that the specified registerInstantiators has ended
-   *
-   * @param startTime the value returned by {@link #startRegisterInstantiators}.
-   * @param timedOut true if registerInstantiators timed out
-   * @param failed true if registerInstantiators failed
-   */
-  public void endRegisterInstantiators(long startTime, boolean timedOut, boolean failed) {
-    long duration = getStatTime() - startTime;
-    endClientOp(duration, timedOut, failed);
-    this.stats.incInt(registerInstantiatorsInProgressId, -1);
-    int endRegisterInstantiatorsId;
-    if (timedOut) {
-      endRegisterInstantiatorsId = registerInstantiatorsTimedOutId;
-    } else if (failed) {
-      endRegisterInstantiatorsId = registerInstantiatorsFailedId;
-    } else {
-      endRegisterInstantiatorsId = registerInstantiatorsId;
-    }
-    this.stats.incInt(endRegisterInstantiatorsId, 1);
-    this.stats.incLong(registerInstantiatorsDurationId, duration);
-  }
-
-  public void endRegisterDataSerializers(long startTime, boolean timedOut, boolean failed) {
-    long duration = getStatTime() - startTime;
-    endClientOp(duration, timedOut, failed);
-    this.stats.incInt(registerDataSerializersInProgressId, -1);
-    int endRegisterDataSerializersId;
-    if (timedOut) {
-      endRegisterDataSerializersId = registerDataSerializersTimedOutId;
-    } else if (failed) {
-      endRegisterDataSerializersId = registerDataSerializersFailedId;
-    } else {
-      endRegisterDataSerializersId = registerDataSerializersId;
-    }
-    this.stats.incInt(endRegisterDataSerializersId, 1);
-    this.stats.incLong(registerDataSerializersDurationId, duration);
-  }
-
-  /**
-   * Records that the specified putAll is starting
-   * <p>
-   * Note: for every call of this method the caller must also call {@link #endPutAllSend} and
-   * {@link #endPutAll}.
-   *
-   * @return the start time of this putAll
-   */
-  public long startPutAll() {
-    this.stats.incInt(putAllInProgressId, 1);
-    this.sendStats.incInt(putAllSendInProgressId, 1);
-    startClientOp();
-    return getStatTime();
-  }
-
-  /**
-   * Records that the send part of the putAll has completed
-   *
-   * @param startTime the value returned by {@link #startPutAll}.
-   * @param failed true if the send of the putAll failed
-   */
-  public void endPutAllSend(long startTime, boolean failed) {
-    long duration = getStatTime() - startTime;
-    endClientOpSend(duration, failed);
-    this.sendStats.incInt(putAllSendInProgressId, -1);
-    int endPutAllSendId;
-    if (failed) {
-      endPutAllSendId = putAllSendFailedId;
-    } else {
-      endPutAllSendId = putAllSendId;
-    }
-    this.sendStats.incInt(endPutAllSendId, 1);
-    this.stats.incLong(putAllSendDurationId, duration);
-  }
-
-  /**
-   * Records that the specified putAll has ended
-   *
-   * @param startTime the value returned by {@link #startPutAll}.
-   * @param timedOut true if putAll timed out
-   * @param failed true if putAll failed
-   */
-  public void endPutAll(long startTime, boolean timedOut, boolean failed) {
-    long duration = getStatTime() - startTime;
-    endClientOp(duration, timedOut, failed);
-    this.stats.incInt(putAllInProgressId, -1);
-    int endPutAllId;
-    if (timedOut) {
-      endPutAllId = putAllTimedOutId;
-    } else if (failed) {
-      endPutAllId = putAllFailedId;
-    } else {
-      endPutAllId = putAllId;
-    }
-    this.stats.incInt(endPutAllId, 1);
-    this.stats.incLong(putAllDurationId, duration);
-  }
-
-  /**
-   * Records that the specified removeAll is starting
-   * <p>
-   * Note: for every call of this method the caller must also call {@link #endRemoveAllSend} and
-   * {@link #endRemoveAll}.
-   *
-   * @return the start time of this removeAll
-   */
-  public long startRemoveAll() {
-    this.stats.incInt(removeAllInProgressId, 1);
-    this.sendStats.incInt(removeAllSendInProgressId, 1);
-    startClientOp();
-    return getStatTime();
-  }
-
-  /**
-   * Records that the send part of the removeAll has completed
-   *
-   * @param startTime the value returned by {@link #startRemoveAll}.
-   * @param failed true if the send of the removeAll failed
-   */
-  public void endRemoveAllSend(long startTime, boolean failed) {
-    long duration = getStatTime() - startTime;
-    endClientOpSend(duration, failed);
-    this.sendStats.incInt(removeAllSendInProgressId, -1);
-    int endRemoveAllSendId;
-    if (failed) {
-      endRemoveAllSendId = removeAllSendFailedId;
-    } else {
-      endRemoveAllSendId = removeAllSendId;
-    }
-    this.sendStats.incInt(endRemoveAllSendId, 1);
-    this.stats.incLong(removeAllSendDurationId, duration);
-  }
-
-  /**
-   * Records that the specified removeAll has ended
-   *
-   * @param startTime the value returned by {@link #startRemoveAll}.
-   * @param timedOut true if removeAll timed out
-   * @param failed true if removeAll failed
-   */
-  public void endRemoveAll(long startTime, boolean timedOut, boolean failed) {
-    long duration = getStatTime() - startTime;
-    endClientOp(duration, timedOut, failed);
-    this.stats.incInt(removeAllInProgressId, -1);
-    int endRemoveAllId;
-    if (timedOut) {
-      endRemoveAllId = removeAllTimedOutId;
-    } else if (failed) {
-      endRemoveAllId = removeAllFailedId;
-    } else {
-      endRemoveAllId = removeAllId;
-    }
-    this.stats.incInt(endRemoveAllId, 1);
-    this.stats.incLong(removeAllDurationId, duration);
-  }
-
-  /**
-   * Records that the specified getAll is starting
-   * <p>
-   * Note: for every call of this method the caller must also call {@link #endGetAllSend} and
-   * {@link #endGetAll}.
-   *
-   * @return the start time of this getAll
-   */
-  public long startGetAll() {
-    this.stats.incInt(getAllInProgressId, 1);
-    this.sendStats.incInt(getAllSendInProgressId, 1);
-    startClientOp();
-    return getStatTime();
-  }
-
-  /**
-   * Records that the send part of the getAll has completed
-   *
-   * @param startTime the value returned by {@link #startGetAll}.
-   * @param failed true if the send of the getAll failed
-   */
-  public void endGetAllSend(long startTime, boolean failed) {
-    long duration = getStatTime() - startTime;
-    endClientOpSend(duration, failed);
-    this.sendStats.incInt(getAllSendInProgressId, -1);
-    int endGetAllSendId;
-    if (failed) {
-      endGetAllSendId = getAllSendFailedId;
-    } else {
-      endGetAllSendId = getAllSendId;
-    }
-    this.sendStats.incInt(endGetAllSendId, 1);
-    this.stats.incLong(getAllSendDurationId, duration);
-  }
-
-  /**
-   * Records that the specified getAll has ended
-   *
-   * @param startTime the value returned by {@link #startGetAll}.
-   * @param timedOut true if getAll timed out
-   * @param failed true if getAll failed
-   */
-  public void endGetAll(long startTime, boolean timedOut, boolean failed) {
-    long duration = getStatTime() - startTime;
-    endClientOp(duration, timedOut, failed);
-    this.stats.incInt(getAllInProgressId, -1);
-    int endGetAllId;
-    if (timedOut) {
-      endGetAllId = getAllTimedOutId;
-    } else if (failed) {
-      endGetAllId = getAllFailedId;
-    } else {
-      endGetAllId = getAllId;
-    }
-    this.stats.incInt(endGetAllId, 1);
-    this.stats.incLong(getAllDurationId, duration);
-  }
-
-  public int getConnections() {
-    return this.stats.getInt(connectionsId);
-  }
-
-  public int getOps() {
-    int ops = 0;
-    for (int i = 0; i < opIds.length; i++) {
-      ops += this.stats.getInt(i);
-    }
-    return ops;
-  }
-
-  public void incConnections(int delta) {
-    this.stats.incInt(connectionsId, delta);
-    if (delta > 0) {
-      this.stats.incInt(connectsId, delta);
-    } else if (delta < 0) {
-      this.stats.incInt(disconnectsId, -delta);
-    }
-    this.poolStats.incConnections(delta);
-  }
-
-  private void startClientOp() {
-    this.poolStats.startClientOp();
-  }
-
-  private void endClientOpSend(long duration, boolean failed) {
-    this.poolStats.endClientOpSend(duration, failed);
-  }
-
-  private void endClientOp(long duration, boolean timedOut, boolean failed) {
-    this.poolStats.endClientOp(duration, timedOut, failed);
-  }
-
-  public void close() {
-    this.stats.close();
-    this.sendStats.close();
-  }
-
-  public void incReceivedBytes(long v) {
-    this.stats.incLong(receivedBytesId, v);
-  }
-
-  public void incSentBytes(long v) {
-    this.stats.incLong(sentBytesId, v);
-  }
-
-  public void incMessagesBeingReceived(int bytes) {
-    stats.incInt(messagesBeingReceivedId, 1);
-    if (bytes > 0) {
-      stats.incLong(messageBytesBeingReceivedId, bytes);
-    }
-  }
-
-  public void decMessagesBeingReceived(int bytes) {
-    stats.incInt(messagesBeingReceivedId, -1);
-    if (bytes > 0) {
-      stats.incLong(messageBytesBeingReceivedId, -bytes);
-    }
-  }
-
-  /**
-   * Records that the specified execute Function is starting
-   * <p>
-   * Note: for every call of this method the caller must also call {@link #endExecuteFunctionSend}
-   * and {@link #endExecuteFunction}.
-   *
-   * @return the start time of this ExecuteFunction
-   */
-  public long startExecuteFunction() {
-    this.stats.incInt(executeFunctionInProgressId, 1);
-    this.sendStats.incInt(executeFunctionSendInProgressId, 1);
-    return getStatTime();
-  }
-
-  /**
-   * Records that the send part of the executeFunction has completed
-   *
-   * @param startTime the value returned by {@link #startExecuteFunction}.
-   * @param failed true if the send of the executeFunction failed
-   */
-  public void endExecuteFunctionSend(long startTime, boolean failed) {
-    long duration = getStatTime() - startTime;
-    this.sendStats.incInt(executeFunctionSendInProgressId, -1);
-    int endExecuteFunctionSendId;
-    if (failed) {
-      endExecuteFunctionSendId = executeFunctionSendFailedId;
-    } else {
-      endExecuteFunctionSendId = executeFunctionSendId;
-    }
-    this.sendStats.incInt(endExecuteFunctionSendId, 1);
-    this.stats.incLong(executeFunctionSendDurationId, duration);
-  }
-
-  /**
-   * Records that the specified executeFunction has ended
-   *
-   * @param startTime the value returned by {@link #startExecuteFunction}.
-   * @param timedOut true if executeFunction timed out
-   * @param failed true if executeFunction failed
-   */
-  public void endExecuteFunction(long startTime, boolean timedOut, boolean failed) {
-    long duration = getStatTime() - startTime;
-    this.stats.incInt(executeFunctionInProgressId, -1);
-    int endExecuteFunctionId;
-    if (timedOut) {
-      endExecuteFunctionId = executeFunctionTimedOutId;
-    } else if (failed) {
-      endExecuteFunctionId = executeFunctionFailedId;
-    } else {
-      endExecuteFunctionId = executeFunctionId;
-    }
-    this.stats.incInt(endExecuteFunctionId, 1);
-    this.stats.incLong(executeFunctionDurationId, duration);
-  }
-
-  public int getExecuteFunctions() {
-    return this.stats.getInt(executeFunctionId);
-  }
-
-  public long getExecuteFunctionDuration() {
-    return this.stats.getLong(executeFunctionDurationId);
-  }
-
-  public int getGetDurableCqs() {
-    return this.stats.getInt(getDurableCQsId);
-  }
-
-  /**
-   * Records that the specified GetClientPRMetadata operation is starting
-   * <p>
-   * Note: for every call of this method the caller must also call
-   * {@link #endGetClientPRMetadataSend} and {@link #endGetClientPRMetadata}.
-   *
-   * @return the start time of this ExecuteFunction
-   */
-  public long startGetClientPRMetadata() {
-    this.stats.incInt(getClientPRMetadataInProgressId, 1);
-    this.sendStats.incInt(getClientPRMetadataSendInProgressId, 1);
-    startClientOp();
-    return getStatTime();
-  }
-
-  /**
-   * Records that the send part of the GetClientPRMetadata has completed
-   *
-   * @param startTime the value returned by {@link #startGetClientPRMetadata}.
-   * @param failed true if the send of the GetClientPRMetadata failed
-   */
-  public void endGetClientPRMetadataSend(long startTime, boolean failed) {
-    long duration = getStatTime() - startTime;
-    endClientOpSend(duration, failed);
-    this.sendStats.incInt(getClientPRMetadataSendInProgressId, -1);
-    int endGetClientPRMetadataSendId;
-    if (failed) {
-      endGetClientPRMetadataSendId = getClientPRMetadataSendFailedId;
-    } else {
-      endGetClientPRMetadataSendId = getClientPRMetadataSendId;
-    }
-    this.sendStats.incInt(endGetClientPRMetadataSendId, 1);
-    this.stats.incLong(getClientPRMetadataSendDurationId, duration);
-  }
-
-  /**
-   * Records that the specified GetClientPRMetadata has ended
-   *
-   * @param startTime the value returned by {@link #startGetClientPRMetadata}.
-   * @param timedOut true if GetClientPRMetadata timed out
-   * @param failed true if GetClientPRMetadata failed
-   */
-  public void endGetClientPRMetadata(long startTime, boolean timedOut, boolean failed) {
-    long duration = getStatTime() - startTime;
-    endClientOp(duration, timedOut, failed);
-    this.stats.incInt(getClientPRMetadataInProgressId, -1);
-    int endGetClientPRMetadataId;
-    if (timedOut) {
-      endGetClientPRMetadataId = getClientPRMetadataTimedOutId;
-    } else if (failed) {
-      endGetClientPRMetadataId = getClientPRMetadataFailedId;
-    } else {
-      endGetClientPRMetadataId = getClientPRMetadataId;
-    }
-    this.stats.incInt(endGetClientPRMetadataId, 1);
-    this.stats.incLong(getClientPRMetadataDurationId, duration);
-  }
-
-  /**
-   * Records that the specified GetClientPartitionAttributes operation is starting
-   * <p>
-   * Note: for every call of this method the caller must also call
-   * {@link #endGetClientPartitionAttributesSend} and {@link #endGetClientPartitionAttributes}.
-   *
-   * @return the start time of this GetClientPartitionAttributes
-   */
-  public long startGetClientPartitionAttributes() {
-    this.stats.incInt(getClientPartitionAttributesInProgressId, 1);
-    this.sendStats.incInt(getClientPartitionAttributesSendInProgressId, 1);
-    startClientOp();
-    return getStatTime();
-  }
-
-  /**
-   * Records that the send part of the GetClientPartitionAttributes operation has completed
-   *
-   * @param startTime the value returned by {@link #startGetClientPartitionAttributes}.
-   * @param failed true if the send of the GetClientPartitionAttributes failed
-   */
-  public void endGetClientPartitionAttributesSend(long startTime, boolean failed) {
-    long duration = getStatTime() - startTime;
-    endClientOpSend(duration, failed);
-    this.sendStats.incInt(getClientPartitionAttributesSendInProgressId, -1);
-    int endGetClientPartitionAttributesSendId;
-    if (failed) {
-      endGetClientPartitionAttributesSendId = getClientPartitionAttributesSendFailedId;
-    } else {
-      endGetClientPartitionAttributesSendId = getClientPartitionAttributesSendId;
-    }
-    this.sendStats.incInt(endGetClientPartitionAttributesSendId, 1);
-    this.stats.incLong(getClientPartitionAttributesSendDurationId, duration);
-  }
-
-  /**
-   * Records that the specified GetClientPartitionAttributes has ended
-   *
-   * @param startTime the value returned by {@link #startGetClientPartitionAttributes}.
-   * @param timedOut true if GetClientPartitionAttributes timed out
-   * @param failed true if GetClientPartitionAttributes failed
-   */
-  public void endGetClientPartitionAttributes(long startTime, boolean timedOut, boolean failed) {
-    long duration = getStatTime() - startTime;
-    endClientOp(duration, timedOut, failed);
-    this.stats.incInt(getClientPartitionAttributesInProgressId, -1);
-    int endGetClientPartitionAttributesId;
-    if (timedOut) {
-      endGetClientPartitionAttributesId = getClientPartitionAttributesTimedOutId;
-    } else if (failed) {
-      endGetClientPartitionAttributesId = getClientPartitionAttributesFailedId;
-    } else {
-      endGetClientPartitionAttributesId = getClientPartitionAttributesId;
-    }
-    this.stats.incInt(endGetClientPartitionAttributesId, 1);
-    this.stats.incLong(getClientPartitionAttributesDurationId, duration);
-  }
-
-  public long startGetPDXTypeById() {
-    this.stats.incInt(getPDXTypeByIdInProgressId, 1);
-    this.sendStats.incInt(getPDXTypeByIdSendInProgressId, 1);
-    startClientOp();
-    return getStatTime();
-  }
-
-  public long startGetPDXIdForType() {
-    this.stats.incInt(getPDXIdForTypeInProgressId, 1);
-    this.sendStats.incInt(getPDXIdForTypeSendInProgressId, 1);
-    startClientOp();
-    return getStatTime();
-  }
-
-  public void endGetPDXTypeByIdSend(long startTime, boolean failed) {
-    long duration = getStatTime() - startTime;
-    endClientOpSend(duration, failed);
-    this.sendStats.incInt(getPDXTypeByIdSendInProgressId, -1);
-    int endGetPDXTypeByIdSendId;
-    if (failed) {
-      endGetPDXTypeByIdSendId = getPDXTypeByIdSendFailedId;
-    } else {
-      endGetPDXTypeByIdSendId = getPDXTypeByIdSendId;
-    }
-    this.sendStats.incInt(endGetPDXTypeByIdSendId, 1);
-    this.stats.incLong(getPDXTypeByIdSendDurationId, duration);
-  }
-
-  public void endGetPDXIdForTypeSend(long startTime, boolean failed) {
-    long duration = getStatTime() - startTime;
-    endClientOpSend(duration, failed);
-    this.sendStats.incInt(getPDXIdForTypeSendInProgressId, -1);
-    int endGetPDXIdForTypeSendId;
-    if (failed) {
-      endGetPDXIdForTypeSendId = getPDXIdForTypeSendFailedId;
-    } else {
-      endGetPDXIdForTypeSendId = getPDXIdForTypeSendId;
-    }
-    this.stats.incInt(endGetPDXIdForTypeSendId, 1);
-    this.stats.incLong(getPDXIdForTypeSendDurationId, duration);
-  }
-
-  public void endGetPDXTypeById(long startTime, boolean timedOut, boolean failed) {
-    long duration = getStatTime() - startTime;
-    endClientOp(duration, timedOut, failed);
-    this.stats.incInt(getPDXTypeByIdInProgressId, -1);
-    int statId;
-    if (timedOut) {
-      statId = getPDXTypeByIdTimedOutId;
-    } else if (failed) {
-      statId = getPDXTypeByIdFailedId;
-    } else {
-      statId = getPDXTypeByIdId;
-    }
-    this.stats.incInt(statId, 1);
-    this.stats.incLong(getPDXTypeByIdDurationId, duration);
-  }
-
-  public void endGetPDXIdForType(long startTime, boolean timedOut, boolean failed) {
-    long duration = getStatTime() - startTime;
-    endClientOp(duration, timedOut, failed);
-    this.stats.incInt(getPDXIdForTypeInProgressId, -1);
-    int statId;
-    if (timedOut) {
-      statId = getPDXIdForTypeTimedOutId;
-    } else if (failed) {
-      statId = getPDXIdForTypeFailedId;
-    } else {
-      statId = getPDXIdForTypeId;
-    }
-    this.stats.incInt(statId, 1);
-    this.stats.incLong(getPDXIdForTypeDurationId, duration);
-  }
-
-  public long startAddPdxType() {
-    this.stats.incInt(addPdxTypeInProgressId, 1);
-    this.sendStats.incInt(addPdxTypeSendInProgressId, 1);
-    startClientOp();
-    return getStatTime();
-  }
-
-  public void endAddPdxTypeSend(long startTime, boolean failed) {
-    long duration = getStatTime() - startTime;
-    endClientOpSend(duration, failed);
-    this.sendStats.incInt(addPdxTypeSendInProgressId, -1);
-    int endAddPdxTypeSendId;
-    if (failed) {
-      endAddPdxTypeSendId = addPdxTypeSendFailedId;
-    } else {
-      endAddPdxTypeSendId = addPdxTypeSendId;
-    }
-    this.sendStats.incInt(endAddPdxTypeSendId, 1);
-    this.stats.incLong(addPdxTypeSendDurationId, duration);
-  }
-
-  public void endAddPdxType(long startTime, boolean timedOut, boolean failed) {
-    long duration = getStatTime() - startTime;
-    endClientOp(duration, timedOut, failed);
-    this.stats.incInt(addPdxTypeInProgressId, -1);
-    int statId;
-    if (timedOut) {
-      statId = addPdxTypeTimedOutId;
-    } else if (failed) {
-      statId = addPdxTypeFailedId;
-    } else {
-      statId = addPdxTypeId;
-    }
-    this.stats.incInt(statId, 1);
-    this.stats.incLong(addPdxTypeDurationId, duration);
-  }
-
-  public long startSize() {
-    this.stats.incInt(sizeInProgressId, 1);
-    this.sendStats.incInt(sizeSendInProgressId, 1);
-    startClientOp();
-    return getStatTime();
-  }
-
-  public void endSizeSend(long startTime, boolean failed) {
-    long duration = getStatTime() - startTime;
-    endClientOpSend(duration, failed);
-    this.sendStats.incInt(sizeSendInProgressId, -1);
-    int endSizeSendId;
-    if (failed) {
-      endSizeSendId = sizeSendFailedId;
-    } else {
-      endSizeSendId = sizeSendId;
-    }
-    this.sendStats.incInt(endSizeSendId, 1);
-    this.stats.incLong(sizeSendDurationId, duration);
-
-  }
-
-  public void endSize(long startTime, boolean timedOut, boolean failed) {
-    long duration = getStatTime() - startTime;
-    endClientOp(duration, timedOut, failed);
-    this.stats.incInt(sizeInProgressId, -1);
-    int endSizeId;
-    if (timedOut) {
-      endSizeId = sizeTimedOutId;
-    } else if (failed) {
-      endSizeId = sizeFailedId;
-    } else {
-      endSizeId = sizeId;
-    }
-    this.stats.incInt(endSizeId, 1);
-    this.stats.incLong(sizeDurationId, duration);
-  }
-
-
-
-  public long startInvalidate() {
-    this.stats.incInt(invalidateInProgressId, 1);
-    this.sendStats.incInt(invalidateSendInProgressId, 1);
-    startClientOp();
-    return getStatTime();
-  }
-
-  public void endInvalidateSend(long startTime, boolean failed) {
-    long duration = getStatTime() - startTime;
-    endClientOpSend(duration, failed);
-    this.sendStats.incInt(invalidateSendInProgressId, -1);
-    int endInvalidateSendId;
-    if (failed) {
-      endInvalidateSendId = invalidateSendFailedId;
-    } else {
-      endInvalidateSendId = invalidateSendId;
-    }
-    this.sendStats.incInt(endInvalidateSendId, 1);
-    this.stats.incLong(invalidateSendDurationId, duration);
-  }
-
-  public void endInvalidate(long startTime, boolean timedOut, boolean failed) {
-    long duration = getStatTime() - startTime;
-    endClientOp(duration, timedOut, failed);
-    this.stats.incInt(invalidateInProgressId, -1);
-    int endInvalidateId;
-    if (timedOut) {
-      endInvalidateId = invalidateTimedOutId;
-    } else if (failed) {
-      endInvalidateId = invalidateFailedId;
-    } else {
-      endInvalidateId = invalidateId;
-    }
-    this.stats.incInt(endInvalidateId, 1);
-    this.stats.incLong(invalidateDurationId, duration);
-  }
-
-  public long startCommit() {
-    this.stats.incInt(commitInProgressId, 1);
-    this.sendStats.incInt(commitSendInProgressId, 1);
-    startClientOp();
-    return getStatTime();
-  }
-
-  public void endCommitSend(long startTime, boolean failed) {
-    long duration = getStatTime() - startTime;
-    endClientOpSend(duration, failed);
-    this.sendStats.incInt(commitSendInProgressId, -1);
-    int endcommitSendId;
-    if (failed) {
-      endcommitSendId = commitSendFailedId;
-    } else {
-      endcommitSendId = commitSendId;
-    }
-    this.sendStats.incInt(endcommitSendId, 1);
-    this.stats.incLong(commitSendDurationId, duration);
-  }
-
-  public void endCommit(long startTime, boolean timedOut, boolean failed) {
-    long duration = getStatTime() - startTime;
-    endClientOp(duration, timedOut, failed);
-    this.stats.incInt(commitInProgressId, -1);
-    int endcommitId;
-    if (timedOut) {
-      endcommitId = commitTimedOutId;
-    } else if (failed) {
-      endcommitId = commitFailedId;
-    } else {
-      endcommitId = commitId;
-    }
-    this.stats.incInt(endcommitId, 1);
-    this.stats.incLong(commitDurationId, duration);
-  }
-
-
-  public long startGetEntry() {
-    this.stats.incInt(getEntryInProgressId, 1);
-    this.sendStats.incInt(getEntrySendInProgressId, 1);
-    startClientOp();
-    return getStatTime();
-  }
-
-  public void endGetEntrySend(long startTime, boolean failed) {
-    long duration = getStatTime() - startTime;
-    endClientOpSend(duration, failed);
-    this.sendStats.incInt(getEntrySendInProgressId, -1);
-    int endGetEntrySendId;
-    if (failed) {
-      endGetEntrySendId = getEntrySendFailedId;
-    } else {
-      endGetEntrySendId = getEntrySendId;
-    }
-    this.sendStats.incInt(endGetEntrySendId, 1);
-    this.stats.incLong(getEntrySendDurationId, duration);
-  }
-
-  public void endGetEntry(long startTime, boolean timedOut, boolean failed) {
-    long duration = getStatTime() - startTime;
-    endClientOp(duration, timedOut, failed);
-    this.stats.incInt(getEntryInProgressId, -1);
-    int endGetEntryId;
-    if (timedOut) {
-      endGetEntryId = getEntryTimedOutId;
-    } else if (failed) {
-      endGetEntryId = getEntryFailedId;
-    } else {
-      endGetEntryId = getEntryId;
-    }
-    this.stats.incInt(endGetEntryId, 1);
-    this.stats.incLong(getEntryDurationId, duration);
-  }
-
-
-  public long startRollback() {
-    this.stats.incInt(rollbackInProgressId, 1);
-    this.sendStats.incInt(rollbackSendInProgressId, 1);
-    startClientOp();
-    return getStatTime();
-  }
-
-  public void endRollbackSend(long startTime, boolean failed) {
-    long duration = getStatTime() - startTime;
-    endClientOpSend(duration, failed);
-    this.sendStats.incInt(rollbackSendInProgressId, -1);
-    int endRollbackSendId;
-    if (failed) {
-      endRollbackSendId = rollbackSendFailedId;
-    } else {
-      endRollbackSendId = rollbackSendId;
-    }
-    this.sendStats.incInt(endRollbackSendId, 1);
-    this.stats.incLong(rollbackSendDurationId, duration);
-  }
-
-  public void endRollback(long startTime, boolean timedOut, boolean failed) {
-    long duration = getStatTime() - startTime;
-    endClientOp(duration, timedOut, failed);
-    this.stats.incInt(rollbackInProgressId, -1);
-    int endRollbackId;
-    if (timedOut) {
-      endRollbackId = rollbackTimedOutId;
-    } else if (failed) {
-      endRollbackId = rollbackFailedId;
-    } else {
-      endRollbackId = rollbackId;
-    }
-    this.stats.incInt(endRollbackId, 1);
-    this.stats.incLong(rollbackDurationId, duration);
-  }
-
-
-
-  public long startTxFailover() {
-    this.stats.incInt(txFailoverInProgressId, 1);
-    this.sendStats.incInt(txFailoverSendInProgressId, 1);
-    startClientOp();
-    return getStatTime();
-  }
-
-  public void endTxFailoverSend(long startTime, boolean failed) {
-    long duration = getStatTime() - startTime;
-    endClientOpSend(duration, failed);
-    this.sendStats.incInt(txFailoverSendInProgressId, -1);
-    int endTxFailoverSendId;
-    if (failed) {
-      endTxFailoverSendId = txFailoverSendFailedId;
-    } else {
-      endTxFailoverSendId = txFailoverSendId;
-    }
-    this.sendStats.incInt(endTxFailoverSendId, 1);
-    this.stats.incLong(txFailoverSendDurationId, duration);
-  }
-
-  public void endTxFailover(long startTime, boolean timedOut, boolean failed) {
-    long duration = getStatTime() - startTime;
-    endClientOp(duration, timedOut, failed);
-    this.stats.incInt(txFailoverInProgressId, -1);
-    int endTxFailoverId;
-    if (timedOut) {
-      endTxFailoverId = txFailoverTimedOutId;
-    } else if (failed) {
-      endTxFailoverId = txFailoverFailedId;
-    } else {
-      endTxFailoverId = txFailoverId;
-    }
-    this.stats.incInt(endTxFailoverId, 1);
-    this.stats.incLong(txFailoverDurationId, duration);
-  }
-
-
-  public long startTxSynchronization() {
-    this.stats.incInt(txSynchronizationInProgressId, 1);
-    this.sendStats.incInt(txSynchronizationSendInProgressId, 1);
-    startClientOp();
-    return getStatTime();
-  }
-
-  public void endTxSynchronizationSend(long startTime, boolean failed) {
-    long duration = getStatTime() - startTime;
-    endClientOpSend(duration, failed);
-    this.sendStats.incInt(txSynchronizationSendInProgressId, -1);
-    int endTxSynchronizationSendId;
-    if (failed) {
-      endTxSynchronizationSendId = txSynchronizationSendFailedId;
-    } else {
-      endTxSynchronizationSendId = txSynchronizationSendId;
-    }
-    this.sendStats.incInt(endTxSynchronizationSendId, 1);
-    this.stats.incLong(txSynchronizationSendDurationId, duration);
-  }
-
-  public void endTxSynchronization(long startTime, boolean timedOut, boolean failed) {
-    long duration = getStatTime() - startTime;
-    endClientOp(duration, timedOut, failed);
-    this.stats.incInt(txSynchronizationInProgressId, -1);
-    int endTxSynchronizationId;
-    if (timedOut) {
-      endTxSynchronizationId = txSynchronizationTimedOutId;
-    } else if (failed) {
-      endTxSynchronizationId = txSynchronizationFailedId;
-    } else {
-      endTxSynchronizationId = txSynchronizationId;
-    }
-    this.stats.incInt(endTxSynchronizationId, 1);
-    this.stats.incLong(txSynchronizationDurationId, duration);
-  }
-}
diff --git a/geode-core/src/main/java/org/apache/geode/cache/client/internal/ConnectionStatsImpl.java b/geode-core/src/main/java/org/apache/geode/cache/client/internal/ConnectionStatsImpl.java
new file mode 100644
index 0000000..91fdbaf
--- /dev/null
+++ b/geode-core/src/main/java/org/apache/geode/cache/client/internal/ConnectionStatsImpl.java
@@ -0,0 +1,3289 @@
+/*
+ * 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.cache.client.internal;
+
+import org.apache.geode.stats.common.cache.client.internal.ConnectionStats;
+import org.apache.geode.stats.common.internal.cache.PoolStats;
+import org.apache.geode.stats.common.statistics.GFSStatsImplementer;
+import org.apache.geode.stats.common.statistics.StatisticDescriptor;
+import org.apache.geode.stats.common.statistics.Statistics;
+import org.apache.geode.stats.common.statistics.StatisticsFactory;
+import org.apache.geode.stats.common.statistics.StatisticsType;
+
+/**
+ * Stats for a client to server {@link Connection}
+ *
+ * @since GemFire 5.7
+ */
+public class ConnectionStatsImpl implements ConnectionStats, GFSStatsImplementer {
+  // static fields
+  private StatisticsType type;
+  private StatisticsType sendType;
+
+  ///////////////////////////////////////////////////////////////////////
+  /*
+   * private int opInProgressId; private int opSendInProgressId; private
+   * static final int opSendFailedId; private int opSendId; private int
+   * opSendDurationId; private int opTimedOutId; private int opFailedId;
+   * private int opId; private int opDurationId;
+   */
+  ///////////////////////////////////////////////////////////////////////
+
+  private int getInProgressId;
+  private int getSendInProgressId;
+  private int getSendFailedId;
+  private int getSendId;
+  private int getSendDurationId;
+  private int getTimedOutId;
+  private int getFailedId;
+  private int getId;
+  private int getDurationId;
+
+  private int putInProgressId;
+  private int putSendInProgressId;
+  private int putSendFailedId;
+  private int putSendId;
+  private int putSendDurationId;
+  private int putTimedOutId;
+  private int putFailedId;
+  private int putId;
+  private int putDurationId;
+
+  private int destroyInProgressId;
+  private int destroySendInProgressId;
+  private int destroySendFailedId;
+  private int destroySendId;
+  private int destroySendDurationId;
+  private int destroyTimedOutId;
+  private int destroyFailedId;
+  private int destroyId;
+  private int destroyDurationId;
+
+  private int destroyRegionInProgressId;
+  private int destroyRegionSendInProgressId;
+  private int destroyRegionSendFailedId;
+  private int destroyRegionSendId;
+  private int destroyRegionSendDurationId;
+  private int destroyRegionTimedOutId;
+  private int destroyRegionFailedId;
+  private int destroyRegionId;
+  private int destroyRegionDurationId;
+
+  private int clearInProgressId;
+  private int clearSendInProgressId;
+  private int clearSendFailedId;
+  private int clearSendId;
+  private int clearSendDurationId;
+  private int clearTimedOutId;
+  private int clearFailedId;
+  private int clearId;
+  private int clearDurationId;
+
+  private int containsKeyInProgressId;
+  private int containsKeySendInProgressId;
+  private int containsKeySendFailedId;
+  private int containsKeySendId;
+  private int containsKeySendDurationId;
+  private int containsKeyTimedOutId;
+  private int containsKeyFailedId;
+  private int containsKeyId;
+  private int containsKeyDurationId;
+
+  private int keySetInProgressId;
+  private int keySetSendInProgressId;
+  private int keySetSendFailedId;
+  private int keySetSendId;
+  private int keySetSendDurationId;
+  private int keySetTimedOutId;
+  private int keySetFailedId;
+  private int keySetId;
+  private int keySetDurationId;
+
+  private int commitInProgressId;
+  private int commitSendInProgressId;
+  private int commitSendFailedId;
+  private int commitSendId;
+  private int commitSendDurationId;
+
+  private int commitFailedId;
+  private int commitTimedOutId;
+  private int commitId;
+  private int commitDurationId;
+
+  private int rollbackInProgressId;
+  private int rollbackSendInProgressId;
+  private int rollbackSendFailedId;
+  private int rollbackSendId;
+  private int rollbackSendDurationId;
+
+  private int rollbackFailedId;
+  private int rollbackTimedOutId;
+  private int rollbackId;
+  private int rollbackDurationId;
+
+  private int getEntryInProgressId;
+  private int getEntrySendInProgressId;
+  private int getEntrySendFailedId;
+  private int getEntrySendId;
+  private int getEntrySendDurationId;
+
+  private int getEntryFailedId;
+  private int getEntryTimedOutId;
+  private int getEntryId;
+  private int getEntryDurationId;
+
+  private int txSynchronizationInProgressId;
+  private int txSynchronizationSendInProgressId;
+  private int txSynchronizationSendFailedId;
+  private int txSynchronizationSendId;
+  private int txSynchronizationSendDurationId;
+
+  private int txSynchronizationFailedId;
+  private int txSynchronizationTimedOutId;
+  private int txSynchronizationId;
+  private int txSynchronizationDurationId;
+
+  private int txFailoverInProgressId;
+  private int txFailoverSendInProgressId;
+  private int txFailoverSendFailedId;
+  private int txFailoverSendId;
+  private int txFailoverSendDurationId;
+
+  private int txFailoverFailedId;
+  private int txFailoverTimedOutId;
+  private int txFailoverId;
+  private int txFailoverDurationId;
+
+  private int sizeInProgressId;
+  private int sizeSendInProgressId;
+  private int sizeSendFailedId;
+  private int sizeSendId;
+  private int sizeSendDurationId;
+
+  private int sizeFailedId;
+  private int sizeTimedOutId;
+  private int sizeId;
+  private int sizeDurationId;
+
+  private int invalidateInProgressId;
+  private int invalidateSendInProgressId;
+  private int invalidateSendFailedId;
+  private int invalidateSendId;
+  private int invalidateSendDurationId;
+
+  private int invalidateFailedId;
+  private int invalidateTimedOutId;
+  private int invalidateId;
+  private int invalidateDurationId;
+
+
+  private int registerInterestInProgressId;
+  private int registerInterestSendInProgressId;
+  private int registerInterestSendFailedId;
+  private int registerInterestSendId;
+  private int registerInterestSendDurationId;
+  private int registerInterestTimedOutId;
+  private int registerInterestFailedId;
+  private int registerInterestId;
+  private int registerInterestDurationId;
+
+  private int unregisterInterestInProgressId;
+  private int unregisterInterestSendInProgressId;
+  private int unregisterInterestSendFailedId;
+  private int unregisterInterestSendId;
+  private int unregisterInterestSendDurationId;
+  private int unregisterInterestTimedOutId;
+  private int unregisterInterestFailedId;
+  private int unregisterInterestId;
+  private int unregisterInterestDurationId;
+
+  private int queryInProgressId;
+  private int querySendInProgressId;
+  private int querySendFailedId;
+  private int querySendId;
+  private int querySendDurationId;
+  private int queryTimedOutId;
+  private int queryFailedId;
+  private int queryId;
+  private int queryDurationId;
+
+  private int createCQInProgressId;
+  private int createCQSendInProgressId;
+  private int createCQSendFailedId;
+  private int createCQSendId;
+  private int createCQSendDurationId;
+  private int createCQTimedOutId;
+  private int createCQFailedId;
+  private int createCQId;
+  private int createCQDurationId;
+  private int stopCQInProgressId;
+  private int stopCQSendInProgressId;
+  private int stopCQSendFailedId;
+  private int stopCQSendId;
+  private int stopCQSendDurationId;
+  private int stopCQTimedOutId;
+  private int stopCQFailedId;
+  private int stopCQId;
+  private int stopCQDurationId;
+  private int closeCQInProgressId;
+  private int closeCQSendInProgressId;
+  private int closeCQSendFailedId;
+  private int closeCQSendId;
+  private int closeCQSendDurationId;
+  private int closeCQTimedOutId;
+  private int closeCQFailedId;
+  private int closeCQId;
+  private int closeCQDurationId;
+  private int gatewayBatchInProgressId;
+  private int gatewayBatchSendInProgressId;
+  private int gatewayBatchSendFailedId;
+  private int gatewayBatchSendId;
+  private int gatewayBatchSendDurationId;
+  private int gatewayBatchTimedOutId;
+  private int gatewayBatchFailedId;
+  private int gatewayBatchId;
+  private int gatewayBatchDurationId;
+  private int getDurableCQsInProgressId;
+  private int getDurableCQsSendsInProgressId;
+  private int getDurableCQsSendFailedId;
+  private int getDurableCQsSendId;
+  private int getDurableCQsSendDurationId;
+  private int getDurableCQsTimedOutId;
+  private int getDurableCQsFailedId;
+  private int getDurableCQsId;
+  private int getDurableCQsDurationId;
+
+  private int readyForEventsInProgressId;
+  private int readyForEventsSendInProgressId;
+  private int readyForEventsSendFailedId;
+  private int readyForEventsSendId;
+  private int readyForEventsSendDurationId;
+  private int readyForEventsTimedOutId;
+  private int readyForEventsFailedId;
+  private int readyForEventsId;
+  private int readyForEventsDurationId;
+
+  private int makePrimaryInProgressId;
+  private int makePrimarySendInProgressId;
+  private int makePrimarySendFailedId;
+  private int makePrimarySendId;
+  private int makePrimarySendDurationId;
+  private int makePrimaryTimedOutId;
+  private int makePrimaryFailedId;
+  private int makePrimaryId;
+  private int makePrimaryDurationId;
+
+  private int closeConInProgressId;
+  private int closeConSendInProgressId;
+  private int closeConSendFailedId;
+  private int closeConSendId;
+  private int closeConSendDurationId;
+  private int closeConTimedOutId;
+  private int closeConFailedId;
+  private int closeConId;
+  private int closeConDurationId;
+
+  private int primaryAckInProgressId;
+  private int primaryAckSendInProgressId;
+  private int primaryAckSendFailedId;
+  private int primaryAckSendId;
+  private int primaryAckSendDurationId;
+  private int primaryAckTimedOutId;
+  private int primaryAckFailedId;
+  private int primaryAckId;
+  private int primaryAckDurationId;
+
+  private int pingInProgressId;
+  private int pingSendInProgressId;
+  private int pingSendFailedId;
+  private int pingSendId;
+  private int pingSendDurationId;
+  private int pingTimedOutId;
+  private int pingFailedId;
+  private int pingId;
+  private int pingDurationId;
+
+  private int registerInstantiatorsInProgressId;
+  private int registerInstantiatorsSendInProgressId;
+  private int registerInstantiatorsSendFailedId;
+  private int registerInstantiatorsSendId;
+  private int registerInstantiatorsSendDurationId;
+  private int registerInstantiatorsTimedOutId;
+  private int registerInstantiatorsFailedId;
+  private int registerInstantiatorsId;
+  private int registerInstantiatorsDurationId;
+
+  private int registerDataSerializersInProgressId;
+  private int registerDataSerializersSendInProgressId;
+  private int registerDataSerializersSendFailedId;
+  private int registerDataSerializersSendId;
+  private int registerDataSerializersSendDurationId;
+  private int registerDataSerializersTimedOutId;
+  private int registerDataSerializersFailedId;
+  private int registerDataSerializersId;
+  private int registerDataSerializersDurationId;
+
+  private int putAllInProgressId;
+  private int putAllSendInProgressId;
+  private int putAllSendFailedId;
+  private int putAllSendId;
+  private int putAllSendDurationId;
+  private int putAllTimedOutId;
+  private int putAllFailedId;
+  private int putAllId;
+  private int putAllDurationId;
+
+  private int removeAllInProgressId;
+  private int removeAllSendInProgressId;
+  private int removeAllSendFailedId;
+  private int removeAllSendId;
+  private int removeAllSendDurationId;
+  private int removeAllTimedOutId;
+  private int removeAllFailedId;
+  private int removeAllId;
+  private int removeAllDurationId;
+
+  private int getAllInProgressId;
+  private int getAllSendInProgressId;
+  private int getAllSendFailedId;
+  private int getAllSendId;
+  private int getAllSendDurationId;
+  private int getAllTimedOutId;
+  private int getAllFailedId;
+  private int getAllId;
+  private int getAllDurationId;
+
+  private int connectionsId;
+  private int connectsId;
+  private int disconnectsId;
+  private int messagesBeingReceivedId;
+  private int messageBytesBeingReceivedId;
+  private int receivedBytesId;
+  private int sentBytesId;
+
+  private int executeFunctionInProgressId;
+  private int executeFunctionSendInProgressId;
+  private int executeFunctionSendFailedId;
+  private int executeFunctionSendId;
+  private int executeFunctionSendDurationId;
+  private int executeFunctionTimedOutId;
+  private int executeFunctionFailedId;
+  private int executeFunctionId;
+  private int executeFunctionDurationId;
+
+  private int getClientPRMetadataInProgressId;
+  private int getClientPRMetadataSendInProgressId;
+  private int getClientPRMetadataSendFailedId;
+  private int getClientPRMetadataSendId;
+  private int getClientPRMetadataSendDurationId;
+  private int getClientPRMetadataTimedOutId;
+  private int getClientPRMetadataFailedId;
+  private int getClientPRMetadataId;
+  private int getClientPRMetadataDurationId;
+
+  private int getClientPartitionAttributesInProgressId;
+  private int getClientPartitionAttributesSendInProgressId;
+  private int getClientPartitionAttributesSendFailedId;
+  private int getClientPartitionAttributesSendId;
+  private int getClientPartitionAttributesSendDurationId;
+  private int getClientPartitionAttributesTimedOutId;
+  private int getClientPartitionAttributesFailedId;
+  private int getClientPartitionAttributesId;
+  private int getClientPartitionAttributesDurationId;
+
+  private int getPDXIdForTypeInProgressId;
+  private int getPDXIdForTypeSendInProgressId;
+  private int getPDXIdForTypeSendFailedId;
+  private int getPDXIdForTypeSendId;
+  private int getPDXIdForTypeSendDurationId;
+  private int getPDXIdForTypeTimedOutId;
+  private int getPDXIdForTypeFailedId;
+  private int getPDXIdForTypeId;
+  private int getPDXIdForTypeDurationId;
+
+  private int getPDXTypeByIdInProgressId;
+  private int getPDXTypeByIdSendInProgressId;
+  private int getPDXTypeByIdSendFailedId;
+  private int getPDXTypeByIdSendId;
+  private int getPDXTypeByIdSendDurationId;
+  private int getPDXTypeByIdTimedOutId;
+  private int getPDXTypeByIdFailedId;
+  private int getPDXTypeByIdId;
+  private int getPDXTypeByIdDurationId;
+
+  private int addPdxTypeInProgressId;
+  private int addPdxTypeSendInProgressId;
+  private int addPdxTypeSendFailedId;
+  private int addPdxTypeSendId;
+  private int addPdxTypeSendDurationId;
+  private int addPdxTypeTimedOutId;
+  private int addPdxTypeFailedId;
+  private int addPdxTypeId;
+  private int addPdxTypeDurationId;
+
+
+  // An array of all of the ids that represent operation statistics. This
+  // is used by the getOps method to aggregate the individual stats
+  // into a total value for all operations.
+  private static int[] opIds;
+
+  public void initializeStats(StatisticsFactory factory) {
+    try {
+      type = factory.createType("ClientStats", "Statistics about client to server communication",
+          new StatisticDescriptor[] {
+              factory.createIntGauge("getsInProgress", "Current number of gets being executed",
+                  "gets"),
+              factory.createIntCounter("gets", "Total number of gets completed successfully",
+                  "gets"),
+              factory.createIntCounter("getFailures",
+                  "Total number of get attempts that have failed",
+                  "gets"),
+              factory.createIntCounter("getTimeouts",
+                  "Total number of get attempts that have timed out",
+                  "gets"),
+              factory.createLongCounter("getTime",
+                  "Total amount of time, in nanoseconds spent doing gets", "nanoseconds"),
+              factory.createIntGauge("putsInProgress", "Current number of puts being executed",
+                  "puts"),
+              factory.createIntCounter("puts", "Total number of puts completed successfully",
+                  "puts"),
+              factory.createIntCounter("putFailures",
+                  "Total number of put attempts that have failed",
+                  "puts"),
+              factory.createIntCounter("putTimeouts",
+                  "Total number of put attempts that have timed out",
+                  "puts"),
+              factory.createLongCounter("putTime",
+                  "Total amount of time, in nanoseconds spent doing puts", "nanoseconds"),
+              factory.createIntGauge("destroysInProgress",
+                  "Current number of destroys being executed",
+                  "destroys"),
+              factory.createIntCounter("destroys",
+                  "Total number of destroys completed successfully",
+                  "destroys"),
+              factory.createIntCounter("destroyFailures",
+                  "Total number of destroy attempts that have failed", "destroys"),
+              factory.createIntCounter("destroyTimeouts",
+                  "Total number of destroy attempts that have timed out", "destroys"),
+              factory.createLongCounter("destroyTime",
+                  "Total amount of time, in nanoseconds spent doing destroys", "nanoseconds"),
+              factory.createIntGauge("destroyRegionsInProgress",
+                  "Current number of destroyRegions being executed", "destroyRegions"),
+              factory.createIntCounter("destroyRegions",
+                  "Total number of destroyRegions completed successfully", "destroyRegions"),
+              factory.createIntCounter("destroyRegionFailures",
+                  "Total number of destroyRegion attempts that have failed", "destroyRegions"),
+              factory.createIntCounter("destroyRegionTimeouts",
+                  "Total number of destroyRegion attempts that have timed out", "destroyRegions"),
+              factory.createLongCounter("destroyRegionTime",
+                  "Total amount of time, in nanoseconds spent doing destroyRegions", "nanoseconds"),
+              factory.createIntGauge("clearsInProgress", "Current number of clears being executed",
+                  "clears"),
+              factory.createIntCounter("clears", "Total number of clears completed successfully",
+                  "clears"),
+              factory.createIntCounter("clearFailures",
+                  "Total number of clear attempts that have failed",
+                  "clears"),
+              factory.createIntCounter("clearTimeouts",
+                  "Total number of clear attempts that have timed out", "clears"),
+              factory.createLongCounter("clearTime",
+                  "Total amount of time, in nanoseconds spent doing clears", "nanoseconds"),
+              factory.createIntGauge("containsKeysInProgress",
+                  "Current number of containsKeys being executed", "containsKeys"),
+              factory.createIntCounter("containsKeys",
+                  "Total number of containsKeys completed successfully", "containsKeys"),
+              factory.createIntCounter("containsKeyFailures",
+                  "Total number of containsKey attempts that have failed", "containsKeys"),
+              factory.createIntCounter("containsKeyTimeouts",
+                  "Total number of containsKey attempts that have timed out", "containsKeys"),
+              factory.createLongCounter("containsKeyTime",
+                  "Total amount of time, in nanoseconds spent doing containsKeys", "nanoseconds"),
+              factory.createIntGauge("keySetsInProgress",
+                  "Current number of keySets being executed",
+                  "keySets"),
+              factory.createIntCounter("keySets", "Total number of keySets completed successfully",
+                  "keySets"),
+              factory.createIntCounter("keySetFailures",
+                  "Total number of keySet attempts that have failed", "keySets"),
+              factory.createIntCounter("keySetTimeouts",
+                  "Total number of keySet attempts that have timed out", "keySets"),
+              factory.createLongCounter("keySetTime",
+                  "Total amount of time, in nanoseconds spent doing keySets", "nanoseconds"),
+
+              factory.createIntGauge("commitsInProgress",
+                  "Current number of commits being executed",
+                  "commits"),
+              factory.createIntCounter("commits", "Total number of commits completed successfully",
+                  "commits"),
+              factory.createIntCounter("commitFailures",
+                  "Total number of commit attempts that have failed", "commits"),
+              factory.createIntCounter("commitTimeouts",
+                  "Total number of commit attempts that have timed out", "commits"),
+              factory.createLongCounter("commitTime",
+                  "Total amount of time, in nanoseconds spent doing commits", "nanoseconds"),
+
+              factory.createIntGauge("rollbacksInProgress",
+                  "Current number of rollbacks being executed",
+                  "rollbacks"),
+              factory.createIntCounter("rollbacks",
+                  "Total number of rollbacks completed successfully",
+                  "rollbacks"),
+              factory.createIntCounter("rollbackFailures",
+                  "Total number of rollback attempts that have failed", "rollbacks"),
+              factory.createIntCounter("rollbackTimeouts",
+                  "Total number of rollback attempts that have timed out", "rollbacks"),
+              factory.createLongCounter("rollbackTime",
+                  "Total amount of time, in nanoseconds spent doing rollbacks", "nanoseconds"),
+
+              factory.createIntGauge("getEntrysInProgress",
+                  "Current number of getEntry messages being executed", "messages"),
+              factory.createIntCounter("getEntrys",
+                  "Total number of getEntry messages completed successfully", "messages"),
+              factory.createIntCounter("getEntryFailures",
+                  "Total number of getEntry attempts that have failed", "attempts"),
+              factory.createIntCounter("getEntryTimeouts",
+                  "Total number of getEntry attempts that have timed out", "attempts"),
+              factory.createLongCounter("getEntryTime",
+                  "Total amount of time, in nanoseconds spent doing getEntry processings",
+                  "nanoseconds"),
+
+              factory.createIntGauge("jtaSynchronizationsInProgress",
+                  "Current number of jtaSynchronizations being executed", "sizes"),
+              factory.createIntCounter("jtaSynchronizations",
+                  "Total number of jtaSynchronizations completed successfully",
+                  "jtaSynchronizations"),
+              factory.createIntCounter("jtaSynchronizationFailures",
+                  "Total number of jtaSynchronization attempts that have failed",
+                  "jtaSynchronizations"),
+              factory.createIntCounter("jtaSynchronizationTimeouts",
+                  "Total number of jtaSynchronization attempts that have timed out",
+                  "jtaSynchronizations"),
+              factory.createLongCounter("jtaSynchronizationTime",
+                  "Total amount of time, in nanoseconds spent doing jtaSynchronizations",
+                  "nanoseconds"),
+
+              factory.createIntGauge("txFailoversInProgress",
+                  "Current number of txFailovers being executed", "txFailovers"),
+              factory.createIntCounter("txFailovers",
+                  "Total number of txFailovers completed successfully", "txFailovers"),
+              factory.createIntCounter("txFailoverFailures",
+                  "Total number of txFailover attempts that have failed", "txFailovers"),
+              factory.createIntCounter("txFailoverTimeouts",
+                  "Total number of txFailover attempts that have timed out", "sizes"),
+              factory.createLongCounter("txFailoverTime",
+                  "Total amount of time, in nanoseconds spent doing txFailovers", "nanoseconds"),
+
+              factory.createIntGauge("sizesInProgress", "Current number of sizes being executed",
+                  "sizes"),
+              factory.createIntCounter("sizes", "Total number of sizes completed successfully",
+                  "sizes"),
+              factory.createIntCounter("sizeFailures",
+                  "Total number of size attempts that have failed",
+                  "sizes"),
+              factory.createIntCounter("sizeTimeouts",
+                  "Total number of size attempts that have timed out", "sizes"),
+              factory.createLongCounter("sizeTime",
+                  "Total amount of time, in nanoseconds spent doing sizes", "nanoseconds"),
+
+              factory.createIntGauge("invalidatesInProgress",
+                  "Current number of invalidates being executed", "invalidates"),
+              factory.createIntCounter("invalidates",
+                  "Total number of invalidates completed successfully", "invalidates"),
+              factory.createIntCounter("invalidateFailures",
+                  "Total number of invalidate attempts that have failed", "invalidates"),
+              factory.createIntCounter("invalidateTimeouts",
+                  "Total number of invalidate attempts that have timed out", "invalidates"),
+              factory.createLongCounter("invalidateTime",
+                  "Total amount of time, in nanoseconds spent doing invalidates", "nanoseconds"),
+
+              factory.createIntGauge("registerInterestsInProgress",
+                  "Current number of registerInterests being executed", "registerInterests"),
+              factory.createIntCounter("registerInterests",
+                  "Total number of registerInterests completed successfully", "registerInterests"),
+              factory.createIntCounter("registerInterestFailures",
+                  "Total number of registerInterest attempts that have failed",
+                  "registerInterests"),
+              factory.createIntCounter("registerInterestTimeouts",
+                  "Total number of registerInterest attempts that have timed out",
+                  "registerInterests"),
+              factory.createLongCounter("registerInterestTime",
+                  "Total amount of time, in nanoseconds spent doing registerInterests",
+                  "nanoseconds"),
+              factory.createIntGauge("unregisterInterestsInProgress",
+                  "Current number of unregisterInterests being executed", "unregisterInterests"),
+              factory.createIntCounter("unregisterInterests",
+                  "Total number of unregisterInterests completed successfully",
+                  "unregisterInterests"),
+              factory.createIntCounter("unregisterInterestFailures",
+                  "Total number of unregisterInterest attempts that have failed",
+                  "unregisterInterests"),
+              factory.createIntCounter("unregisterInterestTimeouts",
+                  "Total number of unregisterInterest attempts that have timed out",
+                  "unregisterInterests"),
+              factory.createLongCounter("unregisterInterestTime",
+                  "Total amount of time, in nanoseconds spent doing unregisterInterests",
+                  "nanoseconds"),
+              factory.createIntGauge("querysInProgress", "Current number of querys being executed",
+                  "querys"),
+              factory.createIntCounter("querys", "Total number of querys completed successfully",
+                  "querys"),
+              factory.createIntCounter("queryFailures",
+                  "Total number of query attempts that have failed",
+                  "querys"),
+              factory.createIntCounter("queryTimeouts",
+                  "Total number of query attempts that have timed out", "querys"),
+              factory.createLongCounter("queryTime",
+                  "Total amount of time, in nanoseconds spent doing querys", "nanoseconds"),
+              factory.createIntGauge("createCQsInProgress",
+                  "Current number of createCQs being executed",
+                  "createCQs"),
+              factory.createIntCounter("createCQs",
+                  "Total number of createCQs completed successfully",
+                  "createCQs"),
+              factory.createIntCounter("createCQFailures",
+                  "Total number of createCQ attempts that have failed", "createCQs"),
+              factory.createIntCounter("createCQTimeouts",
+                  "Total number of createCQ attempts that have timed out", "createCQs"),
+              factory.createLongCounter("createCQTime",
+                  "Total amount of time, in nanoseconds spent doing createCQs", "nanoseconds"),
+              factory.createIntGauge("stopCQsInProgress",
+                  "Current number of stopCQs being executed",
+                  "stopCQs"),
+              factory.createIntCounter("stopCQs", "Total number of stopCQs completed successfully",
+                  "stopCQs"),
+              factory.createIntCounter("stopCQFailures",
+                  "Total number of stopCQ attempts that have failed", "stopCQs"),
+              factory.createIntCounter("stopCQTimeouts",
+                  "Total number of stopCQ attempts that have timed out", "stopCQs"),
+              factory.createLongCounter("stopCQTime",
+                  "Total amount of time, in nanoseconds spent doing stopCQs", "nanoseconds"),
+              factory.createIntGauge("closeCQsInProgress",
+                  "Current number of closeCQs being executed",
+                  "closeCQs"),
+              factory.createIntCounter("closeCQs",
+                  "Total number of closeCQs completed successfully",
+                  "closeCQs"),
+              factory.createIntCounter("closeCQFailures",
+                  "Total number of closeCQ attempts that have failed", "closeCQs"),
+              factory.createIntCounter("closeCQTimeouts",
+                  "Total number of closeCQ attempts that have timed out", "closeCQs"),
+              factory.createLongCounter("closeCQTime",
+                  "Total amount of time, in nanoseconds spent doing closeCQs", "nanoseconds"),
+              factory.createIntGauge("gatewayBatchsInProgress",
+                  "Current number of gatewayBatchs being executed", "gatewayBatchs"),
+              factory.createIntCounter("gatewayBatchs",
+                  "Total number of gatewayBatchs completed successfully", "gatewayBatchs"),
+              factory.createIntCounter("gatewayBatchFailures",
+                  "Total number of gatewayBatch attempts that have failed", "gatewayBatchs"),
+              factory.createIntCounter("gatewayBatchTimeouts",
+                  "Total number of gatewayBatch attempts that have timed out", "gatewayBatchs"),
+              factory.createLongCounter("gatewayBatchTime",
+                  "Total amount of time, in nanoseconds spent doing gatewayBatchs", "nanoseconds"),
+              factory.createIntGauge("getDurableCQsInProgress",
+                  "Current number of getDurableCQs being executed", "getDurableCQs"),
+              factory.createIntCounter("getDurableCQs",
+                  "Total number of getDurableCQs completed successfully", "getDurableCQs"),
+              factory.createIntCounter("getDurableCQsFailures",
+                  "Total number of getDurableCQs attempts that have failed", "getDurableCQs"),
+              factory.createIntCounter("getDurableCQsTimeouts",
+                  "Total number of getDurableCQs attempts that have timed out", "getDurableCQs"),
+              factory.createLongCounter("getDurableCQsTime",
+                  "Total amount of time, in nanoseconds spent doing getDurableCQs", "nanoseconds"),
+              factory.createIntGauge("readyForEventsInProgress",
+                  "Current number of readyForEvents being executed", "readyForEvents"),
+              factory.createIntCounter("readyForEvents",
+                  "Total number of readyForEvents completed successfully", "readyForEvents"),
+              factory.createIntCounter("readyForEventsFailures",
+                  "Total number of readyForEvents attempts that have failed", "readyForEvents"),
+              factory.createIntCounter("readyForEventsTimeouts",
+                  "Total number of readyForEvents attempts that have timed out", "readyForEvents"),
+              factory.createLongCounter("readyForEventsTime",
+                  "Total amount of time, in nanoseconds spent doing readyForEvents", "nanoseconds"),
+              factory.createIntGauge("makePrimarysInProgress",
+                  "Current number of makePrimarys being executed", "makePrimarys"),
+              factory.createIntCounter("makePrimarys",
+                  "Total number of makePrimarys completed successfully", "makePrimarys"),
+              factory.createIntCounter("makePrimaryFailures",
+                  "Total number of makePrimary attempts that have failed", "makePrimarys"),
+              factory.createIntCounter("makePrimaryTimeouts",
+                  "Total number of makePrimary attempts that have timed out", "makePrimarys"),
+              factory.createLongCounter("makePrimaryTime",
+                  "Total amount of time, in nanoseconds spent doing makePrimarys", "nanoseconds"),
+
+              factory.createIntGauge("closeConsInProgress",
+                  "Current number of closeCons being executed",
+                  "closeCons"),
+              factory.createIntCounter("closeCons",
+                  "Total number of closeCons completed successfully",
+                  "closeCons"),
+              factory.createIntCounter("closeConFailures",
+                  "Total number of closeCon attempts that have failed", "closeCons"),
+              factory.createIntCounter("closeConTimeouts",
+                  "Total number of closeCon attempts that have timed out", "closeCons"),
+              factory.createLongCounter("closeConTime",
+                  "Total amount of time, in nanoseconds spent doing closeCons", "nanoseconds"),
+
+              factory.createIntGauge("primaryAcksInProgress",
+                  "Current number of primaryAcks being executed", "primaryAcks"),
+              factory.createIntCounter("primaryAcks",
+                  "Total number of primaryAcks completed successfully", "primaryAcks"),
+              factory.createIntCounter("primaryAckFailures",
+                  "Total number of primaryAck attempts that have failed", "primaryAcks"),
+              factory.createIntCounter("primaryAckTimeouts",
+                  "Total number of primaryAck attempts that have timed out", "primaryAcks"),
+              factory.createLongCounter("primaryAckTime",
+                  "Total amount of time, in nanoseconds spent doing primaryAcks", "nanoseconds"),
+
+              factory.createIntGauge("pingsInProgress", "Current number of pings being executed",
+                  "pings"),
+              factory.createIntCounter("pings", "Total number of pings completed successfully",
+                  "pings"),
+              factory.createIntCounter("pingFailures",
+                  "Total number of ping attempts that have failed",
+                  "pings"),
+              factory.createIntCounter("pingTimeouts",
+                  "Total number of ping attempts that have timed out", "pings"),
+              factory.createLongCounter("pingTime",
+                  "Total amount of time, in nanoseconds spent doing pings", "nanoseconds"),
+
+              factory.createIntGauge("registerInstantiatorsInProgress",
+                  "Current number of registerInstantiators being executed",
+                  "registerInstantiators"),
+              factory.createIntCounter("registerInstantiators",
+                  "Total number of registerInstantiators completed successfully",
+                  "registerInstantiators"),
+              factory.createIntCounter("registerInstantiatorsFailures",
+                  "Total number of registerInstantiators attempts that have failed",
+                  "registerInstantiators"),
+              factory.createIntCounter("registerInstantiatorsTimeouts",
+                  "Total number of registerInstantiators attempts that have timed out",
+                  "registerInstantiators"),
+              factory.createLongCounter("registerInstantiatorsTime",
+                  "Total amount of time, in nanoseconds spent doing registerInstantiators",
+                  "nanoseconds"),
+
+              factory.createIntGauge("registerDataSerializersInProgress",
+                  "Current number of registerDataSerializers being executed",
+                  "registerDataSerializers"),
+              factory.createIntCounter("registerDataSerializers",
+                  "Total number of registerDataSerializers completed successfully",
+                  "registerDataSerializers"),
+              factory.createIntCounter("registerDataSerializersFailures",
+                  "Total number of registerDataSerializers attempts that have failed",
+                  "registerDataSerializers"),
+              factory.createIntCounter("registerDataSerializersTimeouts",
+                  "Total number of registerDataSerializers attempts that have timed out",
+                  "registerDataSerializers"),
+              factory.createLongCounter("registerDataSerializersTime",
+                  "Total amount of time, in nanoseconds spent doing registerDataSerializers",
+                  "nanoseconds"),
+
+              factory.createIntGauge("connections", "Current number of connections", "connections"),
+              factory.createIntCounter("connects",
+                  "Total number of times a connection has been created.",
+                  "connects"),
+              factory.createIntCounter("disconnects",
+                  "Total number of times a connection has been destroyed.", "disconnects"),
+              factory.createIntGauge("putAllsInProgress",
+                  "Current number of putAlls being executed",
+                  "putAlls"),
+              factory.createIntCounter("putAlls", "Total number of putAlls completed successfully",
+                  "putAlls"),
+              factory.createIntCounter("putAllFailures",
+                  "Total number of putAll attempts that have failed", "putAlls"),
+              factory.createIntCounter("putAllTimeouts",
+                  "Total number of putAll attempts that have timed out", "putAlls"),
+              factory.createLongCounter("putAllTime",
+                  "Total amount of time, in nanoseconds spent doing putAlls", "nanoseconds"),
+              factory.createIntGauge("removeAllsInProgress",
+                  "Current number of removeAlls being executed", "removeAlls"),
+              factory.createIntCounter("removeAlls",
+                  "Total number of removeAlls completed successfully",
+                  "removeAlls"),
+              factory.createIntCounter("removeAllFailures",
+                  "Total number of removeAll attempts that have failed", "removeAlls"),
+              factory.createIntCounter("removeAllTimeouts",
+                  "Total number of removeAll attempts that have timed out", "removeAlls"),
+              factory.createLongCounter("removeAllTime",
+                  "Total amount of time, in nanoseconds spent doing removeAlls", "nanoseconds"),
+              factory.createIntGauge("getAllsInProgress",
+                  "Current number of getAlls being executed",
+                  "getAlls"),
+              factory.createIntCounter("getAlls", "Total number of getAlls completed successfully",
+                  "getAlls"),
+              factory.createIntCounter("getAllFailures",
+                  "Total number of getAll attempts that have failed", "getAlls"),
+              factory.createIntCounter("getAllTimeouts",
+                  "Total number of getAll attempts that have timed out", "getAlls"),
+              factory.createLongCounter("getAllTime",
+                  "Total amount of time, in nanoseconds spent doing getAlls", "nanoseconds"),
+              factory.createLongCounter("receivedBytes",
+                  "Total number of bytes received (as responses) from server over a client-to-server connection.",
+                  "bytes"),
+              factory.createLongCounter("sentBytes",
+                  "Total number of bytes sent to server over a client-to-server connection.",
+                  "bytes"),
+              factory.createIntGauge("messagesBeingReceived",
+                  "Current number of message being received off the network or being processed after reception over a client-to-server connection.",
+                  "messages"),
+              factory.createLongGauge("messageBytesBeingReceived",
+                  "Current number of bytes consumed by messages being received or processed over a client-to-server connection.",
+                  "bytes"),
+
+              factory.createIntGauge("executeFunctionsInProgress",
+                  "Current number of Functions being executed", "executeFunctions"),
+              factory.createIntCounter("executeFunctions",
+                  "Total number of Functions completed successfully", "executeFunctions"),
+              factory.createIntCounter("executeFunctionFailures",
+                  "Total number of Function attempts that have failed", "executeFunctions"),
+              factory.createIntCounter("executeFunctionTimeouts",
+                  "Total number of Function attempts that have timed out", "executeFunctions"),
+              factory.createLongCounter("executeFunctionTime",
+                  "Total amount of time, in nanoseconds spent doing Functions", "nanoseconds"),
+
+              factory.createIntGauge("asyncExecuteFunctionsInProgress",
+                  "Current number of Functions being executed asynchronously",
+                  "asyncExecuteFunctions"),
+              factory.createIntCounter("asyncExecuteFunctions",
+                  "Total number of asynchronous Functions completed successfully",
+                  "asyncExecuteFunctions"),
+              factory.createIntCounter("asyncExecuteFunctionFailures",
+                  "Total number of asynchronous Function attempts that have failed",
+                  "asyncExecuteFunctions"),
+              factory.createIntCounter("asyncExecuteFunctionTimeouts",
+                  "Total number of asynchronous Function attempts that have timed out",
+                  "asyncExecuteFunctions"),
+              factory.createLongCounter("asyncExecuteFunctionTime",
+                  "Total amount of time, in nanoseconds spent doing asynchronous Functions",
+                  "nanoseconds"),
+
+              factory.createIntGauge("getClientPRMetadataInProgress",
+                  "Current number of getClientPRMetadata operations being executed",
+                  "getClientPRMetadata"),
+              factory.createIntCounter("getClientPRMetadataFailures",
+                  "Total number of getClientPRMetadata operation attempts that have failed",
+                  "getClientPRMetadata"),
+              factory.createIntCounter("getClientPRMetadataSuccessful",
+                  "Total number of getClientPRMetadata operations completed successfully",
+                  "getClientPRMetadata"),
+              factory.createIntCounter("getClientPRMetadataTimeouts",
+                  "Total number of getClientPRMetadata operation attempts that have timed out",
+                  "getClientPRMetadata"),
+              factory.createLongCounter("getClientPRMetadataTime",
+                  "Total amount of time, in nanoseconds spent doing getClientPRMetadata successfully/unsuccessfully",
+                  "nanoseconds"),
+
+              factory.createIntGauge("getClientPartitionAttributesInProgress",
+                  "Current number of getClientPartitionAttributes operations being executed",
+                  "getClientPartitionAttributes"),
+              factory.createIntCounter("getClientPartitionAttributesFailures",
+                  "Total number of getClientPartitionAttributes operation attempts that have failed",
+                  "getClientPartitionAttributes"),
+              factory.createIntCounter("getClientPartitionAttributesSuccessful",
+                  "Total number of getClientPartitionAttributes operations completed successfully",
+                  "getClientPartitionAttributes"),
+              factory.createIntCounter("getClientPartitionAttributesTimeouts",
+                  "Total number of getClientPartitionAttributes operation attempts that have timed out",
+                  "getClientPartitionAttributes"),
+              factory.createLongCounter("getClientPartitionAttributesTime",
+                  "Total amount of time, in nanoseconds spent doing getClientPartitionAttributes successfully/unsuccessfully.",
+                  "nanoseconds"),
+
+              factory.createIntGauge("getPDXTypeByIdInProgress",
+                  "Current number of getPDXTypeById operations being executed", "getPDXTypeById"),
+              factory.createIntCounter("getPDXTypeByIdFailures",
+                  "Total number of getPDXTypeById operation attempts that have failed",
+                  "getPDXTypeById"),
+              factory.createIntCounter("getPDXTypeByIdSuccessful",
+                  "Total number of getPDXTypeById operations completed successfully",
+                  "getPDXTypeById"),
+              factory.createIntCounter("getPDXTypeByIdTimeouts",
+                  "Total number of getPDXTypeById operation attempts that have timed out",
+                  "getPDXTypeById"),
+              factory.createLongCounter("getPDXTypeByIdTime",
+                  "Total amount of time, in nanoseconds spent doing getPDXTypeById successfully/unsuccessfully.",
+                  "nanoseconds"),
+
+              factory.createIntGauge("getPDXIdForTypeInProgress",
+                  "Current number of getPDXIdForType operations being executed", "getPDXIdForType"),
+              factory.createIntCounter("getPDXIdForTypeFailures",
+                  "Total number of getPDXIdForType operation attempts that have failed",
+                  "getPDXIdForType"),
+              factory.createIntCounter("getPDXIdForTypeSuccessful",
+                  "Total number of getPDXIdForType operations completed successfully",
+                  "getPDXIdForType"),
+              factory.createIntCounter("getPDXIdForTypeTimeouts",
+                  "Total number of getPDXIdForType operation attempts that have timed out",
+                  "getPDXIdForType"),
+              factory.createLongCounter("getPDXIdForTypeTime",
+                  "Total amount of time, in nanoseconds spent doing getPDXIdForType successfully/unsuccessfully.",
+                  "nanoseconds"),
+
+              factory.createIntGauge("addPdxTypeInProgress",
+                  "Current number of addPdxType operations being executed", "addPdxType"),
+              factory.createIntCounter("addPdxTypeFailures",
+                  "Total number of addPdxType operation attempts that have failed", "addPdxType"),
+              factory.createIntCounter("addPdxTypeSuccessful",
+                  "Total number of addPdxType operations completed successfully", "addPdxType"),
+              factory.createIntCounter("addPdxTypeTimeouts",
+                  "Total number of addPdxType operation attempts that have timed out",
+                  "addPdxType"),
+              factory.createLongCounter("addPdxTypeTime",
+                  "Total amount of time, in nanoseconds spent doing addPdxType successfully/unsuccessfully.",
+                  "nanoseconds"),});
+
+      sendType =
+          factory.createType("ClientSendStats", "Statistics about client to server communication",
+              new StatisticDescriptor[] {
+                  ///////////////////////////////////////////////////////////////////////
+                  /*
+                   * factory.createIntGauge("opSendsInProgress",
+                   * "Current number of op sends being executed",
+                   * "sends"), factory.createIntCounter("opSends",
+                   * "Total number of op sends that have completed successfully", "sends"),
+                   * factory.createIntCounter("opSendFailures",
+                   * "Total number of op sends that have failed",
+                   * "sends"), factory.createLongCounter("opSendTime",
+                   * "Total amount of time, in nanoseconds spent doing op sends", "nanoseconds"),
+                   */
+                  ///////////////////////////////////////////////////////////////////////
+                  factory.createIntGauge("getSendsInProgress",
+                      "Current number of get sends being executed",
+                      "sends"),
+                  factory.createIntCounter("getSends",
+                      "Total number of get sends that have completed successfully", "sends"),
+                  factory.createIntCounter("getSendFailures",
+                      "Total number of get sends that have failed",
+                      "sends"),
+                  factory.createLongCounter("getSendTime",
+                      "Total amount of time, in nanoseconds spent doing get sends", "nanoseconds"),
+                  factory.createIntGauge("putSendsInProgress",
+                      "Current number of put sends being executed",
+                      "sends"),
+                  factory.createIntCounter("putSends",
+                      "Total number of put sends that have completed successfully", "sends"),
+                  factory.createIntCounter("putSendFailures",
+                      "Total number of put sends that have failed",
+                      "sends"),
+                  factory.createLongCounter("putSendTime",
+                      "Total amount of time, in nanoseconds spent doing put sends", "nanoseconds"),
+                  factory.createIntGauge("destroySendsInProgress",
+                      "Current number of destroy sends being executed", "sends"),
+                  factory.createIntCounter("destroySends",
+                      "Total number of destroy sends that have completed successfully", "sends"),
+                  factory.createIntCounter("destroySendFailures",
+                      "Total number of destroy sends that have failed", "sends"),
+                  factory.createLongCounter("destroySendTime",
+                      "Total amount of time, in nanoseconds spent doing destroy sends",
+                      "nanoseconds"),
+                  factory.createIntGauge("destroyRegionSendsInProgress",
+                      "Current number of destroyRegion sends being executed", "sends"),
+                  factory.createIntCounter("destroyRegionSends",
+                      "Total number of destroyRegion sends that have completed successfully",
+                      "sends"),
+                  factory.createIntCounter("destroyRegionSendFailures",
+                      "Total number of destroyRegion sends that have failed", "sends"),
+                  factory.createLongCounter("destroyRegionSendTime",
+                      "Total amount of time, in nanoseconds spent doing destroyRegion sends",
+                      "nanoseconds"),
+                  factory.createIntGauge("clearSendsInProgress",
+                      "Current number of clear sends being executed", "sends"),
+                  factory.createIntCounter("clearSends",
+                      "Total number of clear sends that have completed successfully", "sends"),
+                  factory.createIntCounter("clearSendFailures",
+                      "Total number of clear sends that have failed", "sends"),
+                  factory.createLongCounter("clearSendTime",
+                      "Total amount of time, in nanoseconds spent doing clear sends",
+                      "nanoseconds"),
+                  factory.createIntGauge("containsKeySendsInProgress",
+                      "Current number of containsKey sends being executed", "sends"),
+                  factory.createIntCounter("containsKeySends",
+                      "Total number of containsKey sends that have completed successfully",
+                      "sends"),
+                  factory.createIntCounter("containsKeySendFailures",
+                      "Total number of containsKey sends that have failed", "sends"),
+                  factory.createLongCounter("containsKeySendTime",
+                      "Total amount of time, in nanoseconds spent doing containsKey sends",
+                      "nanoseconds"),
+                  factory.createIntGauge("keySetSendsInProgress",
+                      "Current number of keySet sends being executed", "sends"),
+                  factory.createIntCounter("keySetSends",
+                      "Total number of keySet sends that have completed successfully", "sends"),
+                  factory.createIntCounter("keySetSendFailures",
+                      "Total number of keySet sends that have failed", "sends"),
+                  factory.createLongCounter("keySetSendTime",
+                      "Total amount of time, in nanoseconds spent doing keySet sends",
+                      "nanoseconds"),
+
+                  factory.createIntGauge("commitSendsInProgress",
+                      "Current number of commit sends being executed", "sends"),
+                  factory.createIntCounter("commitSendFailures",
+                      "Total number of commit sends that have failed", "sends"),
+                  factory.createIntCounter("commitSends",
+                      "Total number of commit sends that have failed",
+                      "sends"),
+                  factory.createLongCounter("commitSendTime",
+                      "Total amount of time, in nanoseconds spent doing commits", "nanoseconds"),
+                  factory.createIntGauge("rollbackSendsInProgress",
+                      "Current number of rollback sends being executed", "sends"),
+                  factory.createIntCounter("rollbackSendFailures",
+                      "Total number of rollback sends that have failed", "sends"),
+                  factory.createIntCounter("rollbackSends",
+                      "Total number of rollback sends that have failed",
+                      "sends"),
+                  factory.createLongCounter("rollbackSendTime",
+                      "Total amount of time, in nanoseconds spent doing rollbacks", "nanoseconds"),
+                  factory.createIntGauge("getEntrySendsInProgress",
+                      "Current number of getEntry sends being executed", "sends"),
+                  factory.createIntCounter("getEntrySendFailures",
+                      "Total number of getEntry sends that have failed", "sends"),
+                  factory.createIntCounter("getEntrySends",
+                      "Total number of getEntry sends that have failed",
+                      "sends"),
+                  factory.createLongCounter("getEntrySendTime",
+                      "Total amount of time, in nanoseconds spent sending getEntry messages",
+                      "nanoseconds"),
+                  factory.createIntGauge("jtaSynchronizationSendsInProgress",
+                      "Current number of jtaSynchronization sends being executed", "sends"),
+                  factory.createIntCounter("jtaSynchronizationSendFailures",
+                      "Total number of jtaSynchronization sends that have failed", "sends"),
+                  factory.createIntCounter("jtaSynchronizationSends",
+                      "Total number of jtaSynchronization sends that have failed", "sends"),
+                  factory.createLongCounter("jtaSynchronizationSendTime",
+                      "Total amount of time, in nanoseconds spent doing jtaSynchronizations",
+                      "nanoseconds"),
+                  factory.createIntGauge("txFailoverSendsInProgress",
+                      "Current number of txFailover sends being executed", "sends"),
+                  factory.createIntCounter("txFailoverSendFailures",
+                      "Total number of txFailover sends that have failed", "sends"),
+                  factory.createIntCounter("txFailoverSends",
+                      "Total number of txFailover sends that have failed", "sends"),
+                  factory.createLongCounter("txFailoverSendTime",
+                      "Total amount of time, in nanoseconds spent doing txFailovers",
+                      "nanoseconds"),
+                  factory.createIntGauge("sizeSendsInProgress",
+                      "Current number of size sends being executed",
+                      "sends"),
+                  factory.createIntCounter("sizeSendFailures",
+                      "Total number of size sends that have failed",
+                      "sends"),
+                  factory.createIntCounter("sizeSends",
+                      "Total number of size sends that have failed",
+                      "sends"),
+                  factory.createLongCounter("sizeSendTime",
+                      "Total amount of time, in nanoseconds spent doing sizes", "nanoseconds"),
+                  factory.createIntGauge("invalidateSendsInProgress",
+                      "Current number of invalidate sends being executed", "sends"),
+                  factory.createIntCounter("invalidateSendFailures",
+                      "Total number of invalidate sends that have failed", "sends"),
+                  factory.createIntCounter("invalidateSends",
+                      "Total number of invalidate sends that have failed", "sends"),
+                  factory.createLongCounter("invalidateSendTime",
+                      "Total amount of time, in nanoseconds spent doing invalidates",
+                      "nanoseconds"),
+                  factory.createIntGauge("registerInterestSendsInProgress",
+                      "Current number of registerInterest sends being executed", "sends"),
+                  factory.createIntCounter("registerInterestSends",
+                      "Total number of registerInterest sends that have completed successfully",
+                      "sends"),
+                  factory.createIntCounter("registerInterestSendFailures",
+                      "Total number of registerInterest sends that have failed", "sends"),
+                  factory.createLongCounter("registerInterestSendTime",
+                      "Total amount of time, in nanoseconds spent doing registerInterest sends",
+                      "nanoseconds"),
+                  factory.createIntGauge("unregisterInterestSendsInProgress",
+                      "Current number of unregisterInterest sends being executed", "sends"),
+                  factory.createIntCounter("unregisterInterestSends",
+                      "Total number of unregisterInterest sends that have completed successfully",
+                      "sends"),
+                  factory.createIntCounter("unregisterInterestSendFailures",
+                      "Total number of unregisterInterest sends that have failed", "sends"),
+                  factory.createLongCounter("unregisterInterestSendTime",
+                      "Total amount of time, in nanoseconds spent doing unregisterInterest sends",
+                      "nanoseconds"),
+                  factory.createIntGauge("querySendsInProgress",
+                      "Current number of query sends being executed", "sends"),
+                  factory.createIntCounter("querySends",
+                      "Total number of query sends that have completed successfully", "sends"),
+                  factory.createIntCounter("querySendFailures",
+                      "Total number of query sends that have failed", "sends"),
+                  factory.createLongCounter("querySendTime",
+                      "Total amount of time, in nanoseconds spent doing query sends",
+                      "nanoseconds"),
+                  factory.createIntGauge("createCQSendsInProgress",
+                      "Current number of createCQ sends being executed", "sends"),
+                  factory.createIntCounter("createCQSends",
+                      "Total number of createCQ sends that have completed successfully", "sends"),
+                  factory.createIntCounter("createCQSendFailures",
+                      "Total number of createCQ sends that have failed", "sends"),
+                  factory.createLongCounter("createCQSendTime",
+                      "Total amount of time, in nanoseconds spent doing createCQ sends",
+                      "nanoseconds"),
+                  factory.createIntGauge("stopCQSendsInProgress",
+                      "Current number of stopCQ sends being executed", "sends"),
+                  factory.createIntCounter("stopCQSends",
+                      "Total number of stopCQ sends that have completed successfully", "sends"),
+                  factory.createIntCounter("stopCQSendFailures",
+                      "Total number of stopCQ sends that have failed", "sends"),
+                  factory.createLongCounter("stopCQSendTime",
+                      "Total amount of time, in nanoseconds spent doing stopCQ sends",
+                      "nanoseconds"),
+                  factory.createIntGauge("closeCQSendsInProgress",
+                      "Current number of closeCQ sends being executed", "sends"),
+                  factory.createIntCounter("closeCQSends",
+                      "Total number of closeCQ sends that have completed successfully", "sends"),
+                  factory.createIntCounter("closeCQSendFailures",
+                      "Total number of closeCQ sends that have failed", "sends"),
+                  factory.createLongCounter("closeCQSendTime",
+                      "Total amount of time, in nanoseconds spent doing closeCQ sends",
+                      "nanoseconds"),
+                  factory.createIntGauge("gatewayBatchSendsInProgress",
+                      "Current number of gatewayBatch sends being executed", "sends"),
+                  factory.createIntCounter("gatewayBatchSends",
+                      "Total number of gatewayBatch sends that have completed successfully",
+                      "sends"),
+                  factory.createIntCounter("gatewayBatchSendFailures",
+                      "Total number of gatewayBatch sends that have failed", "sends"),
+                  factory.createLongCounter("gatewayBatchSendTime",
+                      "Total amount of time, in nanoseconds spent doing gatewayBatch sends",
+                      "nanoseconds"),
+                  factory.createIntGauge("getDurableCQsSendsInProgressId",
+                      "Current number of getDurableCQs sends being executed", "sends"),
+                  factory.createIntCounter("getDurableCQsSends",
+                      "Total number of getDurableCQs sends that have completed successfully",
+                      "sends"),
+                  factory.createIntCounter("getDurableCQsSendFailures",
+                      "Total number of getDurableCQs sends that have failed", "sends"),
+                  factory.createLongCounter("getDurableCQsSendTime",
+                      "Total amount of time, in nanoseconds spent doing getDurableCQs sends",
+                      "nanoseconds"),
+                  factory.createIntGauge("readyForEventsSendsInProgress",
+                      "Current number of readyForEvents sends being executed", "sends"),
+                  factory.createIntCounter("readyForEventsSends",
+                      "Total number of readyForEvents sends that have completed successfully",
+                      "sends"),
+                  factory.createIntCounter("readyForEventsSendFailures",
+                      "Total number of readyForEvents sends that have failed", "sends"),
+                  factory.createLongCounter("readyForEventsSendTime",
+                      "Total amount of time, in nanoseconds spent doing readyForEvents sends",
+                      "nanoseconds"),
+                  factory.createIntGauge("makePrimarySendsInProgress",
+                      "Current number of makePrimary sends being executed", "sends"),
+                  factory.createIntCounter("makePrimarySends",
+                      "Total number of makePrimary sends that have completed successfully",
+                      "sends"),
+                  factory.createIntCounter("makePrimarySendFailures",
+                      "Total number of makePrimary sends that have failed", "sends"),
+                  factory.createLongCounter("makePrimarySendTime",
+                      "Total amount of time, in nanoseconds spent doing makePrimary sends",
+                      "nanoseconds"),
+                  factory.createIntGauge("closeConSendsInProgress",
+                      "Current number of closeCon sends being executed", "sends"),
+                  factory.createIntCounter("closeConSends",
+                      "Total number of closeCon sends that have completed successfully", "sends"),
+                  factory.createIntCounter("closeConSendFailures",
+                      "Total number of closeCon sends that have failed", "sends"),
+                  factory.createLongCounter("closeConSendTime",
+                      "Total amount of time, in nanoseconds spent doing closeCon sends",
+                      "nanoseconds"),
+                  factory.createIntGauge("primaryAckSendsInProgress",
+                      "Current number of primaryAck sends being executed", "sends"),
+                  factory.createIntCounter("primaryAckSends",
+                      "Total number of primaryAck sends that have completed successfully", "sends"),
+                  factory.createIntCounter("primaryAckSendFailures",
+                      "Total number of primaryAck sends that have failed", "sends"),
+                  factory.createLongCounter("primaryAckSendTime",
+                      "Total amount of time, in nanoseconds spent doing primaryAck sends",
+                      "nanoseconds"),
+                  factory.createIntGauge("pingSendsInProgress",
+                      "Current number of ping sends being executed",
+                      "sends"),
+                  factory.createIntCounter("pingSends",
+                      "Total number of ping sends that have completed successfully", "sends"),
+                  factory.createIntCounter("pingSendFailures",
+                      "Total number of ping sends that have failed",
+                      "sends"),
+                  factory.createLongCounter("pingSendTime",
+                      "Total amount of time, in nanoseconds spent doing ping sends", "nanoseconds"),
+                  factory.createIntGauge("registerInstantiatorsSendsInProgress",
+                      "Current number of registerInstantiators sends being executed", "sends"),
+                  factory.createIntCounter("registerInstantiatorsSends",
+                      "Total number of registerInstantiators sends that have completed successfully",
+                      "sends"),
+                  factory.createIntCounter("registerInstantiatorsSendFailures",
+                      "Total number of registerInstantiators sends that have failed", "sends"),
+                  factory.createLongCounter("registerInstantiatorsSendTime",
+                      "Total amount of time, in nanoseconds spent doing registerInstantiators sends",
+                      "nanoseconds"),
+                  factory.createIntGauge("registerDataSerializersSendInProgress",
+                      "Current number of registerDataSerializers sends being executed", "sends"),
+                  factory.createIntCounter("registerDataSerializersSends",
+                      "Total number of registerDataSerializers sends that have completed successfully",
+                      "sends"),
+                  factory.createIntCounter("registerDataSerializersSendFailures",
+                      "Total number of registerDataSerializers sends that have failed", "sends"),
+                  factory.createLongCounter("registerDataSerializersSendTime",
+                      "Total amount of time, in nanoseconds spent doing registerDataSerializers sends",
+                      "nanoseconds"),
+                  factory.createIntGauge("putAllSendsInProgress",
+                      "Current number of putAll sends being executed", "sends"),
+                  factory.createIntCounter("putAllSends",
+                      "Total number of putAll sends that have completed successfully", "sends"),
+                  factory.createIntCounter("putAllSendFailures",
+                      "Total number of putAll sends that have failed", "sends"),
+                  factory.createLongCounter("putAllSendTime",
+                      "Total amount of time, in nanoseconds spent doing putAll sends",
+                      "nanoseconds"),
+                  factory.createIntGauge("removeAllSendsInProgress",
+                      "Current number of removeAll sends being executed", "sends"),
+                  factory.createIntCounter("removeAllSends",
+                      "Total number of removeAll sends that have completed successfully", "sends"),
+                  factory.createIntCounter("removeAllSendFailures",
+                      "Total number of removeAll sends that have failed", "sends"),
+                  factory.createLongCounter("removeAllSendTime",
+                      "Total amount of time, in nanoseconds spent doing removeAll sends",
+                      "nanoseconds"),
+                  factory.createIntGauge("getAllSendsInProgress",
+                      "Current number of getAll sends being executed", "sends"),
+                  factory.createIntCounter("getAllSends",
+                      "Total number of getAll sends that have completed successfully", "sends"),
+                  factory.createIntCounter("getAllSendFailures",
+                      "Total number of getAll sends that have failed", "sends"),
+                  factory.createLongCounter("getAllSendTime",
+                      "Total amount of time, in nanoseconds spent doing getAll sends",
+                      "nanoseconds"),
+                  factory.createIntGauge("executeFunctionSendsInProgress",
+                      "Current number of Function sends being executed", "sends"),
+                  factory.createIntCounter("executeFunctionSends",
+                      "Total number of Function sends that have completed successfully", "sends"),
+                  factory.createIntCounter("executeFunctionSendFailures",
+                      "Total number of Function sends that have failed", "sends"),
+                  factory.createLongCounter("executeFunctionSendTime",
+                      "Total amount of time, in nanoseconds spent doing Function sends",
+                      "nanoseconds"),
+                  factory.createIntGauge("asyncExecuteFunctionSendsInProgress",
+                      "Current number of Function sends being executed asynchronously", "sends"),
+                  factory.createIntCounter("asyncExecuteFunctionSends",
+                      "Total number of asynchronous Function sends that have completed successfully",
+                      "sends"),
+                  factory.createIntCounter("asyncExecuteFunctionSendFailures",
+                      "Total number of asynchronous Function sends that have failed", "sends"),
+                  factory.createLongCounter("asyncExecuteFunctionSendTime",
+                      "Total amount of time, in nanoseconds spent doing asynchronous Function sends",
+                      "nanoseconds"),
+                  factory.createIntGauge("getClientPRMetadataSendsInProgress",
+                      "Current number of getClientPRMetadata operation's request messages being send from the client to server",
+                      "sends"),
+                  factory.createIntCounter("getClientPRMetadataSendFailures",
+                      "Total number of getClientPRMetadata operation's request messages not sent successfully from the client to server",
+                      "sends"),
+                  factory.createIntCounter("getClientPRMetadataSendsSuccessful",
+                      "Total number of getClientPRMetadata operation's request messages sent successfully from the client to server",
+                      "sends"),
+                  factory.createLongCounter("getClientPRMetadataSendTime",
+                      "Total amount of time, in nanoseconds spent sending getClientPRMetadata operation's request messages successfully/unsuccessfully from the client to server",
+                      "nanoseconds"),
+                  factory.createIntGauge("getClientPartitionAttributesSendsInProgress",
+                      "Current number of getClientPartitionAttributes operation's request messages being send from the client to server",
+                      "sends"),
+                  factory.createIntCounter("getClientPartitionAttributesSendFailures",
+                      "Total number of getClientPartitionAttributes operation's request messages not sent successfully from the client to server",
+                      "sends"),
+                  factory.createIntCounter("getClientPartitionAttributesSendsSuccessful",
+                      "Total number of getClientPartitionAttributes operation's request messages sent successfully from the client to server",
+                      "sends"),
+                  factory.createLongCounter("getClientPartitionAttributesSendTime",
+                      "Total amount of time, in nanoseconds spent sending getClientPartitionAttributes operation's request messages successfully/unsuccessfully from the client to server",
+                      "nanoseconds"),
+                  factory.createIntGauge("getPDXTypeByIdSendsInProgress",
+                      "Current number of getPDXTypeById operation's request messages being send from the client to server",
+                      "sends"),
+                  factory.createIntCounter("getPDXTypeByIdSendFailures",
+                      "Total number of getPDXTypeById operation's request messages not sent successfully from the client to server",
+                      "sends"),
+                  factory.createIntCounter("getPDXTypeByIdSendsSuccessful",
+                      "Total number of getPDXTypeById operation's request messages sent successfully from the client to server",
+                      "sends"),
+                  factory.createLongCounter("getPDXTypeByIdSendTime",
+                      "Total amount of time, in nanoseconds spent sending getPDXTypeById operation's request messages successfully/unsuccessfully from the client to server",
+                      "nanoseconds"),
+                  factory.createIntGauge("getPDXIdForTypeSendsInProgress",
+                      "Current number of getPDXIdForType operation's request messages being send from the client to server",
+                      "sends"),
+                  factory.createIntCounter("getPDXIdForTypeSendFailures",
+                      "Total number of getPDXIdForType operation's request messages not sent successfully from the client to server",
+                      "sends"),
+                  factory.createIntCounter("getPDXIdForTypeSendsSuccessful",
+                      "Total number of getPDXIdForType operation's request messages sent successfully from the client to server",
+                      "sends"),
+                  factory.createLongCounter("getPDXIdForTypeSendTime",
+                      "Total amount of time, in nanoseconds spent sending getPDXIdForType operation's request messages successfully/unsuccessfully from the client to server",
+                      "nanoseconds"),
+                  factory.createIntGauge("addPdxTypeSendsInProgress",
+                      "Current number of addPdxType operation's request messages being send from the client to server",
+                      "sends"),
+                  factory.createIntCounter("addPdxTypeSendFailures",
+                      "Total number of addPdxType operation's request messages not sent successfully from the client to server",
+                      "sends"),
+                  factory.createIntCounter("addPdxTypeSendsSuccessful",
+                      "Total number of addPdxType operation's request messages sent successfully from the client to server",
+                      "sends"),
+                  factory.createLongCounter("addPdxTypeSendTime",
+                      "Total amount of time, in nanoseconds spent sending addPdxType operation's request messages successfully/unsuccessfully from the client to server",
+                      "nanoseconds"),});
+      ///////////////////////////////////////////////////////////////////////
+      /*
+       * opInProgressId = type.nameToId("opsInProgress"); opSendInProgressId =
+       * sendType.nameToId("opSendsInProgress"); opSendFailedId =
+       * sendType.nameToId("opSendFailures"); opSendId = sendType.nameToId("opSends");
+       * opSendDurationId = sendType.nameToId("opSendTime"); opTimedOutId =
+       * type.nameToId("opTimeouts"); opFailedId = type.nameToId("opFailures"); opId =
+       * type.nameToId("ops"); opDurationId = type.nameToId("opTime");
+       */
+      ///////////////////////////////////////////////////////////////////////
+      getInProgressId = type.nameToId("getsInProgress");
+      getSendInProgressId = sendType.nameToId("getSendsInProgress");
+      getSendFailedId = sendType.nameToId("getSendFailures");
+      getSendId = sendType.nameToId("getSends");
+      getSendDurationId = sendType.nameToId("getSendTime");
+      getTimedOutId = type.nameToId("getTimeouts");
+      getFailedId = type.nameToId("getFailures");
+      getId = type.nameToId("gets");
+      getDurationId = type.nameToId("getTime");
+      putInProgressId = type.nameToId("putsInProgress");
+      putSendInProgressId = sendType.nameToId("putSendsInProgress");
+      putSendFailedId = sendType.nameToId("putSendFailures");
+      putSendId = sendType.nameToId("putSends");
+      putSendDurationId = sendType.nameToId("putSendTime");
+      putTimedOutId = type.nameToId("putTimeouts");
+      putFailedId = type.nameToId("putFailures");
+      putId = type.nameToId("puts");
+      putDurationId = type.nameToId("putTime");
+      destroyInProgressId = type.nameToId("destroysInProgress");
+      destroySendInProgressId = sendType.nameToId("destroySendsInProgress");
+      destroySendFailedId = sendType.nameToId("destroySendFailures");
+      destroySendId = sendType.nameToId("destroySends");
+      destroySendDurationId = sendType.nameToId("destroySendTime");
+      destroyTimedOutId = type.nameToId("destroyTimeouts");
+      destroyFailedId = type.nameToId("destroyFailures");
+      destroyId = type.nameToId("destroys");
+      destroyDurationId = type.nameToId("destroyTime");
+      destroyRegionInProgressId = type.nameToId("destroyRegionsInProgress");
+      destroyRegionSendInProgressId = sendType.nameToId("destroyRegionSendsInProgress");
+      destroyRegionSendFailedId = sendType.nameToId("destroyRegionSendFailures");
+      destroyRegionSendId = sendType.nameToId("destroyRegionSends");
+      destroyRegionSendDurationId = sendType.nameToId("destroyRegionSendTime");
+      destroyRegionTimedOutId = type.nameToId("destroyRegionTimeouts");
+      destroyRegionFailedId = type.nameToId("destroyRegionFailures");
+      destroyRegionId = type.nameToId("destroyRegions");
+      destroyRegionDurationId = type.nameToId("destroyRegionTime");
+      clearInProgressId = type.nameToId("clearsInProgress");
+      clearSendInProgressId = sendType.nameToId("clearSendsInProgress");
+      clearSendFailedId = sendType.nameToId("clearSendFailures");
+      clearSendId = sendType.nameToId("clearSends");
+      clearSendDurationId = sendType.nameToId("clearSendTime");
+      clearTimedOutId = type.nameToId("clearTimeouts");
+      clearFailedId = type.nameToId("clearFailures");
+      clearId = type.nameToId("clears");
+      clearDurationId = type.nameToId("clearTime");
+      containsKeyInProgressId = type.nameToId("containsKeysInProgress");
+      containsKeySendInProgressId = sendType.nameToId("containsKeySendsInProgress");
+      containsKeySendFailedId = sendType.nameToId("containsKeySendFailures");
+      containsKeySendId = sendType.nameToId("containsKeySends");
+      containsKeySendDurationId = sendType.nameToId("containsKeySendTime");
+      containsKeyTimedOutId = type.nameToId("containsKeyTimeouts");
+      containsKeyFailedId = type.nameToId("containsKeyFailures");
+      containsKeyId = type.nameToId("containsKeys");
+      containsKeyDurationId = type.nameToId("containsKeyTime");
+
+      keySetInProgressId = type.nameToId("keySetsInProgress");
+      keySetSendInProgressId = sendType.nameToId("keySetSendsInProgress");
+      keySetSendFailedId = sendType.nameToId("keySetSendFailures");
+      keySetSendId = sendType.nameToId("keySetSends");
+      keySetSendDurationId = sendType.nameToId("keySetSendTime");
+      keySetTimedOutId = type.nameToId("keySetTimeouts");
+      keySetFailedId = type.nameToId("keySetFailures");
+      keySetId = type.nameToId("keySets");
+      keySetDurationId = type.nameToId("keySetTime");
+
+      commitInProgressId = type.nameToId("commitsInProgress");
+      commitSendInProgressId = sendType.nameToId("commitSendsInProgress");
+      commitSendFailedId = sendType.nameToId("commitSendFailures");
+      commitSendId = sendType.nameToId("commitSends");
+      commitSendDurationId = sendType.nameToId("commitSendTime");
+      commitDurationId = type.nameToId("commitTime");
+      commitTimedOutId = type.nameToId("commitTimeouts");
+      commitFailedId = type.nameToId("commitFailures");
+      commitId = type.nameToId("commits");
+
+      rollbackInProgressId = type.nameToId("rollbacksInProgress");
+      rollbackSendInProgressId = sendType.nameToId("rollbackSendsInProgress");
+      rollbackSendFailedId = sendType.nameToId("rollbackSendFailures");
+      rollbackSendId = sendType.nameToId("rollbackSends");
... 56977 lines suppressed ...