You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@geode.apache.org by kl...@apache.org on 2016/10/26 21:23:09 UTC

[52/52] incubator-geode git commit: GEODE-2012: always write stat types to archive

GEODE-2012: always write stat types to archive

* write additional tests for stat archive rolling
* expose bug GEODE-2012 in StatTypesAreRolledOverRegressionTest
* fix failure in StatTypesAreRolledOverRegressionTest


Project: http://git-wip-us.apache.org/repos/asf/incubator-geode/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-geode/commit/3e1b407f
Tree: http://git-wip-us.apache.org/repos/asf/incubator-geode/tree/3e1b407f
Diff: http://git-wip-us.apache.org/repos/asf/incubator-geode/diff/3e1b407f

Branch: refs/heads/feature/GEODE-2012
Commit: 3e1b407fc4031eb6cbdbd992c6949326f9296cc7
Parents: 8bf3957
Author: Kirk Lund <kl...@apache.org>
Authored: Wed Oct 26 14:07:44 2016 -0700
Committer: Kirk Lund <kl...@apache.org>
Committed: Wed Oct 26 14:07:44 2016 -0700

----------------------------------------------------------------------
 .../org/apache/geode/internal/NanoTimer.java    |   6 +-
 .../geode/internal/i18n/LocalizedStrings.java   |   2 +-
 .../internal/statistics/HostStatSampler.java    |  31 ++-
 .../internal/statistics/SampleCollector.java    |  10 +-
 .../internal/statistics/SimpleStatSampler.java  |   7 +-
 .../concurrent/StoppableCountDownLatch.java     |   8 +-
 .../DiskSpaceLimitIntegrationTest.java          | 139 ++++++++++++
 .../FileSizeLimitIntegrationTest.java           | 128 ++++++++++++
 .../StatTypesAreRolledOverRegressionTest.java   | 209 +++++++++++++++++++
 9 files changed, 524 insertions(+), 16 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3e1b407f/geode-core/src/main/java/org/apache/geode/internal/NanoTimer.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/internal/NanoTimer.java b/geode-core/src/main/java/org/apache/geode/internal/NanoTimer.java
index 12e91f8..247f9a9 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/NanoTimer.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/NanoTimer.java
@@ -38,7 +38,7 @@ package org.apache.geode.internal;
  * </pre>
  * 
  */
