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/03/20 22:28:47 UTC

[geode] 04/05: Adding Micrometer to PartitionedRegionStats

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

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

commit df27f26facf173f66d1790a2069128351b43e59b
Author: Udo Kohlmeyer <uk...@pivotal.io>
AuthorDate: Thu Jan 11 17:13:51 2018 -0800

    Adding Micrometer to PartitionedRegionStats
---
 .../main/java/org/apache/geode/admin/package.html  |    2 +-
 .../internal/cache/MicroMeterRegistryFactory.kt    |    8 +
 .../cache/MicrometerPartitionRegionStats.kt        |  428 +++----
 .../apache/geode/internal/cache/MicrometerStats.kt |   19 +-
 .../internal/cache/PRHARedundancyProvider.java     |    7 +-
 .../geode/internal/cache/PartitionedRegion.java    |   31 +-
 .../internal/cache/PartitionedRegionDataStore.java |    2 +-
 .../internal/cache/PartitionedRegionStats.java     | 1264 ++------------------
 .../internal/cache/PartitionedRegionStatsImpl.java |  402 ++-----
 .../cache/TimedMicrometerPartitionedRegionStats.kt |    9 +-
 .../cache/partitioned/CreateBucketMessage.java     |    2 +-
 .../partitioned/CreateMissingBucketsTask.java      |    3 +-
 .../cache/partitioned/FetchEntryMessage.java       |    7 +-
 .../internal/cache/tx/PartitionedTXRegionStub.java |    4 +-
 .../org/apache/geode/management/MemberMXBean.java  |    4 +-
 .../internal/beans/MemberMBeanBridge.java          |    6 +-
 .../internal/cli/commands/ShowMetricsCommand.java  |   18 +-
 .../management/internal/cli/i18n/CliStrings.java   |   16 +-
 .../management/OffHeapManagementDUnitTest.java     |    2 +-
 .../bean/stats/MemberLevelStatsJUnitTest.java      |    7 +-
 .../bean/stats/RegionStatsJUnitTest.java           |    3 +-
 .../ShowMetricsCommandIntegrationTest.java         |   24 +-
 .../cli/commands/ShowMetricsDUnitTest.java         |   12 +-
 .../cli/commands/ShowMetricsJUnitTest.java         |    6 +-
 .../management/internal/security/TestCommand.java  |    2 +-
 .../cli/commands/golden-help-offline.properties    |   18 +-
 .../cli/commands/golden-help-online.properties     |    4 +-
 .../statistics/MicrometerClientStatsImpl.kt        |   38 +-
 28 files changed, 503 insertions(+), 1845 deletions(-)

diff --git a/geode-core/src/main/java/org/apache/geode/admin/package.html b/geode-core/src/main/java/org/apache/geode/admin/package.html
index 29ce994..79e4b2b 100644
--- a/geode-core/src/main/java/org/apache/geode/admin/package.html
+++ b/geode-core/src/main/java/org/apache/geode/admin/package.html
@@ -62,7 +62,7 @@ indicates that a GemFire component is unhealthy and needs immediate
 attention.</P>
 
 <P>Because each GemFire application has its own definition of what it
