You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@geode.apache.org by dh...@apache.org on 2019/07/22 22:09:20 UTC

[geode] 07/18: Make regionEntriesGaugeShowsCountOfReplicateRegionValuesInServer pass

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

dhemery pushed a commit to branch GEODE-7001-region-entry-count
in repository https://gitbox.apache.org/repos/asf/geode.git

commit 93a6c1040c84418f424ffccbb82f35bf19d6705c
Author: Kirk Lund <kl...@apache.org>
AuthorDate: Wed Jul 17 09:05:42 2019 -0700

    Make regionEntriesGaugeShowsCountOfReplicateRegionValuesInServer pass
---
 .../geode/metrics/RegionEntriesGaugeTest.java      |  26 ++---
 .../geode/internal/cache/CachePerfStats.java       |   2 +-
 .../geode/internal/cache/DummyCachePerfStats.java  |  15 +--
 .../apache/geode/internal/cache/LocalRegion.java   |   2 +-
 .../internal/cache/PartitionedRegionDataStore.java |   3 +-
 .../geode/internal/cache/RegionPerfStats.java      |  26 ++++-
 .../geode/internal/statistics/StatisticsImpl.java  |   4 +-
 .../internal/statistics/StatisticsTypeImpl.java    |   2 +-
 .../statistics/ValidatingStatisticsType.java       |  24 +++++
 .../statistics/meters/LegacyStatGauge.java         |  71 --------------
 .../geode/internal/cache/RegionPerfStatsTest.java  | 108 ++++++++++++++++++---
 .../internal/statistics/SimpleStatistics.java      |  70 +++++++++++++
 12 files changed, 236 insertions(+), 117 deletions(-)

diff --git a/geode-assembly/src/acceptanceTest/java/org/apache/geode/metrics/RegionEntriesGaugeTest.java b/geode-assembly/src/acceptanceTest/java/org/apache/geode/metrics/RegionEntriesGaugeTest.java
index dfd0f67..0d531ec 100644
--- a/geode-assembly/src/acceptanceTest/java/org/apache/geode/metrics/RegionEntriesGaugeTest.java
+++ b/geode-assembly/src/acceptanceTest/java/org/apache/geode/metrics/RegionEntriesGaugeTest.java
@@ -19,10 +19,12 @@ import static java.util.Arrays.asList;
 import static org.apache.geode.cache.RegionShortcut.PARTITION;
 import static org.apache.geode.cache.RegionShortcut.REPLICATE;
 import static org.apache.geode.cache.client.ClientRegionShortcut.PROXY;
+import static org.apache.geode.test.awaitility.GeodeAwaitility.await;
 import static org.assertj.core.api.Assertions.assertThat;
 
 import java.io.File;
 import java.util.List;
+import java.util.concurrent.TimeUnit;
 
 import io.micrometer.core.instrument.Gauge;
 import org.junit.After;
@@ -143,8 +145,8 @@ public class RegionEntriesGaugeTest {
   }
 
   @Test