-public final class NanoTimer {
+public class NanoTimer {
 
   public static final long NANOS_PER_MILLISECOND = 1000000;
 
@@ -73,7 +73,7 @@ public final class NanoTimer {
   /**
    * For unit testing
    */
-  NanoTimer(TimeService ts) {
+  protected NanoTimer(TimeService ts) {
     this.timeService = ts;
     this.lastResetTime = ts.getTime();
     this.constructionTime = this.lastResetTime;
@@ -164,7 +164,7 @@ public final class NanoTimer {
   /**
    * Allows unit tests to insert a deterministic clock for testing.
    */
-  interface TimeService {
+  public interface TimeService {
     /**
      * Returns the current time.
      */

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3e1b407f/geode-core/src/main/java/org/apache/geode/internal/i18n/LocalizedStrings.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/internal/i18n/LocalizedStrings.java b/geode-core/src/main/java/org/apache/geode/internal/i18n/LocalizedStrings.java
index 0e4fac2..905f291 100755
--- a/geode-core/src/main/java/org/apache/geode/internal/i18n/LocalizedStrings.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/i18n/LocalizedStrings.java
@@ -1265,7 +1265,7 @@ public class LocalizedStrings {
           "Could not free space in {0} directory.  The space used is {1} which exceeds the configured limit of {2}.");
 
   public static final StringId ManagerLogWriter_DELETED_INACTIVE__0___1_ =
-      new StringId(1797, "Deleted inactive  {0}  \"{1}\".");
+      new StringId(1797, "Deleted inactive {0} \"{1}\".");
   public static final StringId ManagerLogWriter_SWITCHING_TO_LOG__0 =
       new StringId(1798, "Switching to log {0}");
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3e1b407f/geode-core/src/main/java/org/apache/geode/internal/statistics/HostStatSampler.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/internal/statistics/HostStatSampler.java b/geode-core/src/main/java/org/apache/geode/internal/statistics/HostStatSampler.java
index 6d7b967..494362c 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/statistics/HostStatSampler.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/statistics/HostStatSampler.java
@@ -89,12 +89,20 @@ public abstract class HostStatSampler
 
   private final CallbackSampler callbackSampler;
 
+  private final NanoTimer timer;
+
   protected HostStatSampler(CancelCriterion stopper, StatSamplerStats samplerStats) {
+    this(stopper, samplerStats, new NanoTimer());
+  }
+
+  protected HostStatSampler(CancelCriterion stopper, StatSamplerStats samplerStats,
+      NanoTimer timer) {
     this.stopper = stopper;
     this.statSamplerInitializedLatch = new StoppableCountDownLatch(this.stopper, 1);
     this.samplerStats = samplerStats;
     this.fileSizeLimitInKB = Boolean.getBoolean(TEST_FILE_SIZE_LIMIT_IN_KB_PROPERTY);
     this.callbackSampler = new CallbackSampler(stopper, samplerStats);
+    this.timer = timer;
   }
 
   public final StatSamplerStats getStatSamplerStats() {
@@ -171,8 +179,6 @@ public abstract class HostStatSampler
    */
   @Override
   public final void run() {
-    NanoTimer timer = new NanoTimer();
-
     final boolean isDebugEnabled_STATISTICS = logger.isTraceEnabled(LogMarker.STATISTICS);
     if (isDebugEnabled_STATISTICS) {
       logger.trace(LogMarker.STATISTICS, "HostStatSampler started");
@@ -182,7 +188,7 @@ public abstract class HostStatSampler
       initSpecialStats();
 
       this.sampleCollector = new SampleCollector(this);
-      this.sampleCollector.initialize(this, NanoTimer.getTime());
+      this.sampleCollector.initialize(this, timer.getTime());
 
       this.statSamplerInitializedLatch.countDown();
       latchCountedDown = true;
@@ -197,7 +203,7 @@ public abstract class HostStatSampler
         }
         final long nanosBeforeSleep = timer.getLastResetTime();
         final long nanosToDelay = nanosLastTimeStamp + getNanoRate();
-        delay(timer, nanosToDelay);
+        delay(nanosToDelay);
         nanosLastTimeStamp = timer.getLastResetTime();
         if (!stopRequested() && isSamplingEnabled()) {
           final long nanosTimeStamp = timer.getLastResetTime();
@@ -371,11 +377,21 @@ public abstract class HostStatSampler
    * @since GemFire 7.0
    */
   public final boolean waitForInitialization(long ms) throws InterruptedException {
-    return this.statSamplerInitializedLatch.await(ms);
+    return awaitInitialization(ms, TimeUnit.MILLISECONDS);
+  }
+
+  /**
+   * Awaits the initialization of special statistics.
+   *
+   * @see #initSpecialStats
+   */
+  public final boolean awaitInitialization(final long timeout, final TimeUnit unit)
+      throws InterruptedException {
+    return this.statSamplerInitializedLatch.await(timeout, unit);
   }
 
   public final void changeArchive(File newFile) {
-    this.sampleCollector.changeArchive(newFile, NanoTimer.getTime());
+    this.sampleCollector.changeArchive(newFile, timer.getTime());
   }
 
   /**
@@ -476,10 +492,9 @@ public abstract class HostStatSampler
   }
 
   /**
-   * @param timer a NanoTimer used to compute the elapsed delay
    * @param nanosToDelay the timestamp to delay until it is the current time
    */
-  private void delay(NanoTimer timer, final long nanosToDelay) throws InterruptedException {
+  private void delay(final long nanosToDelay) throws InterruptedException {
     timer.reset();
     long now = timer.getLastResetTime();
     long remainingNanos = nanosToDelay - now;

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3e1b407f/geode-core/src/main/java/org/apache/geode/internal/statistics/SampleCollector.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/internal/statistics/SampleCollector.java b/geode-core/src/main/java/org/apache/geode/internal/statistics/SampleCollector.java
index 610e848..4892226 100755
--- a/geode-core/src/main/java/org/apache/geode/internal/statistics/SampleCollector.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/statistics/SampleCollector.java
@@ -366,7 +366,8 @@ public class SampleCollector {
     }
 
     // notify unmarked/new handlers but not marked/old handlers
-    notifyNewHandlersOfResources(handlers, this.resourceInstMap.values());
+    notifyNewHandlersOfResources(handlers, this.resourceTypeMap.values(),
+        this.resourceInstMap.values());
   }
 
   private ResourceType getResourceType(List<MarkableSampleHandler> handlers, Statistics statistics)
@@ -489,7 +490,7 @@ public class SampleCollector {
   }
 
   private void notifyNewHandlersOfResources(List<MarkableSampleHandler> handlers,
-      Collection<ResourceInstance> resources) {
+      Collection<ResourceType> types, Collection<ResourceInstance> resources) {
     final boolean isDebugEnabled_STATISTICS = logger.isTraceEnabled(LogMarker.STATISTICS);
     if (isDebugEnabled_STATISTICS) {
       logger.trace(LogMarker.STATISTICS,
@@ -509,6 +510,11 @@ public class SampleCollector {
           // allocatedResourceInstance...
           handler.allocatedResourceInstance(resourceInstance);
         }
+        for (ResourceType resourceType : types) {
+          if (!allocatedResourceTypes.contains(resourceType)) {
+            handler.allocatedResourceType(resourceType);
+          }
+        }
         handler.mark();
         count++;
       }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3e1b407f/geode-core/src/main/java/org/apache/geode/internal/statistics/SimpleStatSampler.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/internal/statistics/SimpleStatSampler.java b/geode-core/src/main/java/org/apache/geode/internal/statistics/SimpleStatSampler.java
index 7eaa1e0..1707397 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/statistics/SimpleStatSampler.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/statistics/SimpleStatSampler.java
@@ -19,6 +19,7 @@ import java.io.File;
 import org.apache.logging.log4j.Logger;
 
 import org.apache.geode.CancelCriterion;
+import org.apache.geode.internal.NanoTimer;
 import org.apache.geode.internal.i18n.LocalizedStrings;
 import org.apache.geode.internal.logging.LogService;
 import org.apache.geode.internal.logging.log4j.LocalizedMessage;
@@ -58,7 +59,11 @@ public class SimpleStatSampler extends HostStatSampler {
   private final StatisticsManager sm;
 
   public SimpleStatSampler(CancelCriterion stopper, StatisticsManager sm) {
-    super(stopper, new StatSamplerStats(sm, sm.getId()));
+    this(stopper, sm, new NanoTimer());
+  }
+
+  public SimpleStatSampler(CancelCriterion stopper, StatisticsManager sm, NanoTimer timer) {
+    super(stopper, new StatSamplerStats(sm, sm.getId()), timer);
     this.sm = sm;
     logger.info(LogMarker.STATISTICS, LocalizedMessage
         .create(LocalizedStrings.SimpleStatSampler_STATSSAMPLERATE_0, getSampleRate()));

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3e1b407f/geode-core/src/main/java/org/apache/geode/internal/util/concurrent/StoppableCountDownLatch.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/internal/util/concurrent/StoppableCountDownLatch.java b/geode-core/src/main/java/org/apache/geode/internal/util/concurrent/StoppableCountDownLatch.java
index 740fd7f..4726e51 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/util/concurrent/StoppableCountDownLatch.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/util/concurrent/StoppableCountDownLatch.java
@@ -46,6 +46,7 @@ public class StoppableCountDownLatch {
   /**
    * @param count the number of times {@link #countDown} must be invoked before threads can pass
    *        through {@link #await()}
+   *
    * @throws IllegalArgumentException if {@code count} is negative
    */
   public StoppableCountDownLatch(CancelCriterion stopper, int count) {
@@ -68,14 +69,19 @@ public class StoppableCountDownLatch {
 
   /**
    * @param msTimeout how long to wait in milliseconds
+   *
    * @return true if it was unlatched
-   * @throws InterruptedException
    */
   public boolean await(long msTimeout) throws InterruptedException {
     stopper.checkCancelInProgress(null);
     return latch.await(msTimeout, TimeUnit.MILLISECONDS);
   }
 
+  public boolean await(final long timeout, final TimeUnit unit) throws InterruptedException {
+    stopper.checkCancelInProgress(null);
+    return latch.await(timeout, unit);
+  }
+
   public synchronized void countDown() {
     latch.countDown();
   }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3e1b407f/geode-core/src/test/java/org/apache/geode/internal/statistics/DiskSpaceLimitIntegrationTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/internal/statistics/DiskSpaceLimitIntegrationTest.java b/geode-core/src/test/java/org/apache/geode/internal/statistics/DiskSpaceLimitIntegrationTest.java
new file mode 100644
index 0000000..541bf54
--- /dev/null
+++ b/geode-core/src/test/java/org/apache/geode/internal/statistics/DiskSpaceLimitIntegrationTest.java
@@ -0,0 +1,139 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
+ * agreements. See the NOTICE file distributed with this work for additional information regarding
+ * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License. You may obtain a
+ * copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License
+ * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
+ * or implied. See the License for the specific language governing permissions and limitations under
+ * the License.
+ */
+package org.apache.geode.internal.statistics;
+
+import static java.util.concurrent.TimeUnit.*;
+import static org.assertj.core.api.Assertions.*;
+import static org.mockito.Mockito.*;
+
+import java.io.File;
+import java.util.concurrent.TimeoutException;
+
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.contrib.java.lang.system.RestoreSystemProperties;
+import org.junit.experimental.categories.Category;
+import org.junit.rules.TemporaryFolder;
+import org.junit.rules.TestName;
+
+import org.apache.geode.StatisticDescriptor;
+import org.apache.geode.Statistics;
+import org.apache.geode.StatisticsType;
+import org.apache.geode.internal.NanoTimer;
+import org.apache.geode.test.junit.categories.IntegrationTest;
+
+@Category(IntegrationTest.class)
+public class DiskSpaceLimitIntegrationTest {
+
+  private static final long FILE_SIZE_LIMIT = 1024 * 1;
+  private static final long DISK_SPACE_LIMIT = Long.MAX_VALUE;
+
+  private File dir;
+  private String archiveFileName;
+
+  private LocalStatisticsFactory factory;
+  private StatisticDescriptor[] statisticDescriptors;
+  private StatisticsType statisticsType;
+  private Statistics statistics;
+
+  private SampleCollector sampleCollector;
+  private StatArchiveHandlerConfig config;
+
+  @Rule
+  public RestoreSystemProperties restoreSystemProperties = new RestoreSystemProperties();
+  @Rule
+  public TemporaryFolder temporaryFolder = new TemporaryFolder();
+  @Rule
+  public TestName testName = new TestName();
+
+  @Before
+  public void setUp() throws Exception {
+    this.dir = this.temporaryFolder.getRoot();
+    this.archiveFileName =
+        new File(this.dir, this.testName.getMethodName() + ".gfs").getAbsolutePath();
+
+    this.factory = new LocalStatisticsFactory(null);
+    this.statisticDescriptors = new StatisticDescriptor[] {
+        this.factory.createIntCounter("stat1", "description of stat1", "units", true),};
+    this.statisticsType =
+        factory.createType("statisticsType1", "statisticsType1", this.statisticDescriptors);
+    this.statistics = factory.createAtomicStatistics(this.statisticsType, "statistics1", 1);
+
+    StatisticsSampler sampler = mock(StatisticsSampler.class);
+    when(sampler.getStatistics()).thenReturn(this.factory.getStatistics());
+
+    this.config = mock(StatArchiveHandlerConfig.class);
+    when(this.config.getArchiveFileName()).thenReturn(new File(this.archiveFileName));
+    when(this.config.getArchiveFileSizeLimit()).thenReturn(FILE_SIZE_LIMIT);
+    when(this.config.getSystemId()).thenReturn(1L);
+    when(this.config.getSystemStartTime()).thenReturn(System.currentTimeMillis());
+    when(this.config.getSystemDirectoryPath())
+        .thenReturn(this.temporaryFolder.getRoot().getAbsolutePath());
+    when(this.config.getProductDescription()).thenReturn(this.testName.getMethodName());
+    when(this.config.getArchiveDiskSpaceLimit()).thenReturn(DISK_SPACE_LIMIT);
+
+    this.sampleCollector = new SampleCollector(sampler);
+    this.sampleCollector.initialize(this.config, NanoTimer.getTime());
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    StatisticsTypeFactoryImpl.clear();
+  }
+
+  @Test
+  public void zeroKeepsAllFiles() throws Exception {
+    when(this.config.getArchiveDiskSpaceLimit()).thenReturn(0L);
+    sampleUntilFileExists(archiveFile(1));
+    sampleUntilFileExists(archiveFile(2));
+    assertThat(archiveFile(1)).exists();
+    assertThat(archiveFile(2)).exists();
+  }
+
+  @Test
+  public void sameKeepsOneFile() throws Exception {
+    when(this.config.getArchiveDiskSpaceLimit()).thenReturn(FILE_SIZE_LIMIT * 2);
+    sampleUntilFileExists(archiveFile(1));
+    sampleUntilFileExists(archiveFile(2));
+    assertThat(archiveFile(1)).doesNotExist();
+    assertThat(archiveFile(2)).exists();
+  }
+
+  private File archiveFile(final int child) {
+    return new File(this.dir,
+        this.testName.getMethodName() + "-01-" + String.format("%02d", child) + ".gfs");
+  }
+
+  private void sampleUntilFileExists(final File file)
+      throws InterruptedException, TimeoutException {
+    long end = System.nanoTime() + MINUTES.toNanos(1);
+    while (!file.exists() && System.nanoTime() < end) {
+      sample();
+    }
+    if (!file.exists()) {
+      throw new TimeoutException("File " + file + " does not exist within " + 1 + " " + MINUTES);
+    }
+  }
+
+  private void sample() {
+    getSampleCollector().sample(System.nanoTime());
+  }
+
+  private SampleCollector getSampleCollector() {
+    return this.sampleCollector;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3e1b407f/geode-core/src/test/java/org/apache/geode/internal/statistics/FileSizeLimitIntegrationTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/internal/statistics/FileSizeLimitIntegrationTest.java b/geode-core/src/test/java/org/apache/geode/internal/statistics/FileSizeLimitIntegrationTest.java
new file mode 100644
index 0000000..5bf31b7
--- /dev/null
+++ b/geode-core/src/test/java/org/apache/geode/internal/statistics/FileSizeLimitIntegrationTest.java
@@ -0,0 +1,128 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
+ * agreements. See the NOTICE file distributed with this work for additional information regarding
+ * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License. You may obtain a
+ * copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License
+ * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
+ * or implied. See the License for the specific language governing permissions and limitations under
+ * the License.
+ */
+package org.apache.geode.internal.statistics;
+
+import static java.util.concurrent.TimeUnit.*;
+import static org.assertj.core.api.Assertions.*;
+import static org.mockito.Mockito.*;
+
+import java.io.File;
+import java.util.concurrent.TimeoutException;
+
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.contrib.java.lang.system.RestoreSystemProperties;
+import org.junit.experimental.categories.Category;
+import org.junit.rules.TemporaryFolder;
+import org.junit.rules.TestName;
+
+import org.apache.geode.StatisticDescriptor;
+import org.apache.geode.Statistics;
+import org.apache.geode.StatisticsType;
+import org.apache.geode.internal.NanoTimer;
+import org.apache.geode.test.junit.categories.IntegrationTest;
+
+@Category(IntegrationTest.class)
+public class FileSizeLimitIntegrationTest {
+
+  private static final long FILE_SIZE_LIMIT = 1024 * 1;
+
+  private File dir;
+  private String archiveFileName;
+
+  private LocalStatisticsFactory factory;
+  private StatisticDescriptor[] statisticDescriptors;
+  private StatisticsType statisticsType;
+  private Statistics statistics;
+
+  private SampleCollector sampleCollector;
+
+  @Rule
+  public RestoreSystemProperties restoreSystemProperties = new RestoreSystemProperties();
+  @Rule
+  public TemporaryFolder temporaryFolder = new TemporaryFolder();
+  @Rule
+  public TestName testName = new TestName();
+
+  @Before
+  public void setUp() throws Exception {
+    this.dir = this.temporaryFolder.getRoot();
+    this.archiveFileName =
+        new File(this.dir, this.testName.getMethodName() + ".gfs").getAbsolutePath();
+
+    this.factory = new LocalStatisticsFactory(null);
+    this.statisticDescriptors = new StatisticDescriptor[] {
+        this.factory.createIntCounter("stat1", "description of stat1", "units", true),};
+    this.statisticsType =
+        factory.createType("statisticsType1", "statisticsType1", this.statisticDescriptors);
+    this.statistics = factory.createAtomicStatistics(this.statisticsType, "statistics1", 1);
+
+    StatisticsSampler sampler = mock(StatisticsSampler.class);
+    when(sampler.getStatistics()).thenReturn(this.factory.getStatistics());
+
+    StatArchiveHandlerConfig config = mock(StatArchiveHandlerConfig.class);
+    when(config.getArchiveFileName()).thenReturn(new File(this.archiveFileName));
+    when(config.getArchiveFileSizeLimit()).thenReturn(FILE_SIZE_LIMIT);
+    when(config.getSystemId()).thenReturn(1L);
+    when(config.getSystemStartTime()).thenReturn(System.currentTimeMillis());
+    when(config.getSystemDirectoryPath())
+        .thenReturn(this.temporaryFolder.getRoot().getAbsolutePath());
+    when(config.getProductDescription()).thenReturn(this.testName.getMethodName());
+    when(config.getArchiveDiskSpaceLimit()).thenReturn(0L);
+
+    this.sampleCollector = new SampleCollector(sampler);
+    this.sampleCollector.initialize(config, NanoTimer.getTime());
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    StatisticsTypeFactoryImpl.clear();
+  }
+
+  private File archiveFile(final int child) {
+    return new File(this.dir,
+        this.testName.getMethodName() + "-01-" + String.format("%02d", child) + ".gfs");
+  }
+
+  @Test
+  public void rollsWhenLimitIsReached() throws Exception { // TODO: add test to assert size is
+                                                           // correct
+    sampleUntilFileExists(archiveFile(1));
+    sampleUntilFileExists(archiveFile(2));
+    assertThat(archiveFile(1)).exists();
+    assertThat(archiveFile(2)).exists();
+  }
+
+  private void sampleUntilFileExists(final File file)
+      throws InterruptedException, TimeoutException {
+    long end = System.nanoTime() + MINUTES.toNanos(1);
+    while (!file.exists() && System.nanoTime() < end) {
+      sample();
+    }
+    if (!file.exists()) {
+      throw new TimeoutException("File " + file + " does not exist within " + 1 + " " + MINUTES);
+    }
+  }
+
+  private void sample() {
+    getSampleCollector().sample(System.nanoTime());
+  }
+
+  private SampleCollector getSampleCollector() {
+    return this.sampleCollector;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3e1b407f/geode-core/src/test/java/org/apache/geode/internal/statistics/StatTypesAreRolledOverRegressionTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/internal/statistics/StatTypesAreRolledOverRegressionTest.java b/geode-core/src/test/java/org/apache/geode/internal/statistics/StatTypesAreRolledOverRegressionTest.java
new file mode 100644
index 0000000..9896ec2
--- /dev/null
+++ b/geode-core/src/test/java/org/apache/geode/internal/statistics/StatTypesAreRolledOverRegressionTest.java
@@ -0,0 +1,209 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
+ * agreements. See the NOTICE file distributed with this work for additional information regarding
+ * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License. You may obtain a
+ * copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License
+ * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
+ * or implied. See the License for the specific language governing permissions and limitations under
+ * the License.
+ */
+package org.apache.geode.internal.statistics;
+
+import static java.util.concurrent.TimeUnit.*;
+import static org.assertj.core.api.Assertions.*;
+import static org.mockito.Mockito.*;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.List;
+import java.util.concurrent.TimeoutException;
+
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.contrib.java.lang.system.RestoreSystemProperties;
+import org.junit.experimental.categories.Category;
+import org.junit.rules.TemporaryFolder;
+import org.junit.rules.TestName;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
+
+import org.apache.geode.StatisticDescriptor;
+import org.apache.geode.Statistics;
+import org.apache.geode.StatisticsType;
+import org.apache.geode.internal.NanoTimer;
+import org.apache.geode.internal.statistics.StatArchiveReader.ResourceInst;
+import org.apache.geode.test.junit.categories.IntegrationTest;
+
+@Category(IntegrationTest.class)
+public class StatTypesAreRolledOverRegressionTest {
+
+  private static final long FILE_SIZE_LIMIT = 1024 * 1;
+
+  private File dir;
+  private String archiveFileName;
+
+  private LocalStatisticsFactory factory;
+  private StatisticDescriptor[] statisticDescriptors;
+  private StatisticsType statisticsType;
+  private Statistics statistics;
+
+  private SampleCollector sampleCollector;
+
+  private NanoTimer timer = new NanoTimer();
+  private long nanosTimeStamp;
+
+  @Rule
+  public RestoreSystemProperties restoreSystemProperties = new RestoreSystemProperties();
+  @Rule
+  public TemporaryFolder temporaryFolder = new TemporaryFolder();
+  @Rule
+  public TestName testName = new TestName();
+
+  @Before
+  public void setUp() throws Exception {
+    this.dir = this.temporaryFolder.getRoot();
+    this.archiveFileName =
+        new File(this.dir, this.testName.getMethodName() + ".gfs").getAbsolutePath();
+
+    this.factory = new LocalStatisticsFactory(null);
+    this.statisticDescriptors = new StatisticDescriptor[] {
+        this.factory.createIntCounter("stat1", "description of stat1", "units", true),};
+    this.statisticsType =
+        factory.createType("statisticsType1", "statisticsType1", this.statisticDescriptors);
+    this.statistics = factory.createAtomicStatistics(this.statisticsType, "statistics1", 1);
+
+    Answer<Statistics[]> statisticsAnswer = new Answer<Statistics[]>() {
+      public Statistics[] answer(InvocationOnMock invocation) throws Throwable {
+        return factory.getStatistics();
+      }
+    };
+
+    Answer<Integer> modCountAnswer = new Answer<Integer>() {
+      public Integer answer(InvocationOnMock invocation) throws Throwable {
+        return factory.getStatListModCount();
+      }
+    };
+
+    StatisticsSampler sampler = mock(StatisticsSampler.class);
+    when(sampler.getStatistics()).thenAnswer(statisticsAnswer);
+    when(sampler.getStatisticsModCount()).thenAnswer(modCountAnswer);
+
+    StatArchiveHandlerConfig config = mock(StatArchiveHandlerConfig.class);
+    when(config.getArchiveFileName()).thenReturn(new File(this.archiveFileName));
+    when(config.getArchiveFileSizeLimit()).thenReturn(FILE_SIZE_LIMIT);
+    when(config.getSystemId()).thenReturn(1L);
+    when(config.getSystemStartTime()).thenReturn(System.currentTimeMillis());
+    when(config.getSystemDirectoryPath())
+        .thenReturn(this.temporaryFolder.getRoot().getAbsolutePath());
+    when(config.getProductDescription()).thenReturn(this.testName.getMethodName());
+    when(config.getArchiveDiskSpaceLimit()).thenReturn(0L);
+
+    this.sampleCollector = new SampleCollector(sampler);
+    this.sampleCollector.initialize(config, this.timer.getTime());
+
+    this.timer.reset();
+    this.nanosTimeStamp = this.timer.getLastResetTime() - getNanoRate();
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    StatisticsTypeFactoryImpl.clear();
+  }
+
+  @Test
+  public void closedInstanceShouldHaveTypeInRolledArchives() throws Exception {
+    // initial state
+    verifyStatisticsTypeIsInArchiveFile(archiveFile(), 0);
+
+    // one sample
+    sample(advanceNanosTimeStamp());
+    verifyStatisticsTypeIsInArchiveFile(archiveFile(), 1);
+
+    // close stats
+    this.statistics.close();
+
+    assertThat(archiveFile(1)).doesNotExist();
+
+    // cause roll
+    sampleUntilFileExists(archiveFile(1));
+
+    sample(advanceNanosTimeStamp());
+    verifyStatisticsTypeIsInArchiveFile(archiveFile(), 0);
+
+    this.statistics = factory.createAtomicStatistics(this.statisticsType, "statistics1", 2);
+
+    sample(advanceNanosTimeStamp());
+    verifyStatisticsTypeIsInArchiveFile(archiveFile(), 1); // should be corrupt?
+
+  }
+
+  private void verifyStatisticsTypeIsInArchiveFile(final File archiveFile,
+      final int expectedResources) throws IOException {
+    StatArchiveReader reader = new StatArchiveReader(new File[] {archiveFile}, null, false);
+
+    // compare all resourceInst values against what was printed above
+
+    List<ResourceInst> resources = reader.getResourceInstList();
+    if (expectedResources > 0) {
+      assertThat(resources).hasAtLeastOneElementOfType(ResourceInst.class);
+    }
+
+    for (ResourceInst resourceInstance : resources) {
+      if (resourceInstance == null)
+        continue;
+      assertThat(resourceInstance.getName()).isNotNull();
+      assertThat(resourceInstance.getType()).isNotNull();
+      assertThat(resourceInstance.getType().getName()).isEqualTo(this.statisticsType.getName());
+    }
+  }
+
+  private void sampleUntilFileExists(final File file)
+      throws InterruptedException, TimeoutException {
+    long timeout = System.nanoTime() + MINUTES.toNanos(1);
+    int count = 0;
+    do {
+      sample(advanceNanosTimeStamp());
+      count++;
+    } while (!file.exists() && System.nanoTime() < timeout);
+    if (!file.exists()) {
+      throw new TimeoutException("File " + file + " does not exist within " + 1 + " " + MINUTES);
+    }
+  }
+
+  private void sample(final long time) {
+    getSampleCollector().sample(time);
+  }
+
+  private SampleCollector getSampleCollector() {
+    return this.sampleCollector;
+  }
+
+  private File archiveFile(final int child) {
+    return new File(this.dir,
+        this.testName.getMethodName() + "-01-" + String.format("%02d", child) + ".gfs");
+  }
+
+  private File archiveFile() {
+    return new File(this.archiveFileName);
+  }
+
+  private long advanceNanosTimeStamp() {
+    this.nanosTimeStamp += getNanoRate();
+    return this.nanosTimeStamp;
+  }
+
+  private long getNanoRate() {
+    return NanoTimer.millisToNanos(getSampleRate());
+  }
+
+  private long getSampleRate() {
+    return 1000; // 1 second
+  }
+}