-means to be "healthy", the metrics that are used to determine health
+means to be "healthy", the meterRegistry that are used to determine health
 are configurable.  {@link
 org.apache.geode.admin.GemFireHealthConfig} provides methods for
 configuring how the health of {@linkplain
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/MicroMeterRegistryFactory.kt b/geode-core/src/main/java/org/apache/geode/internal/cache/MicroMeterRegistryFactory.kt
new file mode 100644
index 0000000..96dfa4e
--- /dev/null
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/MicroMeterRegistryFactory.kt
@@ -0,0 +1,8 @@
+package org.apache.geode.internal.cache
+
+import io.micrometer.core.instrument.MeterRegistry
+
+object MicroMeterRegistryFactory {
+    private val meterRegistry = MicrometerStats().meterRegistry
+    fun getMeterRegistry(): MeterRegistry = meterRegistry
+}
\ No newline at end of file
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/MicrometerPartitionRegionStats.kt b/geode-core/src/main/java/org/apache/geode/internal/cache/MicrometerPartitionRegionStats.kt
index dcb695d..0ddf647 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/MicrometerPartitionRegionStats.kt
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/MicrometerPartitionRegionStats.kt
@@ -1,15 +1,48 @@
 package org.apache.geode.internal.cache
 
+import com.netflix.spectator.impl.AtomicDouble
 import io.micrometer.core.instrument.Counter
 import io.micrometer.core.instrument.Gauge
+import io.micrometer.core.instrument.MeterRegistry
 import io.micrometer.core.instrument.Tag
+import org.apache.geode.Statistics
+import java.lang.Number
 import java.util.concurrent.atomic.AtomicInteger
 
-open class MicrometerPartitionRegionStats(val regionName: String) : MicrometerStats() {
+open class MicrometerPartitionRegionStats(val meterRegistry: MeterRegistry, val regionName: String) : PartitionedRegionStats {
+    override fun getStats(): Statistics? {
+        //we do nothing here... because we don't need to
+        return null;
+    }
 
     @Suppress("PropertyName")
     protected val PARTITIONED_REGION = "PartitionedRegion"
+    private val tags = listOf<Tag>(Tag.of("region", regionName), Tag.of("regionType", PARTITIONED_REGION))
+
+    private fun constructCounterForMetric(metricName: String): Counter {
+        return meterRegistry.counter("${metricName}", tags)
+    }
+
+    private fun <T> constructGaugeForMetric(metricName: String, atomic: T, function: (T) -> Double): Gauge = Gauge.builder(metricName, atomic, function).tags(tags).register(meterRegistry)
+
+
+    private fun incrementAtomic(atomic: AtomicInteger, value: Int) {
+        atomic.addAndGet(value)
+    }
+
+    private fun incrementAtomic(atomic: AtomicDouble, value: Double) {
+        atomic.addAndGet(value)
+    }
+
+    //Atomic values to track
+    private val bucketCountAtomic = AtomicInteger(0)
+    private val lowBucketCountAtomic = AtomicInteger(0)
+    private val numberCopiesBucketCountAtomic = AtomicInteger(0)
+    private val totalNumberOfBucketsAtomic = AtomicInteger(0)
+    private val primaryBucketCountAtomic = AtomicInteger(0)
+    private val numberVolunteeringThreadsAtomic = AtomicInteger(0)
 
+    //Micrometer Meters
     private val putCounter = constructCounterForMetric("put")
     private val putAllCounter = constructCounterForMetric("putAll")
     private val createCounter = constructCounterForMetric("create")
@@ -21,360 +54,243 @@ open class MicrometerPartitionRegionStats(val regionName: String) : MicrometerSt
     private val containValueForKeyCounter = constructCounterForMetric("containValueForKey")
     private val containsKeyValueRetriesCounter = constructCounterForMetric("containsKeyValueRetries")
     private val containsKeyValueOpsRetriedCounter = constructCounterForMetric("containsKeyValueOpsRetried")
-    private val incInvalidateRetriesCounter = constructCounterForMetric("incInvalidateRetries")
-    private val incInvalidateOpsRetriedCounter = constructCounterForMetric("incInvalidateOpsRetried")
-    private val incDestroyRetriesCounter = constructCounterForMetric("incDestroyRetries")
-    private val incDestroyOpsRetriedCounter = constructCounterForMetric("incDestroyOpsRetried")
-    private val incPutRetriesCounter = constructCounterForMetric("incPutRetries")
-    private val incPutOpsRetriedCounter = constructCounterForMetric("incPutOpsRetried")
-    private val incGetOpsRetriedCounter = constructCounterForMetric("incGetOpsRetried")
-    private val incGetRetriesCounter = constructCounterForMetric("incGetRetries")
-    private val incCreateOpsRetriedCounter = constructCounterForMetric("incCreateOpsRetried")
-    private val incCreateRetriesCounter = constructCounterForMetric("incCreateRetries")
-    private val incPreferredReadLocalCounter = constructCounterForMetric("incPreferredReadLocal")
-    private val incPreferredReadRemoteCounter = constructCounterForMetric("incPreferredReadRemote")
-    private val incPutAllRetriesCounter = constructCounterForMetric("incPutAllRetries")
-    private val incPutAllMsgsRetriedCounter = constructCounterForMetric("incPutAllMsgsRetried")
-    private val incRemoveAllRetriesCounter = constructCounterForMetric("incRemoveAllRetries")
-    private val incRemoveAllMsgsRetriedCounter = constructCounterForMetric("incRemoveAllMsgsRetried")
-    private val incPartitionMessagesSentCounter = constructCounterForMetric("incPartitionMessagesSent")
-    private val incBucketCountCounter = constructCounterForMetric("incBucketCount")
-
-    private fun constructCounterForMetric(metricName: String): Counter =
-            metrics.counter("${metricName}Counter", regionName, PARTITIONED_REGION)
-
-    private fun constructAtomicIntegerToMonitor(metricName: String): AtomicInteger =
-            metrics.gauge("${metricName}Gauge",listOf(regionName,PARTITIONED_REGION), AtomicInteger(0),AtomicInteger::get)
-
-    open fun endPut(startTimeInNanos: Long) = putCounter.increment()
-    open fun endPutAll(startTimeInNanos: Long) = putAllCounter.increment()
-    open fun endCreate(startTimeInNanos: Long) = createCounter.increment()
-    open fun endRemoveAll(startTimeInNanos: Long) = removeAllCounter.increment()
-    open fun endGet(startTimeInNanos: Long) = getCounter.increment()
-    open fun endDestroy(startTimeInNanos: Long) = destroyCounter.increment()
-    open fun endInvalidate(startTimeInNanos: Long) = invalidateCounter.increment()
-    open fun endContainsKey(startTimeInNanos: Long) = containsKeyCounter.increment()
-    open fun endContainsValueForKey(startTimeInNanos: Long) = containValueForKeyCounter.increment()
-    fun incContainsKeyValueRetries() = containsKeyValueRetriesCounter.increment()
-    fun incContainsKeyValueOpsRetried() = containsKeyValueOpsRetriedCounter.increment()
-    fun incInvalidateRetries() = incInvalidateRetriesCounter.increment()
-    fun incInvalidateOpsRetried() = incInvalidateOpsRetriedCounter.increment()
-    fun incDestroyRetries() = incDestroyRetriesCounter.increment()
-    fun incDestroyOpsRetried() = incDestroyOpsRetriedCounter.increment()
-    fun incPutRetries() = incPutRetriesCounter.increment()
-    fun incPutOpsRetried() = incPutOpsRetriedCounter.increment()
-    fun incGetOpsRetried() = incGetOpsRetriedCounter.increment()
-    fun incGetRetries() = incGetRetriesCounter.increment()
-    fun incCreateOpsRetried() = incCreateOpsRetriedCounter.increment()
-    fun incCreateRetries() = incCreateRetriesCounter.increment()
-    fun incPreferredReadLocal() = incPreferredReadLocalCounter.increment()
-    fun incPreferredReadRemote() = incPreferredReadRemoteCounter.increment()
-    fun incPutAllRetries() = incPutAllRetriesCounter.increment()
-    fun incPutAllMsgsRetried() = incPutAllMsgsRetriedCounter.increment()
-    fun incRemoveAllRetries() = incRemoveAllRetriesCounter.increment()
-    fun incRemoveAllMsgsRetried() = incRemoveAllMsgsRetriedCounter.increment()
-    fun incPartitionMessagesSent() = incPartitionMessagesSentCounter.increment()
-    fun incBucketCount(bucketCount: Int) = incBucketCountGauge.increment(bucketCount.toDouble())
-    fun incLowRedundancyBucketCount(`val`: Int) {
-        TODO("not implemented") //To change body of created functions use File | Settings | File Templates.
-    }
+    private val invalidateRetriesCounter = constructCounterForMetric("invalidateRetries")
+    private val invalidateOpsRetriedCounter = constructCounterForMetric("invalidateOpsRetried")
+    private val destroyRetriesCounter = constructCounterForMetric("destroyRetries")
+    private val destroyOpsRetriedCounter = constructCounterForMetric("destroyOpsRetried")
+    private val putRetriesCounter = constructCounterForMetric("putRetries")
+    private val putOpsRetriedCounter = constructCounterForMetric("putOpsRetried")
+    private val getOpsRetriedCounter = constructCounterForMetric("getOpsRetried")
+    private val getRetriesCounter = constructCounterForMetric("getRetries")
+    private val createOpsRetriedCounter = constructCounterForMetric("createOpsRetried")
+    private val createRetriesCounter = constructCounterForMetric("createRetries")
+    private val preferredReadLocalCounter = constructCounterForMetric("preferredReadLocal")
+    private val preferredReadRemoteCounter = constructCounterForMetric("preferredReadRemote")
+    private val putAllRetriesCounter = constructCounterForMetric("putAllRetries")
+    private val putAllMsgsRetriedCounter = constructCounterForMetric("putAllMsgsRetried")
+    private val removeAllRetriesCounter = constructCounterForMetric("removeAllRetries")
+    private val removeAllMsgsRetriedCounter = constructCounterForMetric("removeAllMsgsRetried")
+    private val partitionMessagesSentCounter = constructCounterForMetric("partitionMessagesSent")
+    private val prMetaDataSentCounter = constructCounterForMetric("prMetaDataSentCounter")
+    private val bucketCountGauge = constructGaugeForMetric("bucketCount", bucketCountAtomic, { it.get().toDouble() })
+    private val lowBucketCountGauge = constructGaugeForMetric("lowBucketCount", lowBucketCountAtomic, { it.get().toDouble() })
+    private val numberCopiesBucketCountGauge = constructGaugeForMetric("numberCopiesBucketCount", numberCopiesBucketCountAtomic, { it.get().toDouble() })
+    private val totalNumberOfBucketsGauge = constructGaugeForMetric("totalNumberOfBuckets", totalNumberOfBucketsAtomic, { it.get().toDouble() })
+    private val primaryBucketCountGauge = constructGaugeForMetric("primaryBucketCount", primaryBucketCountAtomic, { it.get().toDouble() })
+    private val numberVolunteeringThreadsGauge = constructGaugeForMetric("numberVolunteeringThreads", numberVolunteeringThreadsAtomic, { it.get().toDouble() })
 
-    fun incNoCopiesBucketCount(`val`: Int) {
-        TODO("not implemented") //To change body of created functions use File | Settings | File Templates.
+    override fun close() {
+        //Noop
     }
 
-    fun incTotalNumBuckets(`val`: Int) {
-        TODO("not implemented") //To change body of created functions use File | Settings | File Templates.
-    }
+    override fun endPut(startTimeInNanos: Long) = putCounter.increment()
+    override fun endPutAll(startTimeInNanos: Long) = putAllCounter.increment()
+    override fun endCreate(startTimeInNanos: Long) = createCounter.increment()
+    override fun endRemoveAll(startTimeInNanos: Long) = removeAllCounter.increment()
+    override fun endGet(startTimeInNanos: Long) = getCounter.increment()
+    override fun endDestroy(startTimeInNanos: Long) = destroyCounter.increment()
+    override fun endInvalidate(startTimeInNanos: Long) = invalidateCounter.increment()
+    override fun endContainsKey(startTimeInNanos: Long) = containsKeyCounter.increment()
+    override fun endContainsValueForKey(startTimeInNanos: Long) = containValueForKeyCounter.increment()
+    override fun incContainsKeyValueRetries() = containsKeyValueRetriesCounter.increment()
+    override fun incContainsKeyValueOpsRetried() = containsKeyValueOpsRetriedCounter.increment()
+    override fun incInvalidateRetries() = invalidateRetriesCounter.increment()
+    override fun incInvalidateOpsRetried() = invalidateOpsRetriedCounter.increment()
+    override fun incDestroyRetries() = destroyRetriesCounter.increment()
+    override fun incDestroyOpsRetried() = destroyOpsRetriedCounter.increment()
+    override fun incPutRetries() = putRetriesCounter.increment()
+    override fun incPutOpsRetried() = putOpsRetriedCounter.increment()
+    override fun incGetOpsRetried() = getOpsRetriedCounter.increment()
+    override fun incGetRetries() = getRetriesCounter.increment()
+    override fun incCreateOpsRetried() = createOpsRetriedCounter.increment()
+    override fun incCreateRetries() = createRetriesCounter.increment()
+    override fun incPreferredReadLocal() = preferredReadLocalCounter.increment()
+    override fun incPreferredReadRemote() = preferredReadRemoteCounter.increment()
+    override fun incPutAllRetries() = putAllRetriesCounter.increment()
+    override fun incPutAllMsgsRetried() = putAllMsgsRetriedCounter.increment()
+    override fun incRemoveAllRetries() = removeAllRetriesCounter.increment()
+    override fun incRemoveAllMsgsRetried() = removeAllMsgsRetriedCounter.increment()
+    override fun incPartitionMessagesSent() = partitionMessagesSentCounter.increment()
+    override fun incBucketCount(bucketCount: Int) = incrementAtomic(bucketCountAtomic,bucketCount)
+    override fun incLowRedundancyBucketCount(lowBucketCount: Int) = incrementAtomic(lowBucketCountAtomic,lowBucketCount)
+    override fun incNoCopiesBucketCount(numberCopiesBucketCount: Int) = incrementAtomic(numberCopiesBucketCountAtomic, numberCopiesBucketCount)
+    override fun incTotalNumBuckets(totalNumberOfBuckets: Int) = incrementAtomic(totalNumberOfBucketsAtomic, totalNumberOfBuckets)
+    override fun incPrimaryBucketCount(primaryBucketCount: Int) = incrementAtomic(primaryBucketCountAtomic, primaryBucketCount)
+    override fun incVolunteeringThreads(numberVolunteeringThreads: Int) = incrementAtomic(numberVolunteeringThreadsAtomic, numberVolunteeringThreads)
+    override fun incPRMetaDataSentCount() = prMetaDataSentCounter.increment()
 
-    fun incPrimaryBucketCount(`val`: Int) {
-        TODO("not implemented") //To change body of created functions use File | Settings | File Templates.
-    }
+    override fun incDataStoreEntryCount(amt: Int) {
 
-    fun incVolunteeringThreads(`val`: Int) {
-        TODO("not implemented") //To change body of created functions use File | Settings | File Templates.
     }
 
-    fun incPRMetaDataSentCount() {
-        TODO("not implemented") //To change body of created functions use File | Settings | File Templates.
+    override fun incBytesInUse(delta: Long) {
+        0
     }
 
-    fun incDataStoreEntryCount(amt: Int) {
-        TODO("not implemented") //To change body of created functions use File | Settings | File Templates.
-    }
+    override fun getVolunteeringInProgress(): Int = 0
 
-    fun incBytesInUse(delta: Long) {
-        TODO("not implemented") //To change body of created functions use File | Settings | File Templates.
-    }
+    override fun startPartitionMessageProcessing(): Long = 0
 
-    fun getVolunteeringInProgress(): Int {
-        TODO("not implemented") //To change body of created functions use File | Settings | File Templates.
-    }
+    override fun endPartitionMessagesProcessing(start: Long) {
 
-    fun startPartitionMessageProcessing(): Long {
-        TODO("not implemented") //To change body of created functions use File | Settings | File Templates.
     }
 
-    fun endPartitionMessagesProcessing(start: Long) {
-        TODO("not implemented") //To change body of created functions use File | Settings | File Templates.
-    }
+    override fun setBucketCount(i: Int) {
 
-    fun setBucketCount(i: Int) {
-        TODO("not implemented") //To change body of created functions use File | Settings | File Templates.
     }
 
 
-    fun getDataStoreEntryCount(): Int {
-        TODO("not implemented") //To change body of created functions use File | Settings | File Templates.
-    }
+    override fun getDataStoreEntryCount(): Int = 0
 
 
-    fun getDataStoreBytesInUse(): Long {
-        TODO("not implemented") //To change body of created functions use File | Settings | File Templates.
-    }
+    override fun getDataStoreBytesInUse(): Long = 0
 
-    fun getTotalBucketCount(): Int {
-        TODO("not implemented") //To change body of created functions use File | Settings | File Templates.
-    }
+    override fun getTotalBucketCount(): Int = 0
 
-    fun getVolunteeringBecamePrimary(): Int {
-        TODO("not implemented") //To change body of created functions use File | Settings | File Templates.
-    }
+    override fun getVolunteeringBecamePrimary(): Int = 0
 
-    fun getVolunteeringBecamePrimaryTime(): Long {
-        TODO("not implemented") //To change body of created functions use File | Settings | File Templates.
-    }
+    override fun getVolunteeringBecamePrimaryTime(): Long = 0
 
-    fun getVolunteeringOtherPrimary(): Int {
-        TODO("not implemented") //To change body of created functions use File | Settings | File Templates.
-    }
+    override fun getVolunteeringOtherPrimary(): Int = 0
 
-    fun getVolunteeringOtherPrimaryTime(): Long {
-        TODO("not implemented") //To change body of created functions use File | Settings | File Templates.
-    }
+    override fun getVolunteeringOtherPrimaryTime(): Long = 0
 
-    fun getVolunteeringClosed(): Int {
-        TODO("not implemented") //To change body of created functions use File | Settings | File Templates.
-    }
+    override fun getVolunteeringClosed(): Int = 0
 
-    fun getVolunteeringClosedTime(): Long {
-        TODO("not implemented") //To change body of created functions use File | Settings | File Templates.
-    }
+    override fun getVolunteeringClosedTime(): Long = 0
 
-    fun startVolunteering(): Long {
-        TODO("not implemented") //To change body of created functions use File | Settings | File Templates.
-    }
+    override fun startVolunteering(): Long = 0
 
-    fun endVolunteeringBecamePrimary(start: Long) {
-        TODO("not implemented") //To change body of created functions use File | Settings | File Templates.
-    }
+    override fun endVolunteeringBecamePrimary(start: Long) {
 
-    fun endVolunteeringOtherPrimary(start: Long) {
-        TODO("not implemented") //To change body of created functions use File | Settings | File Templates.
     }
 
-    fun endVolunteeringClosed(start: Long) {
-        TODO("not implemented") //To change body of created functions use File | Settings | File Templates.
-    }
+    override fun endVolunteeringOtherPrimary(start: Long) {
 
-    fun getTotalNumBuckets(): Int {
-        TODO("not implemented") //To change body of created functions use File | Settings | File Templates.
     }
 
+    override fun endVolunteeringClosed(start: Long) {
 
-    fun getPrimaryBucketCount(): Int {
-        TODO("not implemented") //To change body of created functions use File | Settings | File Templates.
     }
 
+    override fun getTotalNumBuckets(): Int = 0
 
-    fun getVolunteeringThreads(): Int {
-        TODO("not implemented") //To change body of created functions use File | Settings | File Templates.
-    }
 
+    override fun getPrimaryBucketCount(): Int = 0
 
-    fun getLowRedundancyBucketCount(): Int {
-        TODO("not implemented") //To change body of created functions use File | Settings | File Templates.
-    }
 
-    fun getNoCopiesBucketCount(): Int {
-        TODO("not implemented") //To change body of created functions use File | Settings | File Templates.
-    }
+    override fun getVolunteeringThreads(): Int = 0
 
 
-    fun getConfiguredRedundantCopies(): Int {
-        TODO("not implemented") //To change body of created functions use File | Settings | File Templates.
-    }
+    override fun getLowRedundancyBucketCount(): Int = 0
 
-    fun setConfiguredRedundantCopies(`val`: Int) {
-        TODO("not implemented") //To change body of created functions use File | Settings | File Templates.
-    }
+    override fun getNoCopiesBucketCount(): Int = 0
 
-    fun setLocalMaxMemory(l: Long) {
-        TODO("not implemented") //To change body of created functions use File | Settings | File Templates.
-    }
 
-    fun getActualRedundantCopies(): Int {
-        TODO("not implemented") //To change body of created functions use File | Settings | File Templates.
-    }
+    override fun getConfiguredRedundantCopies(): Int = 0
 
-    fun setActualRedundantCopies(`val`: Int) {
-        TODO("not implemented") //To change body of created functions use File | Settings | File Templates.
-    }
+    override fun setConfiguredRedundantCopies(value: Int) {
 
-    fun putStartTime(key: Any?, startTime: Long) {
-        TODO("not implemented") //To change body of created functions use File | Settings | File Templates.
     }
 
-    fun removeStartTime(key: Any?): Long {
-        TODO("not implemented") //To change body of created functions use File | Settings | File Templates.
-    }
+    override fun setLocalMaxMemory(l: Long) {
 
-    fun endGetEntry(startTime: Long) {
-        TODO("not implemented") //To change body of created functions use File | Settings | File Templates.
     }
 
-    fun endGetEntry(start: Long, numInc: Int) {
-        TODO("not implemented") //To change body of created functions use File | Settings | File Templates.
-    }
+    override fun getActualRedundantCopies(): Int = 0
 
-    fun startRecovery(): Long {
-        TODO("not implemented") //To change body of created functions use File | Settings | File Templates.
-    }
+    override fun setActualRedundantCopies(value: Int) {
 
-    fun endRecovery(start: Long) {
-        TODO("not implemented") //To change body of created functions use File | Settings | File Templates.
     }
 
-    fun startBucketCreate(isRebalance: Boolean): Long {
-        TODO("not implemented") //To change body of created functions use File | Settings | File Templates.
-    }
+    override fun putStartTime(key: Any?, startTime: Long) {
 
-    fun endBucketCreate(start: Long, success: Boolean, isRebalance: Boolean) {
-        TODO("not implemented") //To change body of created functions use File | Settings | File Templates.
     }
 
-    fun startPrimaryTransfer(isRebalance: Boolean): Long {
-        TODO("not implemented") //To change body of created functions use File | Settings | File Templates.
-    }
+    override fun removeStartTime(key: Any?): Long = 0
 
-    fun endPrimaryTransfer(start: Long, success: Boolean, isRebalance: Boolean) {
-        TODO("not implemented") //To change body of created functions use File | Settings | File Templates.
-    }
+    override fun endGetEntry(startTime: Long) {
 
-    fun getBucketCreatesInProgress(): Int {
-        TODO("not implemented") //To change body of created functions use File | Settings | File Templates.
     }
 
-    fun getBucketCreatesCompleted(): Int {
-        TODO("not implemented") //To change body of created functions use File | Settings | File Templates.
-    }
+    override fun endGetEntry(start: Long, numInc: Int) {
 
-    fun getBucketCreatesFailed(): Int {
-        TODO("not implemented") //To change body of created functions use File | Settings | File Templates.
     }
 
-    fun getBucketCreateTime(): Long {
-        TODO("not implemented") //To change body of created functions use File | Settings | File Templates.
-    }
+    override fun startRecovery(): Long = 0
 
-    fun getPrimaryTransfersInProgress(): Int {
-        TODO("not implemented") //To change body of created functions use File | Settings | File Templates.
-    }
+    override fun endRecovery(start: Long) {
 
-    fun getPrimaryTransfersCompleted(): Int {
-        TODO("not implemented") //To change body of created functions use File | Settings | File Templates.
     }
 
-    fun getPrimaryTransfersFailed(): Int {
-        TODO("not implemented") //To change body of created functions use File | Settings | File Templates.
-    }
+    override fun startBucketCreate(isRebalance: Boolean): Long = 0
 
-    fun getPrimaryTransferTime(): Long {
-        TODO("not implemented") //To change body of created functions use File | Settings | File Templates.
-    }
+    override fun endBucketCreate(start: Long, success: Boolean, isRebalance: Boolean) {
 
-    fun startRebalanceBucketCreate() {
-        TODO("not implemented") //To change body of created functions use File | Settings | File Templates.
     }
 
-    fun endRebalanceBucketCreate(start: Long, end: Long, success: Boolean) {
-        TODO("not implemented") //To change body of created functions use File | Settings | File Templates.
-    }
+    override fun startPrimaryTransfer(isRebalance: Boolean): Long = 0
 
-    fun startRebalancePrimaryTransfer() {
-        TODO("not implemented") //To change body of created functions use File | Settings | File Templates.
-    }
+    override fun endPrimaryTransfer(start: Long, success: Boolean, isRebalance: Boolean) {
 
-    fun endRebalancePrimaryTransfer(start: Long, end: Long, success: Boolean) {
-        TODO("not implemented") //To change body of created functions use File | Settings | File Templates.
     }
 
-    fun getRebalanceBucketCreatesInProgress(): Int {
-        TODO("not implemented") //To change body of created functions use File | Settings | File Templates.
-    }
+    override fun getBucketCreatesInProgress(): Int = 0
 
-    fun getRebalanceBucketCreatesCompleted(): Int {
-        TODO("not implemented") //To change body of created functions use File | Settings | File Templates.
-    }
+    override fun getBucketCreatesCompleted(): Int = 0
 
-    fun getRebalanceBucketCreatesFailed(): Int {
-        TODO("not implemented") //To change body of created functions use File | Settings | File Templates.
-    }
+    override fun getBucketCreatesFailed(): Int = 0
 
-    fun getRebalanceBucketCreateTime(): Long {
-        TODO("not implemented") //To change body of created functions use File | Settings | File Templates.
-    }
+    override fun getBucketCreateTime(): Long = 0
 
-    fun getRebalancePrimaryTransfersInProgress(): Int {
-        TODO("not implemented") //To change body of created functions use File | Settings | File Templates.
-    }
+    override fun getPrimaryTransfersInProgress(): Int = 0
 
-    fun getRebalancePrimaryTransfersCompleted(): Int {
-        TODO("not implemented") //To change body of created functions use File | Settings | File Templates.
-    }
+    override fun getPrimaryTransfersCompleted(): Int = 0
 
-    fun getRebalancePrimaryTransfersFailed(): Int {
-        TODO("not implemented") //To change body of created functions use File | Settings | File Templates.
-    }
+    override fun getPrimaryTransfersFailed(): Int = 0
 
-    fun getRebalancePrimaryTransferTime(): Long {
-        TODO("not implemented") //To change body of created functions use File | Settings | File Templates.
-    }
+    override fun getPrimaryTransferTime(): Long = 0
 
-    fun startApplyReplication(): Long {
-        TODO("not implemented") //To change body of created functions use File | Settings | File Templates.
-    }
+    override fun getRebalanceBucketCreatesInProgress(): Int = 0
 
-    fun endApplyReplication(start: Long) {
-        TODO("not implemented") //To change body of created functions use File | Settings | File Templates.
-    }
+    override fun getRebalanceBucketCreatesCompleted(): Int = 0
 
-    fun startSendReplication(): Long {
-        TODO("not implemented") //To change body of created functions use File | Settings | File Templates.
-    }
+    override fun getRebalanceBucketCreatesFailed(): Int = 0
 
-    fun endSendReplication(start: Long) {
-        TODO("not implemented") //To change body of created functions use File | Settings | File Templates.
-    }
+    override fun getRebalanceBucketCreateTime(): Long = 0
 
-    fun startPutRemote(): Long {
-        TODO("not implemented") //To change body of created functions use File | Settings | File Templates.
-    }
+    override fun getRebalancePrimaryTransfersInProgress(): Int = 0
+
+    override fun getRebalancePrimaryTransfersCompleted(): Int = 0
+
+    override fun getRebalancePrimaryTransfersFailed(): Int = 0
+
+    override fun getRebalancePrimaryTransferTime(): Long = 0
+
+    override fun startApplyReplication(): Long = 0
+
+    override fun endApplyReplication(start: Long) {
 
-    fun endPutRemote(start: Long) {
-        TODO("not implemented") //To change body of created functions use File | Settings | File Templates.
     }
 
-    fun startPutLocal(): Long {
-        TODO("not implemented") //To change body of created functions use File | Settings | File Templates.
+    override fun startSendReplication(): Long = 0
+
+    override fun endSendReplication(start: Long) {
+
     }
 
-    fun endPutLocal(start: Long) {
-        TODO("not implemented") //To change body of created functions use File | Settings | File Templates.
+    override fun startPutRemote(): Long = 0
+
+    override fun endPutRemote(start: Long) {
+
     }
 
+    override fun startPutLocal(): Long = 0
+
+    override fun endPutLocal(start: Long) {
 
-    fun getPRMetaDataSentCount(): Long {
-        TODO("not implemented") //To change body of created functions use File | Settings | File Templates.
     }
+
+
+    override fun getPRMetaDataSentCount(): Long = 0
 }
\ No newline at end of file
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/MicrometerStats.kt b/geode-core/src/main/java/org/apache/geode/internal/cache/MicrometerStats.kt
index b083052..c44a0ab 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/MicrometerStats.kt
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/MicrometerStats.kt
@@ -8,20 +8,27 @@ import io.micrometer.influx.InfluxMeterRegistry
 import io.micrometer.jmx.JmxMeterRegistry
 import java.time.Duration
 
-abstract class MicrometerStats {
-    protected val metrics = CompositeMeterRegistry(Clock.SYSTEM)
+class MicrometerStats {
+    val meterRegistry = CompositeMeterRegistry(Clock.SYSTEM)
     private val influxMetrics: MeterRegistry = InfluxMeterRegistry(object : InfluxConfig {
-        override fun step(): Duration = Duration.ofSeconds(1)
+        override fun step(): Duration = Duration.ofSeconds(10)
         override fun db(): String = "mydb"
         override fun get(k: String): String? = null
         override fun uri(): String = "http://localhost:8086"
     }, Clock.SYSTEM)
 
+//    private val atlasMetrics: MeterRegistry = AtlasMeterRegistry(object : AtlasConfig {
+//        override fun get(k: String?): String? = null
+//        override fun enabled(): Boolean = true
+//        override fun uri(): String = "http://localhost:7101/api/v1/publish"
+//        override fun step(): Duration = Duration.ofSeconds(10)
+//    }, Clock.SYSTEM)
+
     private val jmxMetrics: MeterRegistry = JmxMeterRegistry()
 
     init {
-        metrics.add(influxMetrics)
-//        metrics.add(atlasMetrics)
-        metrics.add(jmxMetrics)
+        meterRegistry.add(influxMetrics)
+//        meterRegistry.add(atlasMetrics)
+        meterRegistry.add(jmxMetrics)
     }
 }
\ No newline at end of file
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/PRHARedundancyProvider.java b/geode-core/src/main/java/org/apache/geode/internal/cache/PRHARedundancyProvider.java
index 7bbed43..9626fa1 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/PRHARedundancyProvider.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/PRHARedundancyProvider.java
@@ -446,7 +446,7 @@ public class PRHARedundancyProvider {
   // public static final boolean ENFORCE_UNIQUE_HOST_STORAGE_ALLOCATION =
   // DistributionConfig.DEFAULT_ENFORCE_UNIQUE_HOST;
 
-  public InternalDistributedMember createBucketOnDataStore(int bucketId, int size, long startTime,
+  public InternalDistributedMember createBucketOnDataStore(int bucketId, int size,
       RetryTimeKeeper snoozer) {
     Set<InternalDistributedMember> attempted = new HashSet<InternalDistributedMember>();
     InternalDistributedMember ret;
@@ -523,7 +523,7 @@ public class PRHARedundancyProvider {
    *         region referred to in the query.
    */
   public InternalDistributedMember createBucketAtomically(final int bucketId,
-      final int newBucketSize, final long startTime, final boolean finishIncompleteCreation,
+      final int newBucketSize,  final boolean finishIncompleteCreation,
       String partitionName) throws PartitionedRegionStorageException, PartitionedRegionException,
       PartitionOfflineException {
     final boolean isDebugEnabled = logger.isDebugEnabled();
@@ -1040,13 +1040,12 @@ public class PRHARedundancyProvider {
 
   public void finishIncompleteBucketCreation(int bucketId) {
     String partitionName = null;
-    final long startTime = PartitionedRegionStats.startTime();
     if (this.prRegion.isFixedPartitionedRegion()) {
       FixedPartitionAttributesImpl fpa =
           PartitionedRegionHelper.getFixedPartitionAttributesForBucket(this.prRegion, bucketId);
       partitionName = fpa.getPartitionName();
     }
-    createBucketAtomically(bucketId, 0, startTime, true, partitionName);
+    createBucketAtomically(bucketId, 0, true, partitionName);
   }
 
   /**
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/PartitionedRegion.java b/geode-core/src/main/java/org/apache/geode/internal/cache/PartitionedRegion.java
index 4287f0e..9ce448c 100755
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/PartitionedRegion.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/PartitionedRegion.java
@@ -526,7 +526,7 @@ public class PartitionedRegion extends LocalRegion
 
   @Override
   public boolean remove(Object key, Object value, Object callbackArg) {
-    final long startTime = PartitionedRegionStats.startTime();
+    final long startTime = prStats.startTime();
     try {
       return super.remove(key, value, callbackArg);
     } finally {
@@ -715,6 +715,7 @@ public class PartitionedRegion extends LocalRegion
 
   private final PartitionedRegionRedundancyTracker redundancyTracker;
 
+
   /**
    * Constructor for a PartitionedRegion. This has an accessor (Region API) functionality and
    * contains a datastore for actual storage. An accessor can act as a local cache by having a local
@@ -726,7 +727,8 @@ public class PartitionedRegion extends LocalRegion
     super(regionName, regionAttributes, parentRegion, cache, internalRegionArgs);
 
     this.node = initializeNode();
-    this.prStats = new PartitionedRegionStats(cache.getDistributedSystem(), getFullPath());
+    this.prStats = new TimedMicrometerPartitionedRegionStats(getFullPath());
+//    this.prStats = new PartitionedRegionStatsImpl(cache.getDistributedSystem(), getFullPath());
     this.regionIdentifier = getFullPath().replace('/', '#');
 
     if (logger.isDebugEnabled()) {
@@ -1553,7 +1555,7 @@ public class PartitionedRegion extends LocalRegion
   @Override
   protected Region.Entry<?, ?> nonTXGetEntry(KeyInfo keyInfo, boolean access,
       boolean allowTombstones) {
-    final long startTime = PartitionedRegionStats.startTime();
+    final long startTime = prStats.startTime();
     final Object key = keyInfo.getKey();
     try {
       int bucketId = keyInfo.getBucketId();
@@ -1986,7 +1988,7 @@ public class PartitionedRegion extends LocalRegion
   boolean virtualPut(EntryEventImpl event, boolean ifNew, boolean ifOld, Object expectedOldValue,
       boolean requireOldValue, long lastModified, boolean overwriteDestroyed)
       throws TimeoutException, CacheWriterException {
-    final long startTime = PartitionedRegionStats.startTime();
+    final long startTime = prStats.startTime();
     boolean result = false;
     final DistributedPutAllOperation putAllOp_save = event.setPutAllOperation(null);
 
@@ -2159,7 +2161,7 @@ public class PartitionedRegion extends LocalRegion
       throw cache.getCacheClosedException("Cache is shutting down");
     }
 
-    final long startTime = PartitionedRegionStats.startTime();
+    final long startTime = prStats.startTime();
     // build all the msgs by bucketid
     HashMap prMsgMap = putAllOp.createPRMessages();
     PutAllPartialResult partialKeys = new PutAllPartialResult(putAllOp.putAllDataSize);
@@ -2251,7 +2253,7 @@ public class PartitionedRegion extends LocalRegion
       throw cache.getCacheClosedException("Cache is shutting down");
     }
 
-    final long startTime = PartitionedRegionStats.startTime();
+    final long startTime = prStats.startTime();
     // build all the msgs by bucketid
     HashMap<Integer, RemoveAllPRMessage> prMsgMap = op.createPRMessages();
     PutAllPartialResult partialKeys = new PutAllPartialResult(op.removeAllDataSize);
@@ -3262,12 +3264,11 @@ public class PartitionedRegion extends LocalRegion
     }
     // Potentially no storage assigned, start bucket creation, be careful of race
     // conditions
-    final long startTime = PartitionedRegionStats.startTime();
     if (isDataStore()) {
-      ret = this.redundancyProvider.createBucketAtomically(bucketId, size, startTime, false,
+      ret = this.redundancyProvider.createBucketAtomically(bucketId, size, false,
           partitionName);
     } else {
-      ret = this.redundancyProvider.createBucketOnDataStore(bucketId, size, startTime, snoozer);
+      ret = this.redundancyProvider.createBucketOnDataStore(bucketId, size, snoozer);
     }
     return ret;
   }
@@ -3280,7 +3281,7 @@ public class PartitionedRegion extends LocalRegion
     Object obj = null;
     final Object key = keyInfo.getKey();
     final Object aCallbackArgument = keyInfo.getCallbackArg();
-    final long startTime = PartitionedRegionStats.startTime();
+    final long startTime = prStats.startTime();
     try {
       int bucketId = keyInfo.getBucketId();
       if (bucketId == KeyInfo.UNKNOWN_BUCKET) {
@@ -5081,7 +5082,7 @@ public class PartitionedRegion extends LocalRegion
       final Object expectedOldValue)
       throws TimeoutException, EntryNotFoundException, CacheWriterException {
 
-    final long startTime = PartitionedRegionStats.startTime();
+    final long startTime = prStats.startTime();
     try {
       if (event.getEventId() == null) {
         event.setNewEventId(this.cache.getDistributedSystem());
@@ -5570,8 +5571,8 @@ public class PartitionedRegion extends LocalRegion
   }
 
   @Override
-  public void basicInvalidate(EntryEventImpl event) throws EntryNotFoundException {
-    final long startTime = PartitionedRegionStats.startTime();
+  void basicInvalidate(EntryEventImpl event) throws EntryNotFoundException {
+    final long startTime = prStats.startTime();
     try {
       if (event.getEventId() == null) {
         event.setNewEventId(this.cache.getDistributedSystem());
@@ -6217,7 +6218,7 @@ public class PartitionedRegion extends LocalRegion
 
   @Override
   protected boolean nonTXContainsKey(KeyInfo keyInfo) {
-    final long startTime = PartitionedRegionStats.startTime();
+    final long startTime = prStats.startTime();
     boolean contains = false;
     try {
       int bucketId = keyInfo.getBucketId();
@@ -6391,7 +6392,7 @@ public class PartitionedRegion extends LocalRegion
     // checkClosed();
     checkReadiness();
     validateKey(key);
-    final long startTime = PartitionedRegionStats.startTime();
+    final long startTime = prStats.startTime();
     boolean containsValueForKey = false;
     try {
       containsValueForKey = getDataView().containsValueForKey(getKeyInfo(key), this);
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/PartitionedRegionDataStore.java b/geode-core/src/main/java/org/apache/geode/internal/cache/PartitionedRegionDataStore.java
index ef8eb99..a1865f2 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/PartitionedRegionDataStore.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/PartitionedRegionDataStore.java
@@ -2672,7 +2672,7 @@ public class PartitionedRegionDataStore implements HasCachePerfStats {
    * @param bucket the Region containing the bucket data
    */
   public void dumpBucket(int bucketId, final LocalRegion bucket) {
-    Integer buckId = Integer.valueOf(bucketId);
+    Integer buckId = bucketId;
     visitBucket(buckId, bucket, new EntryVisitor() {
       final StringBuffer buf = new StringBuffer("Entries in bucket ").append(bucket).append("\n");
 
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/PartitionedRegionStats.java b/geode-core/src/main/java/org/apache/geode/internal/cache/PartitionedRegionStats.java
index 08434c9..03717ed 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/PartitionedRegionStats.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/PartitionedRegionStats.java
@@ -15,17 +15,7 @@
 
 package org.apache.geode.internal.cache;
 
-import java.util.Collections;
-import java.util.Map;
-import java.util.concurrent.ConcurrentHashMap;
-
-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.Region;
-import org.apache.geode.internal.statistics.StatisticsTypeFactoryImpl;
 
 /**
  * Represents a statistics type that can be archived to vsd. Loading of this class automatically
@@ -42,1157 +32,113 @@ import org.apache.geode.internal.statistics.StatisticsTypeFactoryImpl;
  *
  * @since GemFire 5.0
  */
-public class PartitionedRegionStats {
-
-  private static final StatisticsType type;
-
-  private static final int dataStoreEntryCountId;
-  private static final int dataStoreBytesInUseId;
-  private static final int bucketCountId;
-
-  private static final int putsCompletedId;
-  private static final int putOpsRetriedId;
-  private static final int putRetriesId;
-
-  private static final int createsCompletedId;
-  private static final int createOpsRetriedId;
-  private static final int createRetriesId;
-
-  private static final int preferredReadLocalId;
-  private static final int preferredReadRemoteId;
-
-  private static final int getsCompletedId;
-  private static final int getOpsRetriedId;
-  private static final int getRetriesId;
-
-  private static final int destroysCompletedId;
-  private static final int destroyOpsRetriedId;
-  private static final int destroyRetriesId;
-
-  private static final int invalidatesCompletedId;
-  private static final int invalidateOpsRetriedId;
-  private static final int invalidateRetriesId;
-
-  private static final int containsKeyCompletedId;
-  private static final int containsKeyOpsRetriedId;
-  private static final int containsKeyRetriesId;
-
-  private static final int containsValueForKeyCompletedId;
-
-  private static final int partitionMessagesSentId;
-  private static final int partitionMessagesReceivedId;
-  private static final int partitionMessagesProcessedId;
-
-  private static final int putTimeId;
-  private static final int createTimeId;
-  private static final int getTimeId;
-  private static final int destroyTimeId;
-  private static final int invalidateTimeId;
-  private static final int containsKeyTimeId;
-  private static final int containsValueForKeyTimeId;
-  private static final int partitionMessagesProcessingTimeId;
-
-  private static final String PUTALLS_COMPLETED = "putAllsCompleted";
-  private static final String PUTALL_MSGS_RETRIED = "putAllMsgsRetried";
-  private static final String PUTALL_RETRIES = "putAllRetries";
-  private static final String PUTALL_TIME = "putAllTime";
-
-  private static final int fieldId_PUTALLS_COMPLETED;
-  private static final int fieldId_PUTALL_MSGS_RETRIED;
-  private static final int fieldId_PUTALL_RETRIES;
-  private static final int fieldId_PUTALL_TIME;
-
-  private static final String REMOVE_ALLS_COMPLETED = "removeAllsCompleted";
-  private static final String REMOVE_ALL_MSGS_RETRIED = "removeAllMsgsRetried";
-  private static final String REMOVE_ALL_RETRIES = "removeAllRetries";
-  private static final String REMOVE_ALL_TIME = "removeAllTime";
-
-  private static final int fieldId_REMOVE_ALLS_COMPLETED;
-  private static final int fieldId_REMOVE_ALL_MSGS_RETRIED;
-  private static final int fieldId_REMOVE_ALL_RETRIES;
-  private static final int fieldId_REMOVE_ALL_TIME;
-
-  private static final int volunteeringInProgressId; // count of volunteering in progress
-  private static final int volunteeringBecamePrimaryId; // ended as primary
-  private static final int volunteeringBecamePrimaryTimeId; // time spent that ended as primary
-  private static final int volunteeringOtherPrimaryId; // ended as not primary
-  private static final int volunteeringOtherPrimaryTimeId; // time spent that ended as not primary
-  private static final int volunteeringClosedId; // ended as closed
-  private static final int volunteeringClosedTimeId; // time spent that ended as closed
-
-  private static final int applyReplicationCompletedId;
-  private static final int applyReplicationInProgressId;
-  private static final int applyReplicationTimeId;
-  private static final int sendReplicationCompletedId;
-  private static final int sendReplicationInProgressId;
-  private static final int sendReplicationTimeId;
-  private static final int putRemoteCompletedId;
-  private static final int putRemoteInProgressId;
-  private static final int putRemoteTimeId;
-  private static final int putLocalCompletedId;
-  private static final int putLocalInProgressId;
-  private static final int putLocalTimeId;
-
-  private static final int totalNumBucketsId; // total number of buckets
-  private static final int primaryBucketCountId; // number of hosted primary buckets
-  private static final int volunteeringThreadsId; // number of threads actively volunteering
-  private static final int lowRedundancyBucketCountId; // number of buckets currently without full
-                                                       // redundancy
-  private static final int noCopiesBucketCountId; // number of buckets currently without any
-                                                  // redundancy
-
-  private static final int configuredRedundantCopiesId;
-  private static final int actualRedundantCopiesId;
-
-  private static final int getEntriesCompletedId;
-  private static final int getEntryTimeId;
-
-  private static final int recoveriesInProgressId;
-  private static final int recoveriesCompletedId;
-  private static final int recoveriesTimeId;
-  private static final int bucketCreatesInProgressId;
-  private static final int bucketCreatesCompletedId;
-  private static final int bucketCreatesFailedId;
-  private static final int bucketCreateTimeId;
-
-  private static final int rebalanceBucketCreatesInProgressId;
-  private static final int rebalanceBucketCreatesCompletedId;
-  private static final int rebalanceBucketCreatesFailedId;
-  private static final int rebalanceBucketCreateTimeId;
-
-  private static final int primaryTransfersInProgressId;
-  private static final int primaryTransfersCompletedId;
-  private static final int primaryTransfersFailedId;
-  private static final int primaryTransferTimeId;
-
-  private static final int rebalancePrimaryTransfersInProgressId;
-  private static final int rebalancePrimaryTransfersCompletedId;
-  private static final int rebalancePrimaryTransfersFailedId;
-  private static final int rebalancePrimaryTransferTimeId;
-
-  private static final int prMetaDataSentCountId;
-
-  private static final int localMaxMemoryId;
-
-  static {
-    final boolean largerIsBetter = true;
-    StatisticsTypeFactory f = StatisticsTypeFactoryImpl.singleton();
-    type = f.createType("PartitionedRegionStats",
-        "Statistics for operations and connections in the Partitioned Region",
-        new StatisticDescriptor[] {
-
-            f.createIntGauge("bucketCount", "Number of buckets in this node.", "buckets"),
-            f.createIntCounter("putsCompleted", "Number of puts completed.", "operations",
-                largerIsBetter),
-            f.createIntCounter("putOpsRetried",
-                "Number of put operations which had to be retried due to failures.", "operations",
-                false),
-            f.createIntCounter("putRetries",
-                "Total number of times put operations had to be retried.", "retry attempts", false),
-            f.createIntCounter("createsCompleted", "Number of creates completed.", "operations",
-                largerIsBetter),
-            f.createIntCounter("createOpsRetried",
-                "Number of create operations which had to be retried due to failures.",
-                "operations", false),
-            f.createIntCounter("createRetries",
-                "Total number of times put operations had to be retried.", "retry attempts", false),
-            f.createIntCounter("preferredReadLocal", "Number of reads satisfied from local store",
-                "operations", largerIsBetter),
-            f.createIntCounter(PUTALLS_COMPLETED, "Number of putAlls completed.", "operations",
-                largerIsBetter),
-            f.createIntCounter(PUTALL_MSGS_RETRIED,
-                "Number of putAll messages which had to be retried due to failures.", "operations",
-                false),
-            f.createIntCounter(PUTALL_RETRIES,
-                "Total number of times putAll messages had to be retried.", "retry attempts",
-                false),
-            f.createLongCounter(PUTALL_TIME, "Total time spent doing putAlls.", "nanoseconds",
-                !largerIsBetter),
-            f.createIntCounter(REMOVE_ALLS_COMPLETED, "Number of removeAlls completed.",
-                "operations", largerIsBetter),
-            f.createIntCounter(REMOVE_ALL_MSGS_RETRIED,
-                "Number of removeAll messages which had to be retried due to failures.",
-                "operations", false),
-            f.createIntCounter(REMOVE_ALL_RETRIES,
-                "Total number of times removeAll messages had to be retried.", "retry attempts",
-                false),
-            f.createLongCounter(REMOVE_ALL_TIME, "Total time spent doing removeAlls.",
-                "nanoseconds", !largerIsBetter),
-            f.createIntCounter("preferredReadRemote", "Number of reads satisfied from remote store",
-                "operations", false),
-            f.createIntCounter("getsCompleted", "Number of gets completed.", "operations",
-                largerIsBetter),
-            f.createIntCounter("getOpsRetried",
-                "Number of get operations which had to be retried due to failures.", "operations",
-                false),
-            f.createIntCounter("getRetries",
-                "Total number of times get operations had to be retried.", "retry attempts", false),
-            f.createIntCounter("destroysCompleted", "Number of destroys completed.", "operations",
-                largerIsBetter),
-            f.createIntCounter("destroyOpsRetried",
-                "Number of destroy operations which had to be retried due to failures.",
-                "operations", false),
-            f.createIntCounter("destroyRetries",
-                "Total number of times destroy operations had to be retried.", "retry attempts",
-                false),
-            f.createIntCounter("invalidatesCompleted", "Number of invalidates completed.",
-                "operations", largerIsBetter),
-
-            f.createIntCounter("invalidateOpsRetried",
-                "Number of invalidate operations which had to be retried due to failures.",
-                "operations", false),
-            f.createIntCounter("invalidateRetries",
-                "Total number of times invalidate operations had to be retried.", "retry attempts",
-                false),
-            f.createIntCounter("containsKeyCompleted", "Number of containsKeys completed.",
-                "operations", largerIsBetter),
-
-            f.createIntCounter("containsKeyOpsRetried",
-                "Number of containsKey or containsValueForKey operations which had to be retried due to failures.",
-                "operations", false),
-            f.createIntCounter("containsKeyRetries",
-                "Total number of times containsKey or containsValueForKey operations had to be retried.",
-                "operations", false),
-            f.createIntCounter("containsValueForKeyCompleted",
-                "Number of containsValueForKeys completed.", "operations", largerIsBetter),
-            f.createIntCounter("PartitionMessagesSent", "Number of PartitionMessages Sent.",
-                "operations", largerIsBetter),
-            f.createIntCounter("PartitionMessagesReceived", "Number of PartitionMessages Received.",
-                "operations", largerIsBetter),
-            f.createIntCounter("PartitionMessagesProcessed",
-                "Number of PartitionMessages Processed.", "operations", largerIsBetter),
-            f.createLongCounter("putTime", "Total time spent doing puts.", "nanoseconds", false),
-            f.createLongCounter("createTime", "Total time spent doing create operations.",
-                "nanoseconds", false),
-            f.createLongCounter("getTime", "Total time spent performing get operations.",
-                "nanoseconds", false),
-            f.createLongCounter("destroyTime", "Total time spent doing destroys.", "nanoseconds",
-                false),
-            f.createLongCounter("invalidateTime", "Total time spent doing invalidates.",
-                "nanoseconds", false),
-            f.createLongCounter("containsKeyTime",
-                "Total time spent performing containsKey operations.", "nanoseconds", false),
-            f.createLongCounter("containsValueForKeyTime",
-                "Total time spent performing containsValueForKey operations.", "nanoseconds",
-                false),
-            f.createLongCounter("partitionMessagesProcessingTime",
-                "Total time spent on PartitionMessages processing.", "nanoseconds", false),
-            f.createIntGauge("dataStoreEntryCount",
-                "The number of entries stored in this Cache for the named Partitioned Region. This does not include entries which are tombstones. See CachePerfStats.tombstoneCount.",
-                "entries"),
-            f.createLongGauge("dataStoreBytesInUse",
-                "The current number of bytes stored in this Cache for the named Partitioned Region",
-                "bytes"),
-            f.createIntGauge("volunteeringInProgress",
-                "Current number of attempts to volunteer for primary of a bucket.", "operations"),
-            f.createIntCounter("volunteeringBecamePrimary",
-                "Total number of attempts to volunteer that ended when this member became primary.",
-                "operations"),
-            f.createLongCounter("volunteeringBecamePrimaryTime",
-                "Total time spent volunteering that ended when this member became primary.",
-                "nanoseconds", false),
-            f.createIntCounter("volunteeringOtherPrimary",
-                "Total number of attempts to volunteer that ended when this member discovered other primary.",
-                "operations"),
-            f.createLongCounter("volunteeringOtherPrimaryTime",
-                "Total time spent volunteering that ended when this member discovered other primary.",
-                "nanoseconds", false),
-            f.createIntCounter("volunteeringClosed",
-                "Total number of attempts to volunteer that ended when this member's bucket closed.",
-                "operations"),
-            f.createLongCounter("volunteeringClosedTime",
-                "Total time spent volunteering that ended when this member's bucket closed.",
-                "nanoseconds", false),
-            f.createIntGauge("totalNumBuckets", "The total number of buckets.", "buckets"),
-            f.createIntGauge("primaryBucketCount",
-                "Current number of primary buckets hosted locally.", "buckets"),
-            f.createIntGauge("volunteeringThreads",
-                "Current number of threads volunteering for primary.", "threads"),
-            f.createIntGauge("lowRedundancyBucketCount",
-                "Current number of buckets without full redundancy.", "buckets"),
-            f.createIntGauge("noCopiesBucketCount",
-                "Current number of buckets without any copies remaining.", "buckets"),
-            f.createIntGauge("configuredRedundantCopies",
-                "Configured number of redundant copies for this partitioned region.", "copies"),
-            f.createIntGauge("actualRedundantCopies",
-                "Actual number of redundant copies for this partitioned region.", "copies"),
-            f.createIntCounter("getEntryCompleted", "Number of getEntry operations completed.",
-                "operations", largerIsBetter),
-            f.createLongCounter("getEntryTime", "Total time spent performing getEntry operations.",
-                "nanoseconds", false),
-
-            f.createIntGauge("recoveriesInProgress",
-                "Current number of redundancy recovery operations in progress for this region.",
-                "operations"),
-            f.createIntCounter("recoveriesCompleted",
-                "Total number of redundancy recovery operations performed on this region.",
-                "operations"),
-            f.createLongCounter("recoveryTime", "Total number time spent recovering redundancy.",
-                "operations"),
-            f.createIntGauge("bucketCreatesInProgress",
-                "Current number of bucket create operations being performed for rebalancing.",
-                "operations"),
-            f.createIntCounter("bucketCreatesCompleted",
-                "Total number of bucket create operations performed for rebalancing.",
-                "operations"),
-            f.createIntCounter("bucketCreatesFailed",
-                "Total number of bucket create operations performed for rebalancing that failed.",
-                "operations"),
-            f.createLongCounter("bucketCreateTime",
-                "Total time spent performing bucket create operations for rebalancing.",
-                "nanoseconds", false),
-            f.createIntGauge("primaryTransfersInProgress",
-                "Current number of primary transfer operations being performed for rebalancing.",
-                "operations"),
-            f.createIntCounter("primaryTransfersCompleted",
-                "Total number of primary transfer operations performed for rebalancing.",
-                "operations"),
-            f.createIntCounter("primaryTransfersFailed",
-                "Total number of primary transfer operations performed for rebalancing that failed.",
-                "operations"),
-            f.createLongCounter("primaryTransferTime",
-                "Total time spent performing primary transfer operations for rebalancing.",
-                "nanoseconds", false),
-
-            f.createIntCounter("applyReplicationCompleted",
-                "Total number of replicated values sent from a primary to this redundant data store.",
-                "operations", largerIsBetter),
-            f.createIntGauge("applyReplicationInProgress",
-                "Current number of replication operations in progress on this redundant data store.",
-                "operations", !largerIsBetter),
-            f.createLongCounter("applyReplicationTime",
-                "Total time spent storing replicated values on this redundant data store.",
-                "nanoseconds", !largerIsBetter),
-            f.createIntCounter("sendReplicationCompleted",
-                "Total number of replicated values sent from this primary to a redundant data store.",
-                "operations", largerIsBetter),
-            f.createIntGauge("sendReplicationInProgress",
-                "Current number of replication operations in progress from this primary.",
-                "operations", !largerIsBetter),
-            f.createLongCounter("sendReplicationTime",
-                "Total time spent replicating values from this primary to a redundant data store.",
-                "nanoseconds", !largerIsBetter),
-            f.createIntCounter("putRemoteCompleted",
-                "Total number of completed puts that did not originate in the primary. These puts require an extra network hop to the primary.",
-                "operations", largerIsBetter),
-            f.createIntGauge("putRemoteInProgress",
-                "Current number of puts in progress that did not originate in the primary.",
-                "operations", !largerIsBetter),
-            f.createLongCounter("putRemoteTime",
-                "Total time spent doing puts that did not originate in the primary.", "nanoseconds",
-                !largerIsBetter),
-            f.createIntCounter("putLocalCompleted",
-                "Total number of completed puts that did originate in the primary. These puts are optimal.",
-                "operations", largerIsBetter),
-            f.createIntGauge("putLocalInProgress",
-                "Current number of puts in progress that did originate in the primary.",
-                "operations", !largerIsBetter),
-            f.createLongCounter("putLocalTime",
-                "Total time spent doing puts that did originate in the primary.", "nanoseconds",
-                !largerIsBetter),
-
-            f.createIntGauge("rebalanceBucketCreatesInProgress",
-                "Current number of bucket create operations being performed for rebalancing.",
-                "operations"),
-            f.createIntCounter("rebalanceBucketCreatesCompleted",
-                "Total number of bucket create operations performed for rebalancing.",
-                "operations"),
-            f.createIntCounter("rebalanceBucketCreatesFailed",
-                "Total number of bucket create operations performed for rebalancing that failed.",
-                "operations"),
-            f.createLongCounter("rebalanceBucketCreateTime",
-                "Total time spent performing bucket create operations for rebalancing.",
-                "nanoseconds", false),
-            f.createIntGauge("rebalancePrimaryTransfersInProgress",
-                "Current number of primary transfer operations being performed for rebalancing.",
-                "operations"),
-            f.createIntCounter("rebalancePrimaryTransfersCompleted",
-                "Total number of primary transfer operations performed for rebalancing.",
-                "operations"),
-            f.createIntCounter("rebalancePrimaryTransfersFailed",
-                "Total number of primary transfer operations performed for rebalancing that failed.",
-                "operations"),
-            f.createLongCounter("rebalancePrimaryTransferTime",
-                "Total time spent performing primary transfer operations for rebalancing.",
-                "nanoseconds", false),
-            f.createLongCounter("prMetaDataSentCount",
-                "total number of times meta data refreshed sent on client's request.", "operation",
-                false),
-
-            f.createLongGauge("localMaxMemory",
-                "local max memory in bytes for this region on this member", "bytes")
-
-        });
-
-    bucketCountId = type.nameToId("bucketCount");
-
-    putsCompletedId = type.nameToId("putsCompleted");
-    putOpsRetriedId = type.nameToId("putOpsRetried");
-    putRetriesId = type.nameToId("putRetries");
-    createsCompletedId = type.nameToId("createsCompleted");
-    createOpsRetriedId = type.nameToId("createOpsRetried");
-    createRetriesId = type.nameToId("createRetries");
-    getsCompletedId = type.nameToId("getsCompleted");
-    preferredReadLocalId = type.nameToId("preferredReadLocal");
-    preferredReadRemoteId = type.nameToId("preferredReadRemote");
-    getOpsRetriedId = type.nameToId("getOpsRetried");
-    getRetriesId = type.nameToId("getRetries");
-    destroysCompletedId = type.nameToId("destroysCompleted");
-    destroyOpsRetriedId = type.nameToId("destroyOpsRetried");
-    destroyRetriesId = type.nameToId("destroyRetries");
-    invalidatesCompletedId = type.nameToId("invalidatesCompleted");
-    invalidateOpsRetriedId = type.nameToId("invalidateOpsRetried");
-    invalidateRetriesId = type.nameToId("invalidateRetries");
-    containsKeyCompletedId = type.nameToId("containsKeyCompleted");
-    containsKeyOpsRetriedId = type.nameToId("containsKeyOpsRetried");
-    containsKeyRetriesId = type.nameToId("containsKeyRetries");
-    containsValueForKeyCompletedId = type.nameToId("containsValueForKeyCompleted");
-    partitionMessagesSentId = type.nameToId("PartitionMessagesSent");
-    partitionMessagesReceivedId = type.nameToId("PartitionMessagesReceived");
-    partitionMessagesProcessedId = type.nameToId("PartitionMessagesProcessed");
-    fieldId_PUTALLS_COMPLETED = type.nameToId(PUTALLS_COMPLETED);
-    fieldId_PUTALL_MSGS_RETRIED = type.nameToId(PUTALL_MSGS_RETRIED);
-    fieldId_PUTALL_RETRIES = type.nameToId(PUTALL_RETRIES);
-    fieldId_PUTALL_TIME = type.nameToId(PUTALL_TIME);
-    fieldId_REMOVE_ALLS_COMPLETED = type.nameToId(REMOVE_ALLS_COMPLETED);
-    fieldId_REMOVE_ALL_MSGS_RETRIED = type.nameToId(REMOVE_ALL_MSGS_RETRIED);
-    fieldId_REMOVE_ALL_RETRIES = type.nameToId(REMOVE_ALL_RETRIES);
-    fieldId_REMOVE_ALL_TIME = type.nameToId(REMOVE_ALL_TIME);
-    putTimeId = type.nameToId("putTime");
-    createTimeId = type.nameToId("createTime");
-    getTimeId = type.nameToId("getTime");
-    destroyTimeId = type.nameToId("destroyTime");
-    invalidateTimeId = type.nameToId("invalidateTime");
-    containsKeyTimeId = type.nameToId("containsKeyTime");
-    containsValueForKeyTimeId = type.nameToId("containsValueForKeyTime");
-    partitionMessagesProcessingTimeId = type.nameToId("partitionMessagesProcessingTime");
-    dataStoreEntryCountId = type.nameToId("dataStoreEntryCount");
-    dataStoreBytesInUseId = type.nameToId("dataStoreBytesInUse");
-
-    volunteeringInProgressId = type.nameToId("volunteeringInProgress");
-    volunteeringBecamePrimaryId = type.nameToId("volunteeringBecamePrimary");
-    volunteeringBecamePrimaryTimeId = type.nameToId("volunteeringBecamePrimaryTime");
-    volunteeringOtherPrimaryId = type.nameToId("volunteeringOtherPrimary");
-    volunteeringOtherPrimaryTimeId = type.nameToId("volunteeringOtherPrimaryTime");
-    volunteeringClosedId = type.nameToId("volunteeringClosed");
-    volunteeringClosedTimeId = type.nameToId("volunteeringClosedTime");
-
-    totalNumBucketsId = type.nameToId("totalNumBuckets");
-    primaryBucketCountId = type.nameToId("primaryBucketCount");
-    volunteeringThreadsId = type.nameToId("volunteeringThreads");
-    lowRedundancyBucketCountId = type.nameToId("lowRedundancyBucketCount");
-    noCopiesBucketCountId = type.nameToId("noCopiesBucketCount");
-
-    getEntriesCompletedId = type.nameToId("getEntryCompleted");
-    getEntryTimeId = type.nameToId("getEntryTime");
-
-    configuredRedundantCopiesId = type.nameToId("configuredRedundantCopies");
-    actualRedundantCopiesId = type.nameToId("actualRedundantCopies");
-
-    recoveriesCompletedId = type.nameToId("recoveriesCompleted");
-    recoveriesInProgressId = type.nameToId("recoveriesInProgress");
-    recoveriesTimeId = type.nameToId("recoveryTime");
-    bucketCreatesInProgressId = type.nameToId("bucketCreatesInProgress");
-    bucketCreatesCompletedId = type.nameToId("bucketCreatesCompleted");
-    bucketCreatesFailedId = type.nameToId("bucketCreatesFailed");
-    bucketCreateTimeId = type.nameToId("bucketCreateTime");
-    primaryTransfersInProgressId = type.nameToId("primaryTransfersInProgress");
-    primaryTransfersCompletedId = type.nameToId("primaryTransfersCompleted");
-    primaryTransfersFailedId = type.nameToId("primaryTransfersFailed");
-    primaryTransferTimeId = type.nameToId("primaryTransferTime");
-
-    rebalanceBucketCreatesInProgressId = type.nameToId("rebalanceBucketCreatesInProgress");
-    rebalanceBucketCreatesCompletedId = type.nameToId("rebalanceBucketCreatesCompleted");
-    rebalanceBucketCreatesFailedId = type.nameToId("rebalanceBucketCreatesFailed");
-    rebalanceBucketCreateTimeId = type.nameToId("rebalanceBucketCreateTime");
-    rebalancePrimaryTransfersInProgressId = type.nameToId("rebalancePrimaryTransfersInProgress");
-    rebalancePrimaryTransfersCompletedId = type.nameToId("rebalancePrimaryTransfersCompleted");
-    rebalancePrimaryTransfersFailedId = type.nameToId("rebalancePrimaryTransfersFailed");
-    rebalancePrimaryTransferTimeId = type.nameToId("rebalancePrimaryTransferTime");
-
-    applyReplicationCompletedId = type.nameToId("applyReplicationCompleted");
-    applyReplicationInProgressId = type.nameToId("applyReplicationInProgress");
-    applyReplicationTimeId = type.nameToId("applyReplicationTime");
-    sendReplicationCompletedId = type.nameToId("sendReplicationCompleted");
-    sendReplicationInProgressId = type.nameToId("sendReplicationInProgress");
-    sendReplicationTimeId = type.nameToId("sendReplicationTime");
-    putRemoteCompletedId = type.nameToId("putRemoteCompleted");
-    putRemoteInProgressId = type.nameToId("putRemoteInProgress");
-    putRemoteTimeId = type.nameToId("putRemoteTime");
-    putLocalCompletedId = type.nameToId("putLocalCompleted");
-    putLocalInProgressId = type.nameToId("putLocalInProgress");
-    putLocalTimeId = type.nameToId("putLocalTime");
-
-    prMetaDataSentCountId = type.nameToId("prMetaDataSentCount");
-
-    localMaxMemoryId = type.nameToId("localMaxMemory");
-  }
-
-  private final Statistics stats;
-
-  /**
-   * Utility map for temporarily holding stat start times.
-   * <p>
-   * This was originally added to avoid having to add a long volunteeringStarted variable to every
-   * instance of BucketAdvisor. Majority of BucketAdvisors never volunteer and an instance of
-   * BucketAdvisor exists for every bucket defined in a PartitionedRegion which could result in a
-   * lot of unused longs. Volunteering is a rare event and thus the performance implications of a
-   * HashMap lookup is small and preferrable to so many longs. Key: BucketAdvisor, Value: Long
-   */
-  private final Map startTimeMap;
-
-  public static long startTime() {
+public interface PartitionedRegionStats {
+  default long startTime() {
     return CachePerfStats.getStatTime();
   }
-
-  public static long getStatTime() {
+  default long getStatTime() {
     return CachePerfStats.getStatTime();
   }
-
-  public PartitionedRegionStats(StatisticsFactory factory, String name) {
-    this.stats = factory.createAtomicStatistics(type, name /* fixes bug 42343 */);
-
-    if (CachePerfStats.enableClockStats) {
-      this.startTimeMap = new ConcurrentHashMap();
-    } else {
-      this.startTimeMap = Collections.EMPTY_MAP;
-    }
-  }
-
-  public void close() {
-    this.stats.close();
-  }
-
-  public Statistics getStats() {
-    return this.stats;
-  }
-
-  // ------------------------------------------------------------------------
-  // region op stats
-  // ------------------------------------------------------------------------
-
-  public void endPut(long start) {
-    endPut(start, 1);
-  }
-
-  /**
-   * This method sets the end time for putAll and updates the counters
-   *
-   * @param start
-   */
-  public void endPutAll(long start) {
-    endPutAll(start, 1);
-  }
-
-  public void endRemoveAll(long start) {
-    endRemoveAll(start, 1);
-  }
-
-  public void endCreate(long start) {
-    endCreate(start, 1);
-  }
-
-  public void endGet(long start) {
-    endGet(start, 1);
-  }
-
-  public void endContainsKey(long start) {
-    endContainsKey(start, 1);
-  }
-
-  public void endContainsValueForKey(long start) {
-    endContainsValueForKey(start, 1);
-  }
-
-  public void endPut(long start, int numInc) {
-    if (CachePerfStats.enableClockStats) {
-      long delta = CachePerfStats.getStatTime() - start;
-      this.stats.incLong(putTimeId, delta);
-    }
-    this.stats.incInt(putsCompletedId, numInc);
-  }
-
-  /**
-   * This method sets the end time for putAll and updates the counters
-   *
-   * @param start
-   * @param numInc
-   */
-  public void endPutAll(long start, int numInc) {
-    if (CachePerfStats.enableClockStats) {
-      long delta = CachePerfStats.getStatTime() - start;
-      this.stats.incLong(fieldId_PUTALL_TIME, delta);
-      // this.putStatsHistogram.endOp(delta);
-
-    }
-    this.stats.incInt(fieldId_PUTALLS_COMPLETED, numInc);
-  }
-
-  public void endRemoveAll(long start, int numInc) {
-    if (CachePerfStats.enableClockStats) {
-      long delta = CachePerfStats.getStatTime() - start;
-      this.stats.incLong(fieldId_REMOVE_ALL_TIME, delta);
-    }
-    this.stats.incInt(fieldId_REMOVE_ALLS_COMPLETED, numInc);
-  }
-
-  public void endCreate(long start, int numInc) {
-    if (CachePerfStats.enableClockStats) {
-      this.stats.incLong(createTimeId, CachePerfStats.getStatTime() - start);
-    }
-    this.stats.incInt(createsCompletedId, numInc);
-  }
-
-  public void endGet(long start, int numInc) {
-    if (CachePerfStats.enableClockStats) {
-      final long delta = CachePerfStats.getStatTime() - start;
-      this.stats.incLong(getTimeId, delta);
-    }
-    this.stats.incInt(getsCompletedId, numInc);
-  }
-
-  public void endDestroy(long start) {
-    if (CachePerfStats.enableClockStats) {
-      this.stats.incLong(destroyTimeId, CachePerfStats.getStatTime() - start);
-    }
-    this.stats.incInt(destroysCompletedId, 1);
-  }
-
-  public void endInvalidate(long start) {
-    if (CachePerfStats.enableClockStats) {
-      this.stats.incLong(invalidateTimeId, CachePerfStats.getStatTime() - start);
-    }
-    this.stats.incInt(invalidatesCompletedId, 1);
-  }
-
-  public void endContainsKey(long start, int numInc) {
-    if (CachePerfStats.enableClockStats) {
-      this.stats.incLong(containsKeyTimeId, CachePerfStats.getStatTime() - start);
-    }
-    this.stats.incInt(containsKeyCompletedId, numInc);
-  }
-
-  public void endContainsValueForKey(long start, int numInc) {
-    if (CachePerfStats.enableClockStats) {
-      this.stats.incLong(containsValueForKeyTimeId, CachePerfStats.getStatTime() - start);
-    }
-    this.stats.incInt(containsValueForKeyCompletedId, numInc);
-  }
-
-  public void incContainsKeyValueRetries() {
-    this.stats.incInt(containsKeyRetriesId, 1);
-  }
-
-  public void incContainsKeyValueOpsRetried() {
-    this.stats.incInt(containsKeyOpsRetriedId, 1);
-  }
-
-  public void incInvalidateRetries() {
-    this.stats.incInt(invalidateRetriesId, 1);
-  }
-
-  public void incInvalidateOpsRetried() {
-    this.stats.incInt(invalidateOpsRetriedId, 1);
-  }
-
-  public void incDestroyRetries() {
-    this.stats.incInt(destroyRetriesId, 1);
-  }
-
-  public void incDestroyOpsRetried() {
-    this.stats.incInt(destroyOpsRetriedId, 1);
-  }
-
-  public void incPutRetries() {
-    this.stats.incInt(putRetriesId, 1);
-  }
-
-  public void incPutOpsRetried() {
-    this.stats.incInt(putOpsRetriedId, 1);
-  }
-
-  public void incGetOpsRetried() {
-    this.stats.incInt(getOpsRetriedId, 1);
-  }
-
-  public void incGetRetries() {
-    this.stats.incInt(getRetriesId, 1);
-  }
-
-  public void incCreateOpsRetried() {
-    this.stats.incInt(createOpsRetriedId, 1);
-  }
-
-  public void incCreateRetries() {
-    this.stats.incInt(createRetriesId, 1);
-  }
-
-  // ------------------------------------------------------------------------
-  // preferred read stats
-  // ------------------------------------------------------------------------
-
-  public void incPreferredReadLocal() {
-    this.stats.incInt(preferredReadLocalId, 1);
-  }
-
-  public void incPreferredReadRemote() {
-    this.stats.incInt(preferredReadRemoteId, 1);
-  }
-
-  // ------------------------------------------------------------------------
-  // messaging stats
-  // ------------------------------------------------------------------------
-
-  public long startPartitionMessageProcessing() {
-    this.stats.incInt(partitionMessagesReceivedId, 1);
-    return startTime();
-  }
-
-  public void endPartitionMessagesProcessing(long start) {
-    if (CachePerfStats.enableClockStats) {
-      long delta = CachePerfStats.getStatTime() - start;
-      this.stats.incLong(partitionMessagesProcessingTimeId, delta);
-    }
-    this.stats.incInt(partitionMessagesProcessedId, 1);
-  }
-
-  public void incPartitionMessagesSent() {
-    this.stats.incInt(partitionMessagesSentId, 1);
-  }
-
-  // ------------------------------------------------------------------------
-  // datastore stats
-  // ------------------------------------------------------------------------
-
-  public void incBucketCount(int delta) {
-    this.stats.incInt(bucketCountId, delta);
-  }
-
-  public void setBucketCount(int i) {
-    this.stats.setInt(bucketCountId, i);
-  }
-
-  public void incDataStoreEntryCount(int amt) {
-    this.stats.incInt(dataStoreEntryCountId, amt);
-  }
-
-  public int getDataStoreEntryCount() {
-    return this.stats.getInt(dataStoreEntryCountId);
-  }
-
-  public void incBytesInUse(long delta) {
-    this.stats.incLong(dataStoreBytesInUseId, delta);
-  }
-
-  public long getDataStoreBytesInUse() {
-    return this.stats.getLong(dataStoreBytesInUseId);
-  }
-
-  public int getTotalBucketCount() {
-    int bucketCount = this.stats.getInt(bucketCountId);
-    return bucketCount;
-  }
-
-  public void incPutAllRetries() {
-    this.stats.incInt(fieldId_PUTALL_RETRIES, 1);
-  }
-
-  public void incPutAllMsgsRetried() {
-    this.stats.incInt(fieldId_PUTALL_MSGS_RETRIED, 1);
-  }
-
-  public void incRemoveAllRetries() {
-    this.stats.incInt(fieldId_REMOVE_ALL_RETRIES, 1);
-  }
-
-  public void incRemoveAllMsgsRetried() {
-    this.stats.incInt(fieldId_REMOVE_ALL_MSGS_RETRIED, 1);
-  }
-
-  // ------------------------------------------------------------------------
-  // stats for volunteering/discovering/becoming primary
-  // ------------------------------------------------------------------------
-
-  public int getVolunteeringInProgress() {
-    return this.stats.getInt(volunteeringInProgressId);
-  }
-
-  public int getVolunteeringBecamePrimary() {
-    return this.stats.getInt(volunteeringBecamePrimaryId);
-  }
-
-  public long getVolunteeringBecamePrimaryTime() {
-    return this.stats.getLong(volunteeringBecamePrimaryTimeId);
-  }
-
-  public int getVolunteeringOtherPrimary() {
-    return this.stats.getInt(volunteeringOtherPrimaryId);
-  }
-
-  public long getVolunteeringOtherPrimaryTime() {
-    return this.stats.getLong(volunteeringOtherPrimaryTimeId);
-  }
-
-  public int getVolunteeringClosed() {
-    return this.stats.getInt(volunteeringClosedId);
-  }
-
-  public long getVolunteeringClosedTime() {
-    return this.stats.getLong(volunteeringClosedTimeId);
-  }
-
-  public long startVolunteering() {
-    this.stats.incInt(volunteeringInProgressId, 1);
-    return CachePerfStats.getStatTime();
-  }
-
-  public void endVolunteeringBecamePrimary(long start) {
-    long ts = CachePerfStats.getStatTime();
-    this.stats.incInt(volunteeringInProgressId, -1);
-    this.stats.incInt(volunteeringBecamePrimaryId, 1);
-    if (CachePerfStats.enableClockStats) {
-      long time = ts - start;
-      this.stats.incLong(volunteeringBecamePrimaryTimeId, time);
-    }
-  }
-
-  public void endVolunteeringOtherPrimary(long start) {
-    long ts = CachePerfStats.getStatTime();
-    this.stats.incInt(volunteeringInProgressId, -1);
-    this.stats.incInt(volunteeringOtherPrimaryId, 1);
-    if (CachePerfStats.enableClockStats) {
-      long time = ts - start;
-      this.stats.incLong(volunteeringOtherPrimaryTimeId, time);
-    }
-  }
-
-  public void endVolunteeringClosed(long start) {
-    long ts = CachePerfStats.getStatTime();
-    this.stats.incInt(volunteeringInProgressId, -1);
-    this.stats.incInt(volunteeringClosedId, 1);
-    if (CachePerfStats.enableClockStats) {
-      long time = ts - start;
-      this.stats.incLong(volunteeringClosedTimeId, time);
-    }
-  }
-
-  public int getTotalNumBuckets() {
-    return this.stats.getInt(totalNumBucketsId);
-  }
-
-  public void incTotalNumBuckets(int val) {
-    this.stats.incInt(totalNumBucketsId, val);
-  }
-
-  public int getPrimaryBucketCount() {
-    return this.stats.getInt(primaryBucketCountId);
-  }
-
-  public void incPrimaryBucketCount(int val) {
-    this.stats.incInt(primaryBucketCountId, val);
-  }
-
-  public int getVolunteeringThreads() {
-    return this.stats.getInt(volunteeringThreadsId);
-  }
-
-  public void incVolunteeringThreads(int val) {
-    this.stats.incInt(volunteeringThreadsId, val);
-  }
-
-  public int getLowRedundancyBucketCount() {
-    return this.stats.getInt(lowRedundancyBucketCountId);
-  }
-
-  public int getNoCopiesBucketCount() {
-    return this.stats.getInt(noCopiesBucketCountId);
-  }
-
-  public void incLowRedundancyBucketCount(int val) {
-    this.stats.incInt(lowRedundancyBucketCountId, val);
-  }
-
-  public void incNoCopiesBucketCount(int val) {
-    this.stats.incInt(noCopiesBucketCountId, val);
-  }
-
-  public int getConfiguredRedundantCopies() {
-    return this.stats.getInt(configuredRedundantCopiesId);
-  }
-
-  public void setConfiguredRedundantCopies(int val) {
-    this.stats.setInt(configuredRedundantCopiesId, val);
-  }
-
-  public void setLocalMaxMemory(long l) {
-    this.stats.setLong(localMaxMemoryId, l);
-  }
-
-  public int getActualRedundantCopies() {
-    return this.stats.getInt(actualRedundantCopiesId);
-  }
-
-  public void setActualRedundantCopies(int val) {
-    this.stats.setInt(actualRedundantCopiesId, val);
-  }
-
-  // ------------------------------------------------------------------------
-  // startTimeMap methods
-  // ------------------------------------------------------------------------
-
-  /** Put stat start time in holding map for later removal and use by caller */
-  public void putStartTime(Object key, long startTime) {
-    if (CachePerfStats.enableClockStats) {
-      this.startTimeMap.put(key, Long.valueOf(startTime));
-    }
-  }
-
-  /** Remove stat start time from holding map to complete a clock stat */
-  public long removeStartTime(Object key) {
-    Long startTime = (Long) this.startTimeMap.remove(key);
-    return startTime == null ? 0 : startTime.longValue();
-  }
-
-  /**
-   * Statistic to track the {@link Region#getEntry(Object)} call
-   *
-   * @param startTime the time the getEntry operation started
-   */
-  public void endGetEntry(long startTime) {
-    endGetEntry(startTime, 1);
-  }
-
-  /**
-   * This method sets the end time for update and updates the counters
-   *
-   * @param start
-   * @param numInc
-   */
-  public void endGetEntry(long start, int numInc) {
-    if (CachePerfStats.enableClockStats) {
-      this.stats.incLong(getEntryTimeId, CachePerfStats.getStatTime() - start);
-    }
-    this.stats.incInt(getEntriesCompletedId, numInc);
-  }
-
-  // ------------------------------------------------------------------------
-  // bucket creation, primary transfer stats (see also rebalancing stats below)
-  // ------------------------------------------------------------------------
-  public long startRecovery() {
-    this.stats.incInt(recoveriesInProgressId, 1);
-    return PartitionedRegionStats.getStatTime();
-  }
-
-  public void endRecovery(long start) {
-    long ts = PartitionedRegionStats.getStatTime();
-    this.stats.incInt(recoveriesInProgressId, -1);
-    if (CachePerfStats.enableClockStats) {
-      this.stats.incLong(recoveriesTimeId, ts - start);
-    }
-    this.stats.incInt(recoveriesCompletedId, 1);
-  }
-
-  public long startBucketCreate(boolean isRebalance) {
-    this.stats.incInt(bucketCreatesInProgressId, 1);
-    if (isRebalance) {
-      startRebalanceBucketCreate();
-    }
-    return PartitionedRegionStats.getStatTime();
-  }
-
-  public void endBucketCreate(long start, boolean success, boolean isRebalance) {
-    long ts = PartitionedRegionStats.getStatTime();
-    this.stats.incInt(bucketCreatesInProgressId, -1);
-    if (CachePerfStats.enableClockStats) {
-      this.stats.incLong(bucketCreateTimeId, ts - start);
-    }
-    if (success) {
-      this.stats.incInt(bucketCreatesCompletedId, 1);
-    } else {
-      this.stats.incInt(bucketCreatesFailedId, 1);
-    }
-    if (isRebalance) {
-      endRebalanceBucketCreate(start, ts, success);
-    }
-  }
-
-  public long startPrimaryTransfer(boolean isRebalance) {
-    this.stats.incInt(primaryTransfersInProgressId, 1);
-    if (isRebalance) {
-      startRebalancePrimaryTransfer();
-    }
-    return PartitionedRegionStats.getStatTime();
-  }
-
-  public void endPrimaryTransfer(long start, boolean success, boolean isRebalance) {
-    long ts = PartitionedRegionStats.getStatTime();
-    this.stats.incInt(primaryTransfersInProgressId, -1);
-    if (CachePerfStats.enableClockStats) {
-      this.stats.incLong(primaryTransferTimeId, ts - start);
-    }
-    if (success) {
-      this.stats.incInt(primaryTransfersCompletedId, 1);
-    } else {
-      this.stats.incInt(primaryTransfersFailedId, 1);
-    }
-    if (isRebalance) {
-      endRebalancePrimaryTransfer(start, ts, success);
-    }
-  }
-
-  public int getBucketCreatesInProgress() {
-    return this.stats.getInt(bucketCreatesInProgressId);
-  }
-
-  public int getBucketCreatesCompleted() {
-    return this.stats.getInt(bucketCreatesCompletedId);
-  }
-
-  public int getBucketCreatesFailed() {
-    return this.stats.getInt(bucketCreatesFailedId);
-  }
-
-  public long getBucketCreateTime() {
-    return this.stats.getLong(bucketCreateTimeId);
-  }
-
-  public int getPrimaryTransfersInProgress() {
-    return this.stats.getInt(primaryTransfersInProgressId);
-  }
-
-  public int getPrimaryTransfersCompleted() {
-    return this.stats.getInt(primaryTransfersCompletedId);
-  }
-
-  public int getPrimaryTransfersFailed() {
-    return this.stats.getInt(primaryTransfersFailedId);
-  }
-
-  public long getPrimaryTransferTime() {
-    return this.stats.getLong(primaryTransferTimeId);
-  }
-
-  // ------------------------------------------------------------------------
-  // rebalancing stats
-  // ------------------------------------------------------------------------
-
-  private void startRebalanceBucketCreate() {
-    this.stats.incInt(rebalanceBucketCreatesInProgressId, 1);
-  }
-
-  private void endRebalanceBucketCreate(long start, long end, boolean success) {
-    this.stats.incInt(rebalanceBucketCreatesInProgressId, -1);
-    if (CachePerfStats.enableClockStats) {
-      this.stats.incLong(rebalanceBucketCreateTimeId, end - start);
-    }
-    if (success) {
-      this.stats.incInt(rebalanceBucketCreatesCompletedId, 1);
-    } else {
-      this.stats.incInt(rebalanceBucketCreatesFailedId, 1);
-    }
-  }
-
-  private void startRebalancePrimaryTransfer() {
-    this.stats.incInt(rebalancePrimaryTransfersInProgressId, 1);
-  }
-
-  private void endRebalancePrimaryTransfer(long start, long end, boolean success) {
-    this.stats.incInt(rebalancePrimaryTransfersInProgressId, -1);
-    if (CachePerfStats.enableClockStats) {
-      this.stats.incLong(rebalancePrimaryTransferTimeId, end - start);
-    }
-    if (success) {
-      this.stats.incInt(rebalancePrimaryTransfersCompletedId, 1);
-    } else {
-      this.stats.incInt(rebalancePrimaryTransfersFailedId, 1);
-    }
-  }
-
-  public int getRebalanceBucketCreatesInProgress() {
-    return this.stats.getInt(rebalanceBucketCreatesInProgressId);
-  }
-
-  public int getRebalanceBucketCreatesCompleted() {
-    return this.stats.getInt(rebalanceBucketCreatesCompletedId);
-  }
-
-  public int getRebalanceBucketCreatesFailed() {
-    return this.stats.getInt(rebalanceBucketCreatesFailedId);
-  }
-
-  public long getRebalanceBucketCreateTime() {
-    return this.stats.getLong(rebalanceBucketCreateTimeId);
-  }
-
-  public int getRebalancePrimaryTransfersInProgress() {
-    return this.stats.getInt(rebalancePrimaryTransfersInProgressId);
-  }
-
-  public int getRebalancePrimaryTransfersCompleted() {
-    return this.stats.getInt(rebalancePrimaryTransfersCompletedId);
-  }
-
-  public int getRebalancePrimaryTransfersFailed() {
-    return this.stats.getInt(rebalancePrimaryTransfersFailedId);
-  }
-
-  public long getRebalancePrimaryTransferTime() {
-    return this.stats.getLong(rebalancePrimaryTransferTimeId);
-  }
-
-  public long startApplyReplication() {
-    stats.incInt(applyReplicationInProgressId, 1);
-    return CachePerfStats.getStatTime();
-  }
-
-  public void endApplyReplication(long start) {
-    long delta = CachePerfStats.getStatTime() - start;
-    stats.incInt(applyReplicationInProgressId, -1);
-    stats.incInt(applyReplicationCompletedId, 1);
-    stats.incLong(applyReplicationTimeId, delta);
-  }
-
-  public long startSendReplication() {
-    stats.incInt(sendReplicationInProgressId, 1);
-    return CachePerfStats.getStatTime();
-  }
-
-  public void endSendReplication(long start) {
-    long delta = CachePerfStats.getStatTime() - start;
-    stats.incInt(sendReplicationInProgressId, -1);
-    stats.incInt(sendReplicationCompletedId, 1);
-    stats.incLong(sendReplicationTimeId, delta);
-  }
-
-  public long startPutRemote() {
-    stats.incInt(putRemoteInProgressId, 1);
-    return CachePerfStats.getStatTime();
-  }
-
-  public void endPutRemote(long start) {
-    long delta = CachePerfStats.getStatTime() - start;
-    stats.incInt(putRemoteInProgressId, -1);
-    stats.incInt(putRemoteCompletedId, 1);
-    stats.incLong(putRemoteTimeId, delta);
-  }
-
-  public long startPutLocal() {
-    stats.incInt(putLocalInProgressId, 1);
-    return CachePerfStats.getStatTime();
-  }
-
-  public void endPutLocal(long start) {
-    long delta = CachePerfStats.getStatTime() - start;
-    stats.incInt(putLocalInProgressId, -1);
-    stats.incInt(putLocalCompletedId, 1);
-    stats.incLong(putLocalTimeId, delta);
-  }
-
-  public void incPRMetaDataSentCount() {
-    this.stats.incLong(prMetaDataSentCountId, 1);
-  }
-
-  public long getPRMetaDataSentCount() {
-    return this.stats.getLong(prMetaDataSentCountId);
-  }
+  void close();
+  void endPut(long start);
+  void endPutAll(long start);
+  void endRemoveAll(long start);
+  void endCreate(long start);
+  void endGet(long start);
+  void endContainsKey(long start);
+  void endContainsValueForKey(long start);
+  void endDestroy(long start);
+  void endInvalidate(long start);
+  void incContainsKeyValueRetries();
+  void incContainsKeyValueOpsRetried();
+  void incInvalidateRetries();
+  void incInvalidateOpsRetried();
+  void incDestroyRetries();
+  void incDestroyOpsRetried();
+  void incPutRetries();
+  void incPutOpsRetried();
+  void incGetOpsRetried();
+  void incGetRetries();
+  void incCreateOpsRetried();
+  void incCreateRetries();
+  void incPreferredReadLocal();
+  void incPreferredReadRemote();
+  long startPartitionMessageProcessing();
+  void endPartitionMessagesProcessing(long start);
+  void incPartitionMessagesSent();
+  void incBucketCount(int delta);
+  void setBucketCount(int i);
+  void incDataStoreEntryCount(int amt);
+  int getDataStoreEntryCount();
+  void incBytesInUse(long delta);
+  long getDataStoreBytesInUse();
+  int getTotalBucketCount();
+  void incPutAllRetries();
+  void incPutAllMsgsRetried();
+  void incRemoveAllRetries();
+  void incRemoveAllMsgsRetried();
+  int getVolunteeringInProgress();
+  int getVolunteeringBecamePrimary();
+  long getVolunteeringBecamePrimaryTime();
+  int getVolunteeringOtherPrimary();
+  long getVolunteeringOtherPrimaryTime();
+  int getVolunteeringClosed();
+  long getVolunteeringClosedTime();
+  long startVolunteering();
+  void endVolunteeringBecamePrimary(long start);
+  void endVolunteeringOtherPrimary(long start);
+  void endVolunteeringClosed(long start);
+  int getTotalNumBuckets();
+  void incTotalNumBuckets(int val);
+  int getPrimaryBucketCount();
+  void incPrimaryBucketCount(int val);
+  int getVolunteeringThreads();
+  void incVolunteeringThreads(int val);
+  int getLowRedundancyBucketCount();
+  int getNoCopiesBucketCount();
+  void incLowRedundancyBucketCount(int val);
+  void incNoCopiesBucketCount(int val);
+  int getConfiguredRedundantCopies();
+  void setConfiguredRedundantCopies(int val);
+  void setLocalMaxMemory(long l);
+  int getActualRedundantCopies();
+  void setActualRedundantCopies(int val);
+  void putStartTime(Object key, long startTime);
+  long removeStartTime(Object key);
+  void endGetEntry(long startTime);
+  void endGetEntry(long start, int numInc);
+  long startRecovery();
+  void endRecovery(long start);
+  long startBucketCreate(boolean isRebalance);
+  void endBucketCreate(long start, boolean success, boolean isRebalance);
+  long startPrimaryTransfer(boolean isRebalance);
+  void endPrimaryTransfer(long start, boolean success, boolean isRebalance);
+  int getBucketCreatesInProgress();
+  int getBucketCreatesCompleted();
+  int getBucketCreatesFailed();
+  long getBucketCreateTime();
+  int getPrimaryTransfersInProgress();
+  int getPrimaryTransfersCompleted();
+  int getPrimaryTransfersFailed();
+  long getPrimaryTransferTime();
+  int getRebalanceBucketCreatesInProgress();
+  int getRebalanceBucketCreatesCompleted();
+  int getRebalanceBucketCreatesFailed();
+  long getRebalanceBucketCreateTime();
+  int getRebalancePrimaryTransfersInProgress();
+  int getRebalancePrimaryTransfersCompleted();
+  int getRebalancePrimaryTransfersFailed();
+  long getRebalancePrimaryTransferTime();
+  long startApplyReplication();
+  void endApplyReplication(long start);
+  long startSendReplication();
+  void endSendReplication(long start);
+  long startPutRemote();
+  void endPutRemote(long start);
+  long startPutLocal();
+  void endPutLocal(long start);
+  void incPRMetaDataSentCount();
+  long getPRMetaDataSentCount();
+
+    Statistics getStats();
 }
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/PartitionedRegionStatsImpl.java b/geode-core/src/main/java/org/apache/geode/internal/cache/PartitionedRegionStatsImpl.java
index 24cd598..aa43145 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/PartitionedRegionStatsImpl.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/PartitionedRegionStatsImpl.java
@@ -175,253 +175,102 @@ public class PartitionedRegionStatsImpl implements PartitionedRegionStats {
   private static final int localMaxMemoryId;
 
   static {
-    final boolean largerIsBetter = true;
     StatisticsTypeFactory f = StatisticsTypeFactoryImpl.singleton();
     type = f.createType("PartitionedRegionStatsImpl",
         "Statistics for operations and connections in the Partitioned Region",
         new StatisticDescriptor[] {
-
             f.createIntGauge("bucketCount", "Number of buckets in this node.", "buckets"),
-            f.createIntCounter("putsCompleted", "Number of puts completed.", "operations",
-                largerIsBetter),
-            f.createIntCounter("putOpsRetried",
-                "Number of put operations which had to be retried due to failures.", "operations",
-                false),
-            f.createIntCounter("putRetries",
-                "Total number of times put operations had to be retried.", "retry attempts", false),
-            f.createIntCounter("createsCompleted", "Number of creates completed.", "operations",
-                largerIsBetter),
-            f.createIntCounter("createOpsRetried",
-                "Number of create operations which had to be retried due to failures.",
-                "operations", false),
-            f.createIntCounter("createRetries",
-                "Total number of times put operations had to be retried.", "retry attempts", false),
-            f.createIntCounter("preferredReadLocal", "Number of reads satisfied from local store",
-                "operations", largerIsBetter),
-            f.createIntCounter(PUTALLS_COMPLETED, "Number of putAlls completed.", "operations",
-                largerIsBetter),
-            f.createIntCounter(PUTALL_MSGS_RETRIED,
-                "Number of putAll messages which had to be retried due to failures.", "operations",
-                false),
-            f.createIntCounter(PUTALL_RETRIES,
-                "Total number of times putAll messages had to be retried.", "retry attempts",
-                false),
-            f.createLongCounter(PUTALL_TIME, "Total time spent doing putAlls.", "nanoseconds",
-                !largerIsBetter),
-            f.createIntCounter(REMOVE_ALLS_COMPLETED, "Number of removeAlls completed.",
-                "operations", largerIsBetter),
-            f.createIntCounter(REMOVE_ALL_MSGS_RETRIED,
-                "Number of removeAll messages which had to be retried due to failures.",
-                "operations", false),
-            f.createIntCounter(REMOVE_ALL_RETRIES,
-                "Total number of times removeAll messages had to be retried.", "retry attempts",
-                false),
-            f.createLongCounter(REMOVE_ALL_TIME, "Total time spent doing removeAlls.",
-                "nanoseconds", !largerIsBetter),
-            f.createIntCounter("preferredReadRemote", "Number of reads satisfied from remote store",
-                "operations", false),
-            f.createIntCounter("getsCompleted", "Number of gets completed.", "operations",
-                largerIsBetter),
-            f.createIntCounter("getOpsRetried",
-                "Number of get operations which had to be retried due to failures.", "operations",
-                false),
-            f.createIntCounter("getRetries",
-                "Total number of times get operations had to be retried.", "retry attempts", false),
-            f.createIntCounter("destroysCompleted", "Number of destroys completed.", "operations",
-                largerIsBetter),
-            f.createIntCounter("destroyOpsRetried",
-                "Number of destroy operations which had to be retried due to failures.",
-                "operations", false),
-            f.createIntCounter("destroyRetries",
-                "Total number of times destroy operations had to be retried.", "retry attempts",
-                false),
-            f.createIntCounter("invalidatesCompleted", "Number of invalidates completed.",
-                "operations", largerIsBetter),
-
-            f.createIntCounter("invalidateOpsRetried",
-                "Number of invalidate operations which had to be retried due to failures.",
-                "operations", false),
-            f.createIntCounter("invalidateRetries",
-                "Total number of times invalidate operations had to be retried.", "retry attempts",
-                false),
-            f.createIntCounter("containsKeyCompleted", "Number of containsKeys completed.",
-                "operations", largerIsBetter),
-
-            f.createIntCounter("containsKeyOpsRetried",
-                "Number of containsKey or containsValueForKey operations which had to be retried due to failures.",
-                "operations", false),
-            f.createIntCounter("containsKeyRetries",
-                "Total number of times containsKey or containsValueForKey operations had to be retried.",
-                "operations", false),
-            f.createIntCounter("containsValueForKeyCompleted",
-                "Number of containsValueForKeys completed.", "operations", largerIsBetter),
-            f.createIntCounter("PartitionMessagesSent", "Number of PartitionMessages Sent.",
-                "operations", largerIsBetter),
-            f.createIntCounter("PartitionMessagesReceived", "Number of PartitionMessages Received.",
-                "operations", largerIsBetter),
-            f.createIntCounter("PartitionMessagesProcessed",
-                "Number of PartitionMessages Processed.", "operations", largerIsBetter),
+            f.createIntCounter("putsCompleted", "Number of puts completed.", "operations", true),
+            f.createIntCounter("putOpsRetried", "Number of put operations which had to be retried due to failures.", "operations", false),
+            f.createIntCounter("putRetries", "Total number of times put operations had to be retried.", "retry attempts", false),
+            f.createIntCounter("createsCompleted", "Number of creates completed.", "operations", true),
+            f.createIntCounter("createOpsRetried", "Number of create operations which had to be retried due to failures.", "operations", false),
+            f.createIntCounter("createRetries", "Total number of times put operations had to be retried.", "retry attempts", false),
+            f.createIntCounter("preferredReadLocal", "Number of reads satisfied from local store", "operations", true),
+            f.createIntCounter(PUTALLS_COMPLETED, "Number of putAlls completed.", "operations", true),
+            f.createIntCounter(PUTALL_MSGS_RETRIED, "Number of putAll messages which had to be retried due to failures.", "operations", false),
+            f.createIntCounter(PUTALL_RETRIES, "Total number of times putAll messages had to be retried.", "retry attempts", false),
+            f.createLongCounter(PUTALL_TIME, "Total time spent doing putAlls.", "nanoseconds", false),
+            f.createIntCounter(REMOVE_ALLS_COMPLETED, "Number of removeAlls completed.", "operations", true),
+            f.createIntCounter(REMOVE_ALL_MSGS_RETRIED, "Number of removeAll messages which had to be retried due to failures.", "operations", false),
+            f.createIntCounter(REMOVE_ALL_RETRIES, "Total number of times removeAll messages had to be retried.", "retry attempts", false),
+            f.createLongCounter(REMOVE_ALL_TIME, "Total time spent doing removeAlls.", "nanoseconds", false),
+            f.createIntCounter("preferredReadRemote", "Number of reads satisfied from remote store", "operations", false),
+            f.createIntCounter("getsCompleted", "Number of gets completed.", "operations", true),
+            f.createIntCounter("getOpsRetried", "Number of get operations which had to be retried due to failures.", "operations", false),
+            f.createIntCounter("getRetries", "Total number of times get operations had to be retried.", "retry attempts", false),
+            f.createIntCounter("destroysCompleted", "Number of destroys completed.", "operations", true),
+            f.createIntCounter("destroyOpsRetried", "Number of destroy operations which had to be retried due to failures.", "operations", false),
+            f.createIntCounter("destroyRetries", "Total number of times destroy operations had to be retried.", "retry attempts", false),
+            f.createIntCounter("invalidatesCompleted", "Number of invalidates completed.", "operations", true),
+            f.createIntCounter("invalidateOpsRetried", "Number of invalidate operations which had to be retried due to failures.", "operations", false),
+            f.createIntCounter("invalidateRetries", "Total number of times invalidate operations had to be retried.", "retry attempts", false),
+            f.createIntCounter("containsKeyCompleted", "Number of containsKeys completed.", "operations", true),
+            f.createIntCounter("containsKeyOpsRetried", "Number of containsKey or containsValueForKey operations which had to be retried due to failures.", "operations", false),
+            f.createIntCounter("containsKeyRetries", "Total number of times containsKey or containsValueForKey operations had to be retried.", "operations", false),
+            f.createIntCounter("containsValueForKeyCompleted", "Number of containsValueForKeys completed.", "operations", true),
+            f.createIntCounter("PartitionMessagesSent", "Number of PartitionMessages Sent.", "operations", true),
+            f.createIntCounter("PartitionMessagesReceived", "Number of PartitionMessages Received.", "operations", true),
+            f.createIntCounter("PartitionMessagesProcessed", "Number of PartitionMessages Processed.", "operations", true),
             f.createLongCounter("putTime", "Total time spent doing puts.", "nanoseconds", false),
-            f.createLongCounter("createTime", "Total time spent doing create operations.",
-                "nanoseconds", false),
-            f.createLongCounter("getTime", "Total time spent performing get operations.",
-                "nanoseconds", false),
-            f.createLongCounter("destroyTime", "Total time spent doing destroys.", "nanoseconds",
-                false),
-            f.createLongCounter("invalidateTime", "Total time spent doing invalidates.",
-                "nanoseconds", false),
-            f.createLongCounter("containsKeyTime",
-                "Total time spent performing containsKey operations.", "nanoseconds", false),
-            f.createLongCounter("containsValueForKeyTime",
-                "Total time spent performing containsValueForKey operations.", "nanoseconds",
-                false),
-            f.createLongCounter("partitionMessagesProcessingTime",
-                "Total time spent on PartitionMessages processing.", "nanoseconds", false),
-            f.createIntGauge("dataStoreEntryCount",
-                "The number of entries stored in this Cache for the named Partitioned Region. This does not include entries which are tombstones. See CachePerfStats.tombstoneCount.",
-                "entries"),
-            f.createLongGauge("dataStoreBytesInUse",
-                "The current number of bytes stored in this Cache for the named Partitioned Region",
-                "bytes"),
-            f.createIntGauge("volunteeringInProgress",
-                "Current number of attempts to volunteer for primary of a bucket.", "operations"),
-            f.createIntCounter("volunteeringBecamePrimary",
-                "Total number of attempts to volunteer that ended when this member became primary.",
-                "operations"),
-            f.createLongCounter("volunteeringBecamePrimaryTime",
-                "Total time spent volunteering that ended when this member became primary.",
-                "nanoseconds", false),
-            f.createIntCounter("volunteeringOtherPrimary",
-                "Total number of attempts to volunteer that ended when this member discovered other primary.",
-                "operations"),
-            f.createLongCounter("volunteeringOtherPrimaryTime",
-                "Total time spent volunteering that ended when this member discovered other primary.",
-                "nanoseconds", false),
-            f.createIntCounter("volunteeringClosed",
-                "Total number of attempts to volunteer that ended when this member's bucket closed.",
-                "operations"),
-            f.createLongCounter("volunteeringClosedTime",
-                "Total time spent volunteering that ended when this member's bucket closed.",
-                "nanoseconds", false),
+            f.createLongCounter("createTime", "Total time spent doing create operations.", "nanoseconds", false),
+            f.createLongCounter("getTime", "Total time spent performing get operations.", "nanoseconds", false),
+            f.createLongCounter("destroyTime", "Total time spent doing destroys.", "nanoseconds", false),
+            f.createLongCounter("invalidateTime", "Total time spent doing invalidates.", "nanoseconds", false),
+            f.createLongCounter("containsKeyTime", "Total time spent performing containsKey operations.", "nanoseconds", false),
+            f.createLongCounter("containsValueForKeyTime", "Total time spent performing containsValueForKey operations.", "nanoseconds", false),
+            f.createLongCounter("partitionMessagesProcessingTime", "Total time spent on PartitionMessages processing.", "nanoseconds", false),
+            f.createIntGauge("dataStoreEntryCount", "The number of entries stored in this Cache for the named Partitioned Region. This does not include entries which are tombstones. See CachePerfStats.tombstoneCount.", "entries"),
+            f.createLongGauge("dataStoreBytesInUse", "The current number of bytes stored in this Cache for the named Partitioned Region", "bytes"),
+            f.createIntGauge("volunteeringInProgress","Current number of attempts to volunteer for primary of a bucket.", "operations"),
+            f.createIntCounter("volunteeringBecamePrimary", "Total number of attempts to volunteer that ended when this member became primary.", "operations"),
+            f.createLongCounter("volunteeringBecamePrimaryTime", "Total time spent volunteering that ended when this member became primary.", "nanoseconds", false),
+            f.createIntCounter("volunteeringOtherPrimary", "Total number of attempts to volunteer that ended when this member discovered other primary.", "operations"),
+            f.createLongCounter("volunteeringOtherPrimaryTime", "Total time spent volunteering that ended when this member discovered other primary.", "nanoseconds", false),
+            f.createIntCounter("volunteeringClosed", "Total number of attempts to volunteer that ended when this member's bucket closed.", "operations"),
+            f.createLongCounter("volunteeringClosedTime", "Total time spent volunteering that ended when this member's bucket closed.", "nanoseconds", false),
             f.createIntGauge("totalNumBuckets", "The total number of buckets.", "buckets"),
-            f.createIntGauge("primaryBucketCount",
-                "Current number of primary buckets hosted locally.", "buckets"),
-            f.createIntGauge("volunteeringThreads",
-                "Current number of threads volunteering for primary.", "threads"),
-            f.createIntGauge("lowRedundancyBucketCount",
-                "Current number of buckets without full redundancy.", "buckets"),
-            f.createIntGauge("noCopiesBucketCount",
-                "Current number of buckets without any copies remaining.", "buckets"),
-            f.createIntGauge("configuredRedundantCopies",
-                "Configured number of redundant copies for this partitioned region.", "copies"),
-            f.createIntGauge("actualRedundantCopies",
-                "Actual number of redundant copies for this partitioned region.", "copies"),
-            f.createIntCounter("getEntryCompleted", "Number of getEntry operations completed.",
-                "operations", largerIsBetter),
-            f.createLongCounter("getEntryTime", "Total time spent performing getEntry operations.",
-                "nanoseconds", false),
-
-            f.createIntGauge("recoveriesInProgress",
-                "Current number of redundancy recovery operations in progress for this region.",
-                "operations"),
-            f.createIntCounter("recoveriesCompleted",
-                "Total number of redundancy recovery operations performed on this region.",
-                "operations"),
-            f.createLongCounter("recoveryTime", "Total number time spent recovering redundancy.",
-                "operations"),
-            f.createIntGauge("bucketCreatesInProgress",
-                "Current number of bucket create operations being performed for rebalancing.",
-                "operations"),
-            f.createIntCounter("bucketCreatesCompleted",
-                "Total number of bucket create operations performed for rebalancing.",
-                "operations"),
-            f.createIntCounter("bucketCreatesFailed",
-                "Total number of bucket create operations performed for rebalancing that failed.",
-                "operations"),
-            f.createLongCounter("bucketCreateTime",
-                "Total time spent performing bucket create operations for rebalancing.",
-                "nanoseconds", false),
-            f.createIntGauge("primaryTransfersInProgress",
-                "Current number of primary transfer operations being performed for rebalancing.",
-                "operations"),
-            f.createIntCounter("primaryTransfersCompleted",
-                "Total number of primary transfer operations performed for rebalancing.",
-                "operations"),
-            f.createIntCounter("primaryTransfersFailed",
-                "Total number of primary transfer operations performed for rebalancing that failed.",
-                "operations"),
-            f.createLongCounter("primaryTransferTime",
-                "Total time spent performing primary transfer operations for rebalancing.",
-                "nanoseconds", false),
-
-            f.createIntCounter("applyReplicationCompleted",
-                "Total number of replicated values sent from a primary to this redundant data store.",
-                "operations", largerIsBetter),
-            f.createIntGauge("applyReplicationInProgress",
-                "Current number of replication operations in progress on this redundant data store.",
-                "operations", !largerIsBetter),
-            f.createLongCounter("applyReplicationTime",
-                "Total time spent storing replicated values on this redundant data store.",
-                "nanoseconds", !largerIsBetter),
-            f.createIntCounter("sendReplicationCompleted",
-                "Total number of replicated values sent from this primary to a redundant data store.",
-                "operations", largerIsBetter),
-            f.createIntGauge("sendReplicationInProgress",
-                "Current number of replication operations in progress from this primary.",
-                "operations", !largerIsBetter),
-            f.createLongCounter("sendReplicationTime",
-                "Total time spent replicating values from this primary to a redundant data store.",
-                "nanoseconds", !largerIsBetter),
-            f.createIntCounter("putRemoteCompleted",
-                "Total number of completed puts that did not originate in the primary. These puts require an extra network hop to the primary.",
-                "operations", largerIsBetter),
-            f.createIntGauge("putRemoteInProgress",
-                "Current number of puts in progress that did not originate in the primary.",
-                "operations", !largerIsBetter),
-            f.createLongCounter("putRemoteTime",
-                "Total time spent doing puts that did not originate in the primary.", "nanoseconds",
-                !largerIsBetter),
-            f.createIntCounter("putLocalCompleted",
-                "Total number of completed puts that did originate in the primary. These puts are optimal.",
-                "operations", largerIsBetter),
-            f.createIntGauge("putLocalInProgress",
-                "Current number of puts in progress that did originate in the primary.",
-                "operations", !largerIsBetter),
-            f.createLongCounter("putLocalTime",
-                "Total time spent doing puts that did originate in the primary.", "nanoseconds",
-                !largerIsBetter),
-
-            f.createIntGauge("rebalanceBucketCreatesInProgress",
-                "Current number of bucket create operations being performed for rebalancing.",
-                "operations"),
-            f.createIntCounter("rebalanceBucketCreatesCompleted",
-                "Total number of bucket create operations performed for rebalancing.",
-                "operations"),
-            f.createIntCounter("rebalanceBucketCreatesFailed",
-                "Total number of bucket create operations performed for rebalancing that failed.",
-                "operations"),
-            f.createLongCounter("rebalanceBucketCreateTime",
-                "Total time spent performing bucket create operations for rebalancing.",
-                "nanoseconds", false),
-            f.createIntGauge("rebalancePrimaryTransfersInProgress",
-                "Current number of primary transfer operations being performed for rebalancing.",
-                "operations"),
-            f.createIntCounter("rebalancePrimaryTransfersCompleted",
-                "Total number of primary transfer operations performed for rebalancing.",
-                "operations"),
-            f.createIntCounter("rebalancePrimaryTransfersFailed",
-                "Total number of primary transfer operations performed for rebalancing that failed.",
-                "operations"),
-            f.createLongCounter("rebalancePrimaryTransferTime",
-                "Total time spent performing primary transfer operations for rebalancing.",
-                "nanoseconds", false),
-            f.createLongCounter("prMetaDataSentCount",
-                "total number of times meta data refreshed sent on client's request.", "operation",
-                false),
-
-            f.createLongGauge("localMaxMemory",
-                "local max memory in bytes for this region on this member", "bytes")
-
+            f.createIntGauge("primaryBucketCount", "Current number of primary buckets hosted locally.", "buckets"),
+            f.createIntGauge("volunteeringThreads", "Current number of threads volunteering for primary.", "threads"),
+            f.createIntGauge("lowRedundancyBucketCount", "Current number of buckets without full redundancy.", "buckets"),
+            f.createIntGauge("noCopiesBucketCount", "Current number of buckets without any copies remaining.", "buckets"),
+            f.createIntGauge("configuredRedundantCopies", "Configured number of redundant copies for this partitioned region.", "copies"),
+            f.createIntGauge("actualRedundantCopies", "Actual number of redundant copies for this partitioned region.", "copies"),
+            f.createIntCounter("getEntryCompleted", "Number of getEntry operations completed.", "operations", true),
+            f.createLongCounter("getEntryTime", "Total time spent performing getEntry operations.", "nanoseconds", false),
+            f.createIntGauge("recoveriesInProgress", "Current number of redundancy recovery operations in progress for this region.", "operations"),
+            f.createIntCounter("recoveriesCompleted", "Total number of redundancy recovery operations performed on this region.", "operations"),
+            f.createLongCounter("recoveryTime", "Total number time spent recovering redundancy.", "operations"),
+            f.createIntGauge("bucketCreatesInProgress", "Current number of bucket create operations being performed for rebalancing.", "operations"),
+            f.createIntCounter("bucketCreatesCompleted", "Total number of bucket create operations performed for rebalancing.", "operations"),
+            f.createIntCounter("bucketCreatesFailed", "Total number of bucket create operations performed for rebalancing that failed.", "operations"),
+            f.createLongCounter("bucketCreateTime", "Total time spent performing bucket create operations for rebalancing.", "nanoseconds", false),
+            f.createIntGauge("primaryTransfersInProgress", "Current number of primary transfer operations being performed for rebalancing.", "operations"),
+            f.createIntCounter("primaryTransfersCompleted", "Total number of primary transfer operations performed for rebalancing.", "operations"),
+            f.createIntCounter("primaryTransfersFailed", "Total number of primary transfer operations performed for rebalancing that failed.", "operations"),
+            f.createLongCounter("primaryTransferTime", "Total time spent performing primary transfer operations for rebalancing.", "nanoseconds", false),
+            f.createIntCounter("applyReplicationCompleted", "Total number of replicated values sent from a primary to this redundant data store.", "operations", true),
+            f.createIntGauge("applyReplicationInProgress", "Current number of replication operations in progress on this redundant data store.", "operations", false),
+            f.createLongCounter("applyReplicationTime", "Total time spent storing replicated values on this redundant data store.", "nanoseconds", false),
+            f.createIntCounter("sendReplicationCompleted", "Total number of replicated values sent from this primary to a redundant data store.", "operations", true),
+            f.createIntGauge("sendReplicationInProgress", "Current number of replication operations in progress from this primary.", "operations", false),
+            f.createLongCounter("sendReplicationTime", "Total time spent replicating values from this primary to a redundant data store.", "nanoseconds", false),
+            f.createIntCounter("putRemoteCompleted", "Total number of completed puts that did not originate in the primary. These puts require an extra network hop to the primary.", "operations", true),
+            f.createIntGauge("putRemoteInProgress", "Current number of puts in progress that did not originate in the primary.", "operations", false),
+            f.createLongCounter("putRemoteTime", "Total time spent doing puts that did not originate in the primary.", "nanoseconds", false),
+            f.createIntCounter("putLocalCompleted", "Total number of completed puts that did originate in the primary. These puts are optimal.", "operations", true),
+            f.createIntGauge("putLocalInProgress", "Current number of puts in progress that did originate in the primary.", "operations", false),
+            f.createLongCounter("putLocalTime", "Total time spent doing puts that did originate in the primary.", "nanoseconds", false),
+            f.createIntGauge("rebalanceBucketCreatesInProgress", "Current number of bucket create operations being performed for rebalancing.", "operations"),
+            f.createIntCounter("rebalanceBucketCreatesCompleted", "Total number of bucket create operations performed for rebalancing.", "operations"),
+            f.createIntCounter("rebalanceBucketCreatesFailed", "Total number of bucket create operations performed for rebalancing that failed.", "operations"),
+            f.createLongCounter("rebalanceBucketCreateTime", "Total time spent performing bucket create operations for rebalancing.", "nanoseconds", false),
+            f.createIntGauge("rebalancePrimaryTransfersInProgress", "Current number of primary transfer operations being performed for rebalancing.", "operations"),
+            f.createIntCounter("rebalancePrimaryTransfersCompleted", "Total number of primary transfer operations performed for rebalancing.", "operations"),
+            f.createIntCounter("rebalancePrimaryTransfersFailed", "Total number of primary transfer operations performed for rebalancing that failed.", "operations"),
+            f.createLongCounter("rebalancePrimaryTransferTime", "Total time spent performing primary transfer operations for rebalancing.", "nanoseconds", false),
+            f.createLongCounter("prMetaDataSentCount", "total number of times meta data refreshed sent on client's request.", "operation", false),
+            f.createLongGauge("localMaxMemory", "local max memory in bytes for this region on this member", "bytes")
         });
 
     bucketCountId = type.nameToId("bucketCount");
@@ -566,94 +415,53 @@ public class PartitionedRegionStatsImpl implements PartitionedRegionStats {
 
   @Override
   public void endPut(long start) {
-    endPut(start, 1);
-  }
-
-  /**
-   * This method sets the end time for putAll and updates the counters
-   *
-   * @param start
-   */
-  @Override
-  public void endPutAll(long start) {
-    endPutAll(start, 1);
-  }
-
-  @Override
-  public void endRemoveAll(long start) {
-    endRemoveAll(start, 1);
-  }
-
-  @Override
-  public void endCreate(long start) {
-    endCreate(start, 1);
-  }
-
-  @Override
-  public void endGet(long start) {
-    endGet(start, 1);
-  }
-
-  @Override
-  public void endContainsKey(long start) {
-    endContainsKey(start, 1);
-  }
-
-  @Override
-  public void endContainsValueForKey(long start) {
-    endContainsValueForKey(start, 1);
-  }
-
-  @Override
-  public void endPut(long start, int numInc) {
     if (CachePerfStats.enableClockStats) {
       long delta = CachePerfStats.getStatTime() - start;
       this.stats.incLong(putTimeId, delta);
     }
-    this.stats.incInt(putsCompletedId, numInc);
+    this.stats.incInt(putsCompletedId, 1);
   }
 
   /**
    * This method sets the end time for putAll and updates the counters
    *
    * @param start
-   * @param numInc
    */
   @Override
-  public void endPutAll(long start, int numInc) {
+  public void endPutAll(long start) {
     if (CachePerfStats.enableClockStats) {
       long delta = CachePerfStats.getStatTime() - start;
       this.stats.incLong(fieldId_PUTALL_TIME, delta);
       // this.putStatsHistogram.endOp(delta);
 
     }
-    this.stats.incInt(fieldId_PUTALLS_COMPLETED, numInc);
+    this.stats.incInt(fieldId_PUTALLS_COMPLETED, 1);
   }
 
   @Override
-  public void endRemoveAll(long start, int numInc) {
+  public void endRemoveAll(long start) {
     if (CachePerfStats.enableClockStats) {
       long delta = CachePerfStats.getStatTime() - start;
       this.stats.incLong(fieldId_REMOVE_ALL_TIME, delta);
     }
-    this.stats.incInt(fieldId_REMOVE_ALLS_COMPLETED, numInc);
+    this.stats.incInt(fieldId_REMOVE_ALLS_COMPLETED, 1);
   }
 
   @Override
-  public void endCreate(long start, int numInc) {
+  public void endCreate(long start) {
     if (CachePerfStats.enableClockStats) {
       this.stats.incLong(createTimeId, CachePerfStats.getStatTime() - start);
     }
-    this.stats.incInt(createsCompletedId, numInc);
+    this.stats.incInt(createsCompletedId, 1);
   }
 
   @Override
-  public void endGet(long start, int numInc) {
+  public void endGet(long start) {
     if (CachePerfStats.enableClockStats) {
       final long delta = CachePerfStats.getStatTime() - start;
       this.stats.incLong(getTimeId, delta);
     }
-    this.stats.incInt(getsCompletedId, numInc);
+    this.stats.incInt(getsCompletedId, 1);
   }
 
   @Override
@@ -673,19 +481,19 @@ public class PartitionedRegionStatsImpl implements PartitionedRegionStats {
   }
 
   @Override
-  public void endContainsKey(long start, int numInc) {
+  public void endContainsKey(long start) {
     if (CachePerfStats.enableClockStats) {
       this.stats.incLong(containsKeyTimeId, CachePerfStats.getStatTime() - start);
     }
-    this.stats.incInt(containsKeyCompletedId, numInc);
+    this.stats.incInt(containsKeyCompletedId, 1);
   }
 
   @Override
-  public void endContainsValueForKey(long start, int numInc) {
+  public void endContainsValueForKey(long start) {
     if (CachePerfStats.enableClockStats) {
       this.stats.incLong(containsValueForKeyTimeId, CachePerfStats.getStatTime() - start);
     }
-    this.stats.incInt(containsValueForKeyCompletedId, numInc);
+    this.stats.incInt(containsValueForKeyCompletedId, 1);
   }
 
   @Override
@@ -1065,7 +873,7 @@ public class PartitionedRegionStatsImpl implements PartitionedRegionStats {
   public long startBucketCreate(boolean isRebalance) {
     this.stats.incInt(bucketCreatesInProgressId, 1);
     if (isRebalance) {
-      startRebalanceBucketCreate();
+      this.stats.incInt(rebalanceBucketCreatesInProgressId, 1);
     }
     return getStatTime();
   }
@@ -1157,10 +965,6 @@ public class PartitionedRegionStatsImpl implements PartitionedRegionStats {
   // rebalancing stats
   // ------------------------------------------------------------------------
 
-  private void startRebalanceBucketCreate() {
-    this.stats.incInt(rebalanceBucketCreatesInProgressId, 1);
-  }
-
   private void endRebalanceBucketCreate(long start, long end, boolean success) {
     this.stats.incInt(rebalanceBucketCreatesInProgressId, -1);
     if (CachePerfStats.enableClockStats) {
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/TimedMicrometerPartitionedRegionStats.kt b/geode-core/src/main/java/org/apache/geode/internal/cache/TimedMicrometerPartitionedRegionStats.kt
index b9018db..b9f4a70 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/TimedMicrometerPartitionedRegionStats.kt
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/TimedMicrometerPartitionedRegionStats.kt
@@ -1,11 +1,16 @@
 package org.apache.geode.internal.cache
 
+import io.micrometer.core.instrument.MeterRegistry
+import io.micrometer.core.instrument.Tag
 import io.micrometer.core.instrument.Timer
 import java.util.concurrent.TimeUnit
 
-class TimedMicrometerPartitionedRegionStats(regionName: String) : MicrometerPartitionRegionStats(regionName) {
+class TimedMicrometerPartitionedRegionStats(meterRegistry: MeterRegistry, regionName: String) : MicrometerPartitionRegionStats(meterRegistry,regionName) {
+
+    constructor(regionName:String) : this(MicroMeterRegistryFactory.getMeterRegistry(),regionName)
+
     private fun constructTimerForMetric(metricName: String): Timer =
-            metrics.timer("${metricName}Latency", regionName, PARTITIONED_REGION)
+            meterRegistry.timer("${metricName}Latency", listOf(Tag.of("region", regionName), Tag.of("regionType", PARTITIONED_REGION)))
 
     private val putTimer = constructTimerForMetric("put")
     private val putAllTimer = constructTimerForMetric("putAll")
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/partitioned/CreateBucketMessage.java b/geode-core/src/main/java/org/apache/geode/internal/cache/partitioned/CreateBucketMessage.java
index e134f94..31a80ff 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/partitioned/CreateBucketMessage.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/partitioned/CreateBucketMessage.java
@@ -144,7 +144,7 @@ public class CreateBucketMessage extends PartitionMessage {
     }
     r.checkReadiness();
     InternalDistributedMember primary = r.getRedundancyProvider().createBucketAtomically(bucketId,
-        bucketSize, startTime, false, partitionName);
+        bucketSize,  false, partitionName);
     r.getPrStats().endPartitionMessagesProcessing(startTime);
     CreateBucketReplyMessage.sendResponse(getSender(), getProcessorId(), dm, primary);
     return false;
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/partitioned/CreateMissingBucketsTask.java b/geode-core/src/main/java/org/apache/geode/internal/cache/partitioned/CreateMissingBucketsTask.java
index 2d1b3e8..bfe5fd9 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/partitioned/CreateMissingBucketsTask.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/partitioned/CreateMissingBucketsTask.java
@@ -55,8 +55,7 @@ public class CreateMissingBucketsTask extends RecoveryRunnable {
       if (parentRegion.getRegionAdvisor().getBucketAdvisor(i).getBucketRedundancy() != region
           .getRegionAdvisor().getBucketAdvisor(i).getBucketRedundancy()) {
         /* if (leaderRegion.getRegionAdvisor().isStorageAssignedForBucket(i)) { */
-        final long startTime = PartitionedRegionStats.startTime();
-        region.getRedundancyProvider().createBucketAtomically(i, 0, startTime, true, null);
+        region.getRedundancyProvider().createBucketAtomically(i, 0, true, null);
       }
     }
   }
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/partitioned/FetchEntryMessage.java b/geode-core/src/main/java/org/apache/geode/internal/cache/partitioned/FetchEntryMessage.java
index ea08ef2..ecdb891 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/partitioned/FetchEntryMessage.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/partitioned/FetchEntryMessage.java
@@ -128,9 +128,9 @@ public class FetchEntryMessage extends PartitionMessage {
         val = (EntrySnapshot) r.getDataView().getEntryOnRemote(keyInfo, r, true);
         r.getPrStats().endPartitionMessagesProcessing(startTime);
         FetchEntryReplyMessage.send(getSender(), getProcessorId(), val, dm, null);
-      } catch (TransactionException tex) {
+      } catch (TransactionException | PrimaryBucketException ex) {
         FetchEntryReplyMessage.send(getSender(), getProcessorId(), null, dm,
-            new ReplyException(tex));
+            new ReplyException(ex));
       } catch (PRLocallyDestroyedException pde) {
         FetchEntryReplyMessage.send(getSender(), getProcessorId(), null, dm,
             new ReplyException(new ForceReattemptException(
@@ -140,9 +140,6 @@ public class FetchEntryMessage extends PartitionMessage {
       } catch (EntryNotFoundException enfe) {
         FetchEntryReplyMessage.send(getSender(), getProcessorId(), null, dm, new ReplyException(
             LocalizedStrings.FetchEntryMessage_ENTRY_NOT_FOUND.toLocalizedString(), enfe));
-      } catch (PrimaryBucketException pbe) {
-        FetchEntryReplyMessage.send(getSender(), getProcessorId(), null, dm,
-            new ReplyException(pbe));
       } catch (ForceReattemptException pbe) {
         pbe.checkKey(key);
         // Slightly odd -- we're marshalling the retry to the peer on another host...
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/tx/PartitionedTXRegionStub.java b/geode-core/src/main/java/org/apache/geode/internal/cache/tx/PartitionedTXRegionStub.java
index c029175..ab90f30 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/tx/PartitionedTXRegionStub.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/tx/PartitionedTXRegionStub.java
@@ -394,7 +394,7 @@ public class PartitionedTXRegionStub extends AbstractPeerTXRegionStub {
     }
 
     PartitionedRegion pr = (PartitionedRegion) r;
-    final long startTime = PartitionedRegionStats.startTime();
+    final long startTime = pr.prStats.startTime();
     // build all the msgs by bucketid
     HashMap prMsgMap = putallO.createPRMessages();
     PutAllPartialResult partialKeys = new PutAllPartialResult(putallO.putAllDataSize);
@@ -454,7 +454,7 @@ public class PartitionedTXRegionStub extends AbstractPeerTXRegionStub {
     }
 
     PartitionedRegion pr = (PartitionedRegion) r;
-    final long startTime = PartitionedRegionStats.startTime();
+    final long startTime = pr.prStats.startTime();
     // build all the msgs by bucketid
     HashMap<Integer, RemoveAllPRMessage> prMsgMap = op.createPRMessages();
     PutAllPartialResult partialKeys = new PutAllPartialResult(op.removeAllDataSize);
diff --git a/geode-core/src/main/java/org/apache/geode/management/MemberMXBean.java b/geode-core/src/main/java/org/apache/geode/management/MemberMXBean.java
index a240106..e195a1e 100644
--- a/geode-core/src/main/java/org/apache/geode/management/MemberMXBean.java
+++ b/geode-core/src/main/java/org/apache/geode/management/MemberMXBean.java
@@ -180,12 +180,12 @@ public interface MemberMXBean {
   void shutDownMember();
 
   /**
-   * Returns JVM metrics.
+   * Returns JVM meterRegistry.
    */
   JVMMetrics showJVMMetrics();
 
   /**
-   * Returns operating system metrics.
+   * Returns operating system meterRegistry.
    */
   OSMetrics showOSMetrics();
 
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/beans/MemberMBeanBridge.java b/geode-core/src/main/java/org/apache/geode/management/internal/beans/MemberMBeanBridge.java
index e010fc5..46446a5 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/beans/MemberMBeanBridge.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/beans/MemberMBeanBridge.java
@@ -726,7 +726,7 @@ public class MemberMBeanBridge {
   }
 
   /**
-   * @return Some basic JVM metrics at the particular instance
+   * @return Some basic JVM meterRegistry at the particular instance
    */
   public JVMMetrics fetchJVMMetrics() {
     long gcCount = getGCStatistic(StatsKey.VM_GC_STATS_COLLECTIONS).longValue();
@@ -745,11 +745,11 @@ public class MemberMBeanBridge {
   }
 
   /**
-   * All OS metrics are not present in java.lang.management.OperatingSystemMXBean It has to be cast
+   * All OS meterRegistry are not present in java.lang.management.OperatingSystemMXBean It has to be cast
    * to com.sun.management.OperatingSystemMXBean. To avoid the cast using dynamic call so that Java
    * platform will take care of the details in a native manner;
    *
-   * @return Some basic OS metrics at the particular instance
+   * @return Some basic OS meterRegistry at the particular instance
    */
   public OSMetrics fetchOSMetrics() {
     OSMetrics metrics = null;
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/ShowMetricsCommand.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/ShowMetricsCommand.java
index e1d35b9..d98b301 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/ShowMetricsCommand.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/ShowMetricsCommand.java
@@ -136,10 +136,10 @@ public class ShowMetricsCommand extends GfshCommand {
   }
 
   /**
-   * Gets the system wide metrics
+   * Gets the system wide meterRegistry
    *
    * @return ResultData with required System wide statistics or ErrorResultData if DS MBean is not
-   *         found to gather metrics
+   *         found to gather meterRegistry
    */
   private ResultData getSystemWideMetrics(String export_to_report_to, String[] categoriesArr,
       StringBuilder csvBuilder) {
@@ -164,17 +164,17 @@ public class ShowMetricsCommand extends GfshCommand {
     writeSystemWideMetricValues(dsMxBean, csvBuilder, metricsTable, categoriesToDisplay);
     if (StringUtils.isNotEmpty(export_to_report_to)) {
       crd.addAsFile(export_to_report_to, csvBuilder.toString(),
-          "Cluster wide metrics exported to {0}.", false);
+          "Cluster wide meterRegistry exported to {0}.", false);
     }
 
     return crd;
   }
 
   /**
-   * Gets the Cluster wide metrics for a given member
+   * Gets the Cluster wide meterRegistry for a given member
    *
    * @return ResultData with required Member statistics or ErrorResultData if MemberMbean is not
-   *         found to gather metrics
+   *         found to gather meterRegistry
    * @throws ResultDataException if building result fails
    */
   private ResultData getMemberMetrics(DistributedMember distributedMember,
@@ -228,14 +228,14 @@ public class ShowMetricsCommand extends GfshCommand {
 
     if (StringUtils.isNotEmpty(export_to_report_to)) {
       crd.addAsFile(export_to_report_to, csvBuilder != null ? csvBuilder.toString() : null,
-          "Member metrics exported to {0}.", false);
+          "Member meterRegistry exported to {0}.", false);
     }
     return crd;
 
   }
 
   /**
-   * Gets the Cluster-wide metrics for a region
+   * Gets the Cluster-wide meterRegistry for a region
    *
    * @return ResultData containing the table
    * @throws ResultDataException if building result fails
@@ -275,10 +275,10 @@ public class ShowMetricsCommand extends GfshCommand {
   }
 
   /**
-   * Gets the metrics of region on a given member
+   * Gets the meterRegistry of region on a given member
    *
    * @return ResultData with required Region statistics or ErrorResultData if Region MBean is not
-   *         found to gather metrics
+   *         found to gather meterRegistry
    * @throws ResultDataException if building result fails
    */
   private ResultData getRegionMetricsFromMember(String regionName,
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/i18n/CliStrings.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/i18n/CliStrings.java
index fe8650c..139a19c 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/i18n/CliStrings.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/i18n/CliStrings.java
@@ -2034,15 +2034,15 @@ public class CliStrings {
   public static final String SHOW_LOG_MSG_INVALID_NUMBER = "Invalid number";
   public static final String SHOW_LOG_ERROR = "Error";
 
-  /* show metrics */
-  public static final String SHOW_METRICS = "show metrics";
+  /* show meterRegistry */
+  public static final String SHOW_METRICS = "show meterRegistry";
   public static final String SHOW_METRICS__HELP =
-      "Display or export metrics for the entire distributed system, a member or a region.";
+      "Display or export meterRegistry for the entire distributed system, a member or a region.";
   public static final String SHOW_METRICS__REGION = "region";
   public static final String SHOW_METRICS__REGION__HELP =
-      "Name/Path of the region whose metrics will be displayed/exported.";
+      "Name/Path of the region whose meterRegistry will be displayed/exported.";
   public static final String SHOW_METRICS__MEMBER__HELP =
-      "Name/Id of the member whose metrics will be displayed/exported.";
+      "Name/Id of the member whose meterRegistry will be displayed/exported.";
   public static final String SHOW_METRICS__CATEGORY = "categories";
   public static final String SHOW_METRICS__CATEGORY__HELP =
       "Categories available based upon the parameters specified are:\n"
@@ -2053,14 +2053,14 @@ public class CliStrings {
           + "- member and region specified: callback, diskstore, eviction, partition, region";
   public static final String SHOW_METRICS__FILE = "file";
   public static final String SHOW_METRICS__FILE__HELP =
-      "Name of the file to which metrics will be written.";
-  public static final String SHOW_METRICS__ERROR = "Unable to retrieve metrics : {0} ";
+      "Name of the file to which meterRegistry will be written.";
+  public static final String SHOW_METRICS__ERROR = "Unable to retrieve meterRegistry : {0} ";
   public static final String SHOW_METRICS__TYPE__HEADER = "Category";
   public static final String SHOW_METRICS__METRIC__HEADER = "Metric";
   public static final String SHOW_METRICS__VALUE__HEADER = "Value";
   public static final String SHOW_METRICS__CACHESERVER__PORT = "port";
   public static final String SHOW_METRICS__CACHESERVER__PORT__HELP =
-      "Port number of the Cache Server whose metrics are to be displayed/exported. This can only be used along with the --member parameter.";
+      "Port number of the Cache Server whose meterRegistry are to be displayed/exported. This can only be used along with the --member parameter.";
   public static final String SHOW_METRICS__CANNOT__USE__CACHESERVERPORT =
       "If the --port parameter is specified, then the --member parameter must also be specified.";
   public static final String SHOW_METRICS__CANNOT__USE__REGION__WITH__CACHESERVERPORT =
diff --git a/geode-core/src/test/java/org/apache/geode/management/OffHeapManagementDUnitTest.java b/geode-core/src/test/java/org/apache/geode/management/OffHeapManagementDUnitTest.java
index d50ee3e..a4ef96d 100644
--- a/geode-core/src/test/java/org/apache/geode/management/OffHeapManagementDUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/management/OffHeapManagementDUnitTest.java
@@ -199,7 +199,7 @@ public class OffHeapManagementDUnitTest extends CacheTestCase {
     assertOffHeapMetricsOnVm(vm, TOTAL_MEMORY, 0, 0, 0);
 
     /*
-     * Perform ops on the off-heap region and assert that the off-heap metrics correctly reflect the
+     * Perform ops on the off-heap region and assert that the off-heap meterRegistry correctly reflect the
      * ops
      */
     doPutOnVm(vm, KEY, VALUE, OFF_HEAP_REGION_NAME, false);
diff --git a/geode-core/src/test/java/org/apache/geode/management/bean/stats/MemberLevelStatsJUnitTest.java b/geode-core/src/test/java/org/apache/geode/management/bean/stats/MemberLevelStatsJUnitTest.java
index e430b49..98a75a1 100644
--- a/geode-core/src/test/java/org/apache/geode/management/bean/stats/MemberLevelStatsJUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/management/bean/stats/MemberLevelStatsJUnitTest.java
@@ -21,6 +21,7 @@ import java.util.ArrayList;
 import java.util.List;
 import java.util.concurrent.TimeUnit;
 
+import org.apache.geode.internal.cache.*;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
@@ -30,10 +31,6 @@ import org.apache.geode.distributed.internal.DistributionStats;
 import org.apache.geode.distributed.internal.locks.DLockStats;
 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.stats50.VMStats50;
@@ -94,7 +91,7 @@ public class MemberLevelStatsJUnitTest extends MBeanStatsTestCase {
     }
 
     for (int i = 0; i < 4; i++) {
-      PartitionedRegionStats stats = new PartitionedRegionStats(system, name.getMethodName() + i);
+      PartitionedRegionStats stats = new PartitionedRegionStatsImpl(system, name.getMethodName() + i);
       parRegionStatsList.add(stats);
       bridge.addPartionRegionStats(stats);
     }
diff --git a/geode-core/src/test/java/org/apache/geode/management/bean/stats/RegionStatsJUnitTest.java b/geode-core/src/test/java/org/apache/geode/management/bean/stats/RegionStatsJUnitTest.java
index dd9e19c..db7dce2 100644
--- a/geode-core/src/test/java/org/apache/geode/management/bean/stats/RegionStatsJUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/management/bean/stats/RegionStatsJUnitTest.java
@@ -17,6 +17,7 @@ package org.apache.geode.management.bean.stats;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 
+import org.apache.geode.internal.cache.PartitionedRegionStatsImpl;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
@@ -49,7 +50,7 @@ public class RegionStatsJUnitTest extends MBeanStatsTestCase {
 
   protected void init() {
     cachePerfStats = new CachePerfStats(system);
-    partitionedRegionStats = new PartitionedRegionStats(system, "/tests");
+    partitionedRegionStats = new PartitionedRegionStatsImpl(system, "/tests");
     diskRegionStats = new DiskRegionStats(system, "test-disk");
 
     bridge = new RegionMBeanBridge(cachePerfStats);
diff --git a/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/ShowMetricsCommandIntegrationTest.java b/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/ShowMetricsCommandIntegrationTest.java
index 7bbad90..d0139d8 100644
--- a/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/ShowMetricsCommandIntegrationTest.java
+++ b/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/ShowMetricsCommandIntegrationTest.java
@@ -69,14 +69,14 @@ public class ShowMetricsCommandIntegrationTest {
   @Test
   public void commandFailsWhenNotConnected() throws Exception {
     gfsh.disconnect();
-    gfsh.executeAndAssertThat("show metrics")
+    gfsh.executeAndAssertThat("show meterRegistry")
         .containsOutput("was found but is not currently available");
   }
 
   @Test
   public void getRegionMetricsShowsExactlyDefaultCategories() throws Exception {
     // Use --region and --member to get RegionMetricsFromMember
-    String cmd = "show metrics --region=/" + REGION_NAME + " --member=" + MEMBER_NAME;
+    String cmd = "show meterRegistry --region=/" + REGION_NAME + " --member=" + MEMBER_NAME;
     List<String> expectedCategories =
         ShowMetricsInterceptor.getValidCategoriesAsStrings(true, true, false);
     // Blank lines are permitted for grouping.
@@ -89,7 +89,7 @@ public class ShowMetricsCommandIntegrationTest {
   @Test
   public void getSystemRegionMetricsShowsExactlyDefaultCategories() throws Exception {
     // Use --region alone to get SystemRegionMetrics
-    String cmd = "show metrics --region=/" + REGION_NAME;
+    String cmd = "show meterRegistry --region=/" + REGION_NAME;
     List<String> expectedCategories =
         ShowMetricsInterceptor.getValidCategoriesAsStrings(true, false, false);
     // Blank lines are permitted for grouping.
@@ -102,8 +102,8 @@ public class ShowMetricsCommandIntegrationTest {
 
   @Test
   public void getMemberMetricsShowsExactlyDefaultCategories() throws Exception {
-    // Use --member to get member metrics
-    String cmd = "show metrics --member=" + MEMBER_NAME;
+    // Use --member to get member meterRegistry
+    String cmd = "show meterRegistry --member=" + MEMBER_NAME;
     List<String> expectedCategories =
         ShowMetricsInterceptor.getValidCategoriesAsStrings(false, true, false);
     // Blank lines are permitted for grouping.
@@ -116,8 +116,8 @@ public class ShowMetricsCommandIntegrationTest {
 
   @Test
   public void getMemberWithPortMetricsShowsExactlyDefaultCategories() throws Exception {
-    // Use --member and --port to get member metrics with port info
-    String cmd = "show metrics --member=" + MEMBER_NAME + " --port=" + server.getPort();
+    // Use --member and --port to get member meterRegistry with port info
+    String cmd = "show meterRegistry --member=" + MEMBER_NAME + " --port=" + server.getPort();
     List<String> expectedCategories =
         ShowMetricsInterceptor.getValidCategoriesAsStrings(false, true, true);
     // Blank lines are permitted for grouping.
@@ -130,8 +130,8 @@ public class ShowMetricsCommandIntegrationTest {
 
   @Test
   public void getSystemMetricsShowsExactlyDefaultCategories() throws Exception {
-    // No specified options yield system-wide metrics
-    String cmd = "show metrics";
+    // No specified options yield system-wide meterRegistry
+    String cmd = "show meterRegistry";
     List<String> expectedCategories =
         ShowMetricsInterceptor.getValidCategoriesAsStrings(false, false, false);
     // Blank lines are permitted for grouping.
@@ -145,7 +145,7 @@ public class ShowMetricsCommandIntegrationTest {
   @Test
   public void invalidCategoryGetsReported() throws Exception {
     String cmd =
-        "show metrics --categories=\"cluster,cache,some_invalid_category,another_invalid_category\"";
+        "show meterRegistry --categories=\"cluster,cache,some_invalid_category,another_invalid_category\"";
 
     gfsh.executeAndAssertThat(cmd).containsOutput("Invalid Categories")
         .containsOutput("some_invalid_category").containsOutput("another_invalid_category")
@@ -154,7 +154,7 @@ public class ShowMetricsCommandIntegrationTest {
 
   @Test
   public void categoryOptionAbridgesOutput() throws Exception {
-    String cmd = "show metrics --categories=\"cluster,cache\"";
+    String cmd = "show meterRegistry --categories=\"cluster,cache\"";
     List<String> expectedCategories = Arrays.asList("cluster", "cache", "");
     logger.info("Expecting categories: " + String.join(", ", expectedCategories));
 
@@ -166,7 +166,7 @@ public class ShowMetricsCommandIntegrationTest {
   public void getRegionMetricsForPartitionedRegionWithStatistics() throws Exception {
     String cmd = "create region --name=region2 --type=PARTITION --enable-statistics";
     gfsh.executeAndAssertThat(cmd).statusIsSuccess();
-    String cmd2 = "show metrics --member=" + MEMBER_NAME + " --region=region2";
+    String cmd2 = "show meterRegistry --member=" + MEMBER_NAME + " --region=region2";
     gfsh.executeAndAssertThat(cmd2).statusIsSuccess().tableHasRowWithValues("Category", "Metric",
         "Value", "", "missCount", "-1");
   }
diff --git a/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/ShowMetricsDUnitTest.java b/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/ShowMetricsDUnitTest.java
index 82bd123..75c7baa 100644
--- a/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/ShowMetricsDUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/ShowMetricsDUnitTest.java
@@ -118,19 +118,19 @@ public class ShowMetricsDUnitTest {
 
   @Test
   public void testShowMetricsDefault() throws Exception {
-    gfsh.executeAndAssertThat("show metrics").statusIsSuccess();
+    gfsh.executeAndAssertThat("show meterRegistry").statusIsSuccess();
   }
 
   @Test
   public void testShowMetricsRegion() throws Exception {
-    gfsh.executeAndAssertThat("show metrics --region=REGION1").statusIsSuccess();
+    gfsh.executeAndAssertThat("show meterRegistry --region=REGION1").statusIsSuccess();
     assertThat(gfsh.getGfshOutput()).contains("Cluster-wide Region Metrics");
   }
 
   @Test
   public void testShowMetricsMember() throws Exception {
     gfsh.executeAndAssertThat(
-        "show metrics --member=" + server.getName() + " --port=" + server.getPort())
+        "show meterRegistry --member=" + server.getName() + " --port=" + server.getPort())
         .statusIsSuccess().containsOutput("Member Metrics").containsOutput("cacheserver");
   }
 
@@ -139,17 +139,17 @@ public class ShowMetricsDUnitTest {
     File output = tempFolder.newFile("memberMetricReport.csv");
     output.delete();
 
-    gfsh.executeAndAssertThat("show metrics --member=" + server.getName() + " --port="
+    gfsh.executeAndAssertThat("show meterRegistry --member=" + server.getName() + " --port="
         + server.getPort() + " --file=" + output.getAbsolutePath()).statusIsSuccess()
         .containsOutput("Member Metrics").containsOutput("cacheserver")
-        .containsOutput("Member metrics exported to " + output.getAbsolutePath());
+        .containsOutput("Member meterRegistry exported to " + output.getAbsolutePath());
 
     assertThat(output).exists();
   }
 
   @Test
   public void testShowMetricsRegionFromMember() throws Exception {
-    gfsh.executeAndAssertThat("show metrics --member=" + server.getName() + " --region=REGION1")
+    gfsh.executeAndAssertThat("show meterRegistry --member=" + server.getName() + " --region=REGION1")
         .statusIsSuccess();
     assertThat(gfsh.getGfshOutput()).contains("Metrics for region:/REGION1 On Member server-1");
   }
diff --git a/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/ShowMetricsJUnitTest.java b/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/ShowMetricsJUnitTest.java
index e6f15e4..b625b1d 100644
--- a/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/ShowMetricsJUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/ShowMetricsJUnitTest.java
@@ -36,7 +36,7 @@ public class ShowMetricsJUnitTest {
   public void testPortAndRegion() throws Exception {
     ShowMetricsCommand command = spy(ShowMetricsCommand.class);
     CommandResult result =
-        parser.executeCommandWithInstance(command, "show metrics --port=0 --region=regionA");
+        parser.executeCommandWithInstance(command, "show meterRegistry --port=0 --region=regionA");
     assertThat(result.getStatus()).isEqualTo(Result.Status.ERROR);
     assertThat(result.getContent().toString())
         .contains("The --region and --port parameters are mutually exclusive");
@@ -45,7 +45,7 @@ public class ShowMetricsJUnitTest {
   @Test
   public void testPortOnly() throws Exception {
     ShowMetricsCommand command = spy(ShowMetricsCommand.class);
-    CommandResult result = parser.executeCommandWithInstance(command, "show metrics --port=0");
+    CommandResult result = parser.executeCommandWithInstance(command, "show meterRegistry --port=0");
     assertThat(result.getStatus()).isEqualTo(Result.Status.ERROR);
     assertThat(result.getContent().toString()).contains(
         "If the --port parameter is specified, then the --member parameter must also be specified.");
@@ -54,7 +54,7 @@ public class ShowMetricsJUnitTest {
   @Test
   public void invalidPortNumber() throws Exception {
     ShowMetricsCommand command = spy(ShowMetricsCommand.class);
-    CommandResult result = parser.executeCommandWithInstance(command, "show metrics --port=abc");
+    CommandResult result = parser.executeCommandWithInstance(command, "show meterRegistry --port=abc");
     assertThat(result.getStatus()).isEqualTo(Result.Status.ERROR);
     // When relying on Spring's converters, any command that does not parse is "Invalid"
     assertThat(result.getContent().toString()).contains("Invalid command");
diff --git a/geode-core/src/test/java/org/apache/geode/management/internal/security/TestCommand.java b/geode-core/src/test/java/org/apache/geode/management/internal/security/TestCommand.java
index c5705c2..9a18ed6 100644
--- a/geode-core/src/test/java/org/apache/geode/management/internal/security/TestCommand.java
+++ b/geode-core/src/test/java/org/apache/geode/management/internal/security/TestCommand.java
@@ -224,7 +224,7 @@ public class TestCommand {
     createTestCommand("netstat --member=server1", ResourcePermissions.CLUSTER_READ);
     createTestCommand("show dead-locks --file=deadlocks.txt", ResourcePermissions.CLUSTER_READ);
     createTestCommand("show log --member=locator1 --lines=5", ResourcePermissions.CLUSTER_READ);
-    createTestCommand("show metrics", ResourcePermissions.CLUSTER_READ);
+    createTestCommand("show meterRegistry", ResourcePermissions.CLUSTER_READ);
 
 
     // PDX Commands
diff --git a/geode-core/src/test/resources/org/apache/geode/management/internal/cli/commands/golden-help-offline.properties b/geode-core/src/test/resources/org/apache/geode/management/internal/cli/commands/golden-help-offline.properties
index 8199dbf..dfc685e 100644
--- a/geode-core/src/test/resources/org/apache/geode/management/internal/cli/commands/golden-help-offline.properties
+++ b/geode-core/src/test/resources/org/apache/geode/management/internal/cli/commands/golden-help-offline.properties
@@ -2196,7 +2196,7 @@ PARAMETERS\n\
 \ \ \ \ \ \ \ \ Default (if the parameter is specified without value): true\n\
 \ \ \ \ \ \ \ \ Default (if the parameter is not specified): false\n\
 \n\
-Other commands starting with "sh" are: show dead-locks, show log, show metrics, show missing-disk-stores, show subscription-queue-size, shutdown\n\
+Other commands starting with "sh" are: show dead-locks, show log, show meterRegistry, show missing-disk-stores, show subscription-queue-size, shutdown\n\
 
 show-dead-locks.help=\
 NAME\n\
@@ -2230,28 +2230,28 @@ PARAMETERS\n\
 \ \ \ \ \ \ \ \ Required: false\n\
 \ \ \ \ \ \ \ \ Default (if the parameter is not specified): 0\n\
 
-show-metrics.help=\
+show-meterRegistry.help=\
 NAME\n\
-\ \ \ \ show metrics\n\
+\ \ \ \ show meterRegistry\n\
 IS AVAILABLE\n\
 \ \ \ \ false\n\
 SYNOPSIS\n\
-\ \ \ \ Display or export metrics for the entire distributed system, a member or a region.\n\
+\ \ \ \ Display or export meterRegistry for the entire distributed system, a member or a region.\n\
 SYNTAX\n\
-\ \ \ \ show metrics [--member=value] [--region=value] [--file=value] [--port=value]\n\
+\ \ \ \ show meterRegistry [--member=value] [--region=value] [--file=value] [--port=value]\n\
 \ \ \ \ [--categories=value(,value)*]\n\
 PARAMETERS\n\
 \ \ \ \ member\n\
-\ \ \ \ \ \ \ \ Name/Id of the member whose metrics will be displayed/exported.\n\
+\ \ \ \ \ \ \ \ Name/Id of the member whose meterRegistry will be displayed/exported.\n\
 \ \ \ \ \ \ \ \ Required: false\n\
 \ \ \ \ region\n\
-\ \ \ \ \ \ \ \ Name/Path of the region whose metrics will be displayed/exported.\n\
+\ \ \ \ \ \ \ \ Name/Path of the region whose meterRegistry will be displayed/exported.\n\
 \ \ \ \ \ \ \ \ Required: false\n\
 \ \ \ \ file\n\
-\ \ \ \ \ \ \ \ Name of the file to which metrics will be written.\n\
+\ \ \ \ \ \ \ \ Name of the file to which meterRegistry will be written.\n\
 \ \ \ \ \ \ \ \ Required: false\n\
 \ \ \ \ port\n\
-\ \ \ \ \ \ \ \ Port number of the Cache Server whose metrics are to be displayed/exported. This can only\n\
+\ \ \ \ \ \ \ \ Port number of the Cache Server whose meterRegistry are to be displayed/exported. This can only\n\
 \ \ \ \ \ \ \ \ be used along with the --member parameter.\n\
 \ \ \ \ \ \ \ \ Required: false\n\
 \ \ \ \ categories\n\
diff --git a/geode-core/src/test/resources/org/apache/geode/management/internal/cli/commands/golden-help-online.properties b/geode-core/src/test/resources/org/apache/geode/management/internal/cli/commands/golden-help-online.properties
index 2b22650..14763f4 100644
--- a/geode-core/src/test/resources/org/apache/geode/management/internal/cli/commands/golden-help-online.properties
+++ b/geode-core/src/test/resources/org/apache/geode/management/internal/cli/commands/golden-help-online.properties
@@ -506,9 +506,9 @@ NAME\n\
 IS AVAILABLE\n\
 \ \ \ \ true\n\
 
-show-metrics.help=\
+show-meterRegistry.help=\
 NAME\n\
-\ \ \ \ show metrics\n\
+\ \ \ \ show meterRegistry\n\
 IS AVAILABLE\n\
 \ \ \ \ true\n\
 
diff --git a/geode-protobuf/src/main/java/org/apache/geode/internal/protocol/protobuf/statistics/MicrometerClientStatsImpl.kt b/geode-protobuf/src/main/java/org/apache/geode/internal/protocol/protobuf/statistics/MicrometerClientStatsImpl.kt
index bcf2ff1..2192627 100644
--- a/geode-protobuf/src/main/java/org/apache/geode/internal/protocol/protobuf/statistics/MicrometerClientStatsImpl.kt
+++ b/geode-protobuf/src/main/java/org/apache/geode/internal/protocol/protobuf/statistics/MicrometerClientStatsImpl.kt
@@ -7,43 +7,21 @@ import io.micrometer.core.instrument.composite.CompositeMeterRegistry
 import io.micrometer.influx.InfluxConfig
 import io.micrometer.influx.InfluxMeterRegistry
 import io.micrometer.jmx.JmxMeterRegistry
+import org.apache.geode.internal.cache.MicroMeterRegistryFactory
 import org.apache.geode.internal.protocol.statistics.ProtocolClientStatistics
 import java.time.Duration
 import java.util.concurrent.atomic.AtomicInteger
 
-class MicrometerClientStatsImpl : ProtocolClientStatistics {
+class MicrometerClientStatsImpl(val meterRegistry: MeterRegistry) : ProtocolClientStatistics {
+    constructor() : this(MicroMeterRegistryFactory.getMeterRegistry())
 
     private val clientsConnected = AtomicInteger(0)
 
-    private val influxMetrics: MeterRegistry = InfluxMeterRegistry(object : InfluxConfig {
-        override fun step(): Duration = Duration.ofSeconds(1)
-        override fun db(): String = "mydb"
-        override fun get(k: String): String? = null
-        override fun uri(): String = "http://localhost:8086"
-    }, Clock.SYSTEM)
-
-//    private val atlasMetrics: MeterRegistry = AtlasMeterRegistry(object : AtlasConfig {
-//        override fun get(k: String?): String? = null
-//        override fun enabled(): Boolean = true
-//        override fun uri(): String = "http://localhost:7101/api/v1/publish"
-//        override fun step(): Duration = Duration.ofSeconds(10)
-//    }, Clock.SYSTEM)
-
-    private val jmxMetrics: MeterRegistry = JmxMeterRegistry()
-
-    private val metrics = CompositeMeterRegistry(Clock.SYSTEM)
-
-    init {
-        metrics.add(influxMetrics)
-//        metrics.add(atlasMetrics)
-        metrics.add(jmxMetrics)
-    }
-
-    val clientConnectedCounter = metrics.gauge("clientConnected", clientsConnected)
-    val messageReceivedCounter = metrics.summary("messageReceived")
-    val messageSentCounter = metrics.summary("messageSent")
-    val authorizationViolationsCounter = metrics.counter("authorizationViolations")
-    val authenticationFailureCounter = metrics.counter("authenticationFailures")
+    val clientConnectedCounter = meterRegistry.gauge("clientConnected", clientsConnected)
+    val messageReceivedCounter = meterRegistry.summary("messageReceived")
+    val messageSentCounter = meterRegistry.summary("messageSent")
+    val authorizationViolationsCounter = meterRegistry.counter("authorizationViolations")
+    val authenticationFailureCounter = meterRegistry.counter("authenticationFailures")
 
     override fun clientConnected() {
         clientsConnected.incrementAndGet()

-- 
To stop receiving notification emails like this one, please contact
udo@apache.org.