-  public void regionEntriesGaugeShowsCountOfPartitionedRegionValuesInServer() {
-    createRegion(PARTITION.name(), "partitionedRegion");
+  public void regionEntriesGaugeShowsCountOfReplicateRegionValuesInServer() {
+    createRegion(REPLICATE.name(), "replicateRegion");
 
     List<String> keys = asList("a", "b", "c", "d", "e", "f", "g", "h");
     for (String key : keys) {
@@ -161,19 +163,19 @@ public class RegionEntriesGaugeTest {
     String executeFunctionCommand =
         "execute function --id=" + GetMemberRegionEntriesGaugeFunction.ID;
 
-    // await().untilAsserted(() -> {
-    String output =
-        gfshRule.execute(connectToLocatorCommand, executeFunctionCommand).getOutputText();
+    await().atMost(1, TimeUnit.MINUTES).untilAsserted(() -> {
+      String output =
+          gfshRule.execute(connectToLocatorCommand, executeFunctionCommand).getOutputText();
 
-    assertThat(output.trim())
-        .as("Returned gauge of region entries.")
-        .endsWith("[" + keys.size() + ".0]");
-    // });
+      assertThat(output.trim())
+          .as("Returned gauge of region entries.")
+          .endsWith("[" + expectedNumberOfEntries + ".0]");
+    });
   }
 
   @Test
-  public void regionEntriesGaugeShowsCountOfReplicateRegionValuesInServer() {
-    createRegion(REPLICATE.name(), "replicateRegion");
+  public void regionEntriesGaugeShowsCountOfPartitionedRegionValuesInServer() {
+    createRegion(PARTITION.name(), "partitionedRegion");
 
     List<String> keys = asList("a", "b", "c", "d", "e", "f", "g", "h");
     for (String key : keys) {
@@ -196,7 +198,7 @@ public class RegionEntriesGaugeTest {
 
     assertThat(output.trim())
         .as("Returned gauge of region entries.")
-        .endsWith("[" + keys.size() + ".0]");
+        .endsWith("[" + expectedNumberOfEntries + ".0]");
     // });
   }
 
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/CachePerfStats.java b/geode-core/src/main/java/org/apache/geode/internal/cache/CachePerfStats.java
index c9d9f47..aedbd81 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/CachePerfStats.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/CachePerfStats.java
@@ -1273,7 +1273,7 @@ public class CachePerfStats {
    *
    * @since GemFire 3.5
    */
-  void close() {
+  protected void close() {
     stats.close();
   }
 
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/DummyCachePerfStats.java b/geode-core/src/main/java/org/apache/geode/internal/cache/DummyCachePerfStats.java
index 9a1505b..1a574e5 100755
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/DummyCachePerfStats.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/DummyCachePerfStats.java
@@ -22,15 +22,6 @@ import org.apache.geode.distributed.internal.PoolStatHelper;
  */
 public class DummyCachePerfStats extends CachePerfStats {
 
-  // ////////////////////// Constructors ////////////////////////
-
-  /**
-   * Creates a new <code>DummyCachePerfStats</code>
-   */
-  public DummyCachePerfStats() {}
-
-  // //////////////////// Accessing Stats //////////////////////
-
   @Override
   public int getLoadsCompleted() {
     return 0;
@@ -328,10 +319,10 @@ public class DummyCachePerfStats extends CachePerfStats {
   @Override
   public void txRollback(long opTime, long txLifeTime, int txChanges) {}
 
-  // //// Special Instance Methods /////
-
   @Override
-  void close() {}
+  protected void close() {
+    // nothing
+  }
 
   @Override
   public boolean isClosed() {
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/LocalRegion.java b/geode-core/src/main/java/org/apache/geode/internal/cache/LocalRegion.java
index 548bbe8..992dab3 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/LocalRegion.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/LocalRegion.java
@@ -599,7 +599,7 @@ public class LocalRegion extends AbstractRegion implements LoaderHelperFactory,
         hasOwnStats = true;
         cachePerfStats = new RegionPerfStats(
             cache.getInternalDistributedSystem().getStatisticsManager(), cache.getCachePerfStats(),
-            regionName);
+            regionName, cache.getMeterRegistry());
       }
     }
 
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 8c73c41..c0a14c0 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
@@ -204,7 +204,8 @@ public class PartitionedRegionDataStore implements HasCachePerfStats {
     // this.bucketStats = new CachePerfStats(pr.getSystem(), "partition-" + pr.getName());
     this.bucketStats =
         new RegionPerfStats(pr.getCache().getInternalDistributedSystem().getStatisticsManager(),
-            pr.getCachePerfStats(), "partition-" + pr.getName());
+            pr.getCachePerfStats(), "partition-" + pr.getName(),
+            pr.getCache().getMeterRegistry());
     this.keysOfInterest = new ConcurrentHashMap();
   }
 
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/RegionPerfStats.java b/geode-core/src/main/java/org/apache/geode/internal/cache/RegionPerfStats.java
index 2956d78..d36f012 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/RegionPerfStats.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/RegionPerfStats.java
@@ -16,6 +16,9 @@ package org.apache.geode.internal.cache;
 
 import java.util.function.LongSupplier;
 
+import io.micrometer.core.instrument.Gauge;
+import io.micrometer.core.instrument.MeterRegistry;
+
 import org.apache.geode.StatisticsFactory;
 import org.apache.geode.annotations.VisibleForTesting;
 import org.apache.geode.internal.NanoTimer;
@@ -23,21 +26,36 @@ import org.apache.geode.internal.NanoTimer;
 class RegionPerfStats extends CachePerfStats {
 
   private final CachePerfStats cachePerfStats;
-  // private final Gauge entriesGauge;
+  private final MeterRegistry meterRegistry;
+  private final Gauge entriesGauge;
 
   RegionPerfStats(StatisticsFactory statisticsFactory, CachePerfStats cachePerfStats,
-      String regionName) {
-    this(statisticsFactory, cachePerfStats, regionName,
+      String regionName, MeterRegistry meterRegistry) {
+    this(statisticsFactory, cachePerfStats, regionName, meterRegistry,
         enableClockStats ? NanoTimer::getTime : () -> 0);
   }
 
   @VisibleForTesting
   RegionPerfStats(StatisticsFactory statisticsFactory, CachePerfStats cachePerfStats,
-      String regionName, LongSupplier clock) {
+      String regionName, MeterRegistry meterRegistry, LongSupplier clock) {
     super(statisticsFactory, "RegionStats-" + regionName, clock);
     this.cachePerfStats = cachePerfStats;
+    this.meterRegistry = meterRegistry;
 
+    entriesGauge = Gauge.builder("member.region.entries", stats,
+        value -> {
+          long longValue = value.getLong(entryGaugeId);
+          return (double) longValue;
+        })
+        .description("Current number of entries in the region.")
+        .baseUnit("entries")
+        .register(meterRegistry);
+  }
 
+  @Override
+  protected void close() {
+    meterRegistry.remove(entriesGauge);
+    super.close();
   }
 
   @Override
diff --git a/geode-core/src/main/java/org/apache/geode/internal/statistics/StatisticsImpl.java b/geode-core/src/main/java/org/apache/geode/internal/statistics/StatisticsImpl.java
index 72cb9ff..c8a8551 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/statistics/StatisticsImpl.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/statistics/StatisticsImpl.java
@@ -45,7 +45,7 @@ public abstract class StatisticsImpl implements Statistics {
   private static final Logger logger = LogService.getLogger();
 
   /** The type of this statistics instance */
-  protected final StatisticsTypeImpl type;
+  protected final ValidatingStatisticsType type;
 
   /** The display name of this statistics instance */
   private final String textId;
@@ -127,7 +127,7 @@ public abstract class StatisticsImpl implements Statistics {
    */
   StatisticsImpl(StatisticsType type, String textId, long numericId, long uniqueId,
       int osStatFlags, StatisticsManager statisticsManager, StatisticsLogger statisticsLogger) {
-    this.type = (StatisticsTypeImpl) type;
+    this.type = (ValidatingStatisticsType) type;
     this.textId = StringUtils.isEmpty(textId) ? statisticsManager.getName() : textId;
     this.numericId = numericId == 0 ? statisticsManager.getPid() : numericId;
     this.uniqueId = uniqueId;
diff --git a/geode-core/src/main/java/org/apache/geode/internal/statistics/StatisticsTypeImpl.java b/geode-core/src/main/java/org/apache/geode/internal/statistics/StatisticsTypeImpl.java
index 19608b4..9af5258 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/statistics/StatisticsTypeImpl.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/statistics/StatisticsTypeImpl.java
@@ -32,7 +32,7 @@ import org.apache.geode.annotations.Immutable;
  * @since GemFire 3.0
  */
 @Immutable
-public class StatisticsTypeImpl implements StatisticsType {
+public class StatisticsTypeImpl implements ValidatingStatisticsType {
 
   /** The name of this statistics type */
   private final String name;
diff --git a/geode-core/src/main/java/org/apache/geode/internal/statistics/ValidatingStatisticsType.java b/geode-core/src/main/java/org/apache/geode/internal/statistics/ValidatingStatisticsType.java
new file mode 100644
index 0000000..a6bd059
--- /dev/null
+++ b/geode-core/src/main/java/org/apache/geode/internal/statistics/ValidatingStatisticsType.java
@@ -0,0 +1,24 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
+ * agreements. See the NOTICE file distributed with this work for additional information regarding
+ * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License. You may obtain a
+ * copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License
+ * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
+ * or implied. See the License for the specific language governing permissions and limitations under
+ * the License.
+ */
+package org.apache.geode.internal.statistics;
+
+import org.apache.geode.StatisticsType;
+
+public interface ValidatingStatisticsType extends StatisticsType {
+
+  boolean isValidLongId(int id);
+
+  boolean isValidDoubleId(int id);
+}
diff --git a/geode-core/src/main/java/org/apache/geode/internal/statistics/meters/LegacyStatGauge.java b/geode-core/src/main/java/org/apache/geode/internal/statistics/meters/LegacyStatGauge.java
deleted file mode 100644
index 9c5b084..0000000
--- a/geode-core/src/main/java/org/apache/geode/internal/statistics/meters/LegacyStatGauge.java
+++ /dev/null
@@ -1,71 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
- * agreements. See the NOTICE file distributed with this work for additional information regarding
- * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License. You may obtain a
- * copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software distributed under the License
- * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
- * or implied. See the License for the specific language governing permissions and limitations under
- * the License.
- */
-
-package org.apache.geode.internal.statistics.meters;
-
-import io.micrometer.core.instrument.Gauge;
-import io.micrometer.core.instrument.MeterRegistry;
-
-import org.apache.geode.Statistics;
-
-public class LegacyStatGauge implements Gauge {
-  public static Builder builder(String name) {
-    return new Builder(name);
-  }
-
-  @Override
-  public double value() {
-    return 0;
-  }
-
-  @Override
-  public Id getId() {
-    return null;
-  }
-
-  public static class Builder {
-
-    public Builder(String name) {
-
-    }
-
-    /**
-     * Prepares to associate the eventual {@code LegacyStatCounter} with the specified {@code
-     * long} stat. The given {@code Statistics} and {@code statId} must identify a {@code long}
-     * stat.
-     */
-    public Builder longStatistic(Statistics statistics, int statId) {
-      return this;
-    }
-
-    public Builder description(String description) {
-      return this;
-    }
-
-    public Builder baseUnit(String unit) {
-      return this;
-    }
-
-    /**
-     * Registers a {@code Counter} with the given registry, and returns a {@code LegacyStatCounter}
-     * that wraps the counter to increment and read the associated stat. Note that the returned
-     * {@code LegacyStatCounter} is not registered with the registry, but it has the same ID, so
-     * it can be used to remove the registered counter from the registry.
-     */
-    public LegacyStatGauge register(MeterRegistry registry) {
-      return new LegacyStatGauge();
-    }
-  }
-}
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/RegionPerfStatsTest.java b/geode-core/src/test/java/org/apache/geode/internal/cache/RegionPerfStatsTest.java
index fb39158..33b6472 100644
--- a/geode-core/src/test/java/org/apache/geode/internal/cache/RegionPerfStatsTest.java
+++ b/geode-core/src/test/java/org/apache/geode/internal/cache/RegionPerfStatsTest.java
@@ -22,6 +22,12 @@ import static org.mockito.Mockito.verify;
 import static org.mockito.Mockito.when;
 import static org.mockito.quality.Strictness.STRICT_STUBS;
 
+import io.micrometer.core.instrument.Gauge;
+import io.micrometer.core.instrument.Meter;
+import io.micrometer.core.instrument.MeterRegistry;
+import io.micrometer.core.instrument.Timer;
+import io.micrometer.core.instrument.simple.SimpleMeterRegistry;
+import org.junit.After;
 import org.junit.Before;
 import org.junit.Rule;
 import org.junit.Test;
@@ -34,25 +40,36 @@ import org.apache.geode.StatisticsFactory;
 
 public class RegionPerfStatsTest {
 
+  private static final String REGION_NAME = "regionName";
+
+  private MeterRegistry meterRegistry;
   private StatisticsFactory statisticsFactory;
-  private CachePerfStats cachePerfStats;
   private Statistics statistics;
-  private final String theRegionName = "TheRegionName";
-  private RegionPerfStats subject;
+  private CachePerfStats cachePerfStats;
+
+  private RegionPerfStats regionPerfStats;
 
   @Rule
   public MockitoRule mockitoRule = MockitoJUnit.rule().strictness(STRICT_STUBS);
 
   @Before
   public void setUp() {
+    meterRegistry = new SimpleMeterRegistry();
     statisticsFactory = mock(StatisticsFactory.class);
-    statistics = mock(Statistics.class);
     cachePerfStats = mock(CachePerfStats.class);
+    statistics = mock(Statistics.class);
+
+    when(statisticsFactory.createAtomicStatistics(any(), any())).thenReturn(statistics);
 
-    when(statisticsFactory.createAtomicStatistics(any(), any()))
-        .thenReturn(statistics);
+    regionPerfStats =
+        new RegionPerfStats(statisticsFactory, cachePerfStats, REGION_NAME, meterRegistry, () -> 0);
+  }
 
-    subject = new RegionPerfStats(statisticsFactory, cachePerfStats, theRegionName, () -> 0);
+  @After
+  public void closeStats() {
+    if (regionPerfStats != null) {
+      regionPerfStats.close();
+    }
   }
 
   @Test
@@ -60,20 +77,87 @@ public class RegionPerfStatsTest {
     ArgumentCaptor<String> captor = ArgumentCaptor.forClass(String.class);
     verify(statisticsFactory).createAtomicStatistics(any(), captor.capture());
 
-    assertThat(captor.getValue()).isEqualTo("RegionStats-" + theRegionName);
+    assertThat(captor.getValue()).isEqualTo("RegionStats-" + REGION_NAME);
   }
 
   @Test
-  public void incEntryCount_whenDeltaIsPositive_increasesGauge() {
-    subject.incEntryCount(1);
+  public void incEntryCount_whenDeltaIsPositive_increasesEntryGaugeStat() {
+    regionPerfStats.incEntryCount(1);
 
     verify(statistics).incLong(entryGaugeId, 1);
   }
 
   @Test
-  public void incEntryCount_whenDeltaIsNegative_decreasesGauge() {
-    subject.incEntryCount(-1);
+  public void incEntryCount_whenDeltaIsNegative_decreasesEntryGaugeStat() {
+    regionPerfStats.incEntryCount(-1);
 
     verify(statistics).incLong(entryGaugeId, -1);
   }
+
+  @Test
+  public void getEntries_getsValueFromEntryGaugeStat() {
+    long statValue = 22;
+    when(statistics.getLong(entryGaugeId)).thenReturn(statValue);
+
+    regionPerfStats =
+        new RegionPerfStats(statisticsFactory, cachePerfStats, REGION_NAME, meterRegistry, () -> 0);
+
+    assertThat(regionPerfStats.getEntries()).isEqualTo(statValue);
+  }
+
+  @Test
+  public void regionPerfStatsConstruction_createsEntriesGauge() {
+    Gauge entriesGauge = meterRegistry
+        .find("member.region.entries")
+        .gauge();
+    assertThat(entriesGauge).isNotNull();
+  }
+
+  @Test
+  public void entriesGauge_getsValueFromEntryGaugeStat() {
+    long statValue = 22;
+    when(statistics.getLong(entryGaugeId)).thenReturn(statValue);
+
+    regionPerfStats =
+        new RegionPerfStats(statisticsFactory, cachePerfStats, REGION_NAME, meterRegistry, () -> 0);
+
+    assertThat(regionPerfStats.getEntries()).isEqualTo(statValue);
+  }
+
+  @Test
+  public void close_removesItsOwnMetersFromTheRegistry() {
+    assertThat(meterNamed("member.region.entries"))
+        .as("entries gauge before closing the stats")
+        .isNotNull();
+
+    regionPerfStats.close();
+
+    assertThat(meterNamed("member.region.entries"))
+        .as("entries gauge after closing the stats")
+        .isNull();
+
+    regionPerfStats = null;
+  }
+
+  @Test
+  public void close_doesNotRemoveMetersItDoesNotOwn() {
+    String foreignMeterName = "some.meter.not.created.by.the.gateway.receiver.stats";
+
+    Timer.builder(foreignMeterName)
+        .register(meterRegistry);
+
+    regionPerfStats.close();
+
+    assertThat(meterNamed(foreignMeterName))
+        .as("foreign meter after closing the stats")
+        .isNotNull();
+
+    regionPerfStats = null;
+  }
+
+  private Meter meterNamed(String meterName) {
+    return meterRegistry
+        .find(meterName)
+        .meter();
+  }
 }
diff --git a/geode-core/src/test/java/org/apache/geode/internal/statistics/SimpleStatistics.java b/geode-core/src/test/java/org/apache/geode/internal/statistics/SimpleStatistics.java
new file mode 100644
index 0000000..96e05f3
--- /dev/null
+++ b/geode-core/src/test/java/org/apache/geode/internal/statistics/SimpleStatistics.java
@@ -0,0 +1,70 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
+ * agreements. See the NOTICE file distributed with this work for additional information regarding
+ * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License. You may obtain a
+ * copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License
+ * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
+ * or implied. See the License for the specific language governing permissions and limitations under
+ * the License.
+ */
+package org.apache.geode.internal.statistics;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.geode.StatisticsType;
+
+public class SimpleStatistics extends StatisticsImpl {
+
+  private final Map<Number, Number> values = new HashMap<>();
+
+  public SimpleStatistics(StatisticsType type, String textId, long numericId, long uniqueId,
+      int osStatFlags, StatisticsManager statisticsManager) {
+    super(type, textId, numericId, uniqueId, osStatFlags, statisticsManager);
+  }
+
+  public SimpleStatistics(StatisticsType type, String textId, long numericId, long uniqueId,
+      int osStatFlags, StatisticsManager statisticsManager, StatisticsLogger statisticsLogger) {
+    super(type, textId, numericId, uniqueId, osStatFlags, statisticsManager, statisticsLogger);
+  }
+
+  @Override
+  public boolean isAtomic() {
+    return false;
+  }
+
+  @Override
+  protected void _setLong(int offset, long value) {
+    values.put(offset, value);
+  }
+
+  @Override
+  protected void _setDouble(int offset, double value) {
+    values.put(offset, value);
+  }
+
+  @Override
+  protected long _getLong(int offset) {
+    return (long) values.get(offset);
+  }
+
+  @Override
+  protected double _getDouble(int offset) {
+    return (double) values.get(offset);
+  }
+
+  @Override
+  protected void _incLong(int offset, long delta) {
+    values.put(offset, (long) values.get(delta) + 1);
+  }
+
+  @Override
+  protected void _incDouble(int offset, double delta) {
+    values.put(offset, (double) values.get(delta) + 1);
+  }
+}