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/06/16 17:23:24 UTC

[04/23] incubator-geode git commit: GEODE-1494: Allowing stats to be measured with callbacks

GEODE-1494: Allowing stats to be measured with callbacks

Adding support to measure statistics with suppliers that are sampled
every sample interval. The suppliers are invoked in a separate thread
than the stat sampler so that slow suppliers won't screw up all stats.


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

Branch: refs/heads/feature/GEODE-835
Commit: 01848b6eea4f90e9525eb42aba4d10ba0584c415
Parents: e9ffdce
Author: Dan Smith <up...@apache.org>
Authored: Fri Jun 3 12:07:37 2016 -0700
Committer: Dan Smith <up...@apache.org>
Committed: Mon Jun 13 10:45:13 2016 -0700

----------------------------------------------------------------------
 .../java/com/gemstone/gemfire/Statistics.java   | 253 +++++++++++++++++++
 .../gemfire/internal/DummyStatisticsImpl.java   |  44 ++++
 .../gemfire/internal/HostStatSampler.java       |  10 +-
 .../gemfire/internal/StatSamplerStats.java      |  34 ++-
 .../gemfire/internal/StatisticsImpl.java        | 143 ++++++++++-
 .../gemfire/internal/StatisticsTypeImpl.java    |   6 +-
 .../internal/statistics/CallbackSampler.java    |  96 +++++++
 .../internal/statistics/SampleCollector.java    |   4 +-
 .../internal/LocalStatisticsImplJUnitTest.java  | 139 ++++++++++
 .../StatArchiveWriterReaderJUnitTest.java       |  16 +-
 .../gemfire/internal/StatSamplerJUnitTest.java  |  34 ++-
 .../statistics/CallbackSamplerJUnitTest.java    |  98 +++++++
 .../internal/statistics/DummyStatistics.java    | 208 ---------------
 13 files changed, 842 insertions(+), 243 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/01848b6e/geode-core/src/main/java/com/gemstone/gemfire/Statistics.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/Statistics.java b/geode-core/src/main/java/com/gemstone/gemfire/Statistics.java
index d7f45a5..0b9c18e 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/Statistics.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/Statistics.java
@@ -18,6 +18,10 @@ package com.gemstone.gemfire;
 
 //import com.gemstone.gemfire.distributed.DistributedSystem;
 
+import java.util.function.DoubleSupplier;
+import java.util.function.IntSupplier;
+import java.util.function.LongSupplier;
+
 /**
  * Instances of this interface maintain the values of various application-defined
  * statistics.  The statistics themselves are described by an instance
@@ -441,4 +445,253 @@ public interface Statistics {
    *         type <code>double</code>.
    */
   public void incDouble(String name, double delta);
+
+  /**
+   * Provide a callback to compute the value of this statistic
+   * every sample interval and use that as the value of the stat.
+   * <p>
+   * The callback should return quickly because it is invoked on a shared thread.
+   * It should not do any expensive computations, network calls, or access any resources
+   * under locks that may be locked by long running processes.
+   * <p>
+   * This callback will only be invoked if the distributed system property
+   * statistic-sampling-enabled is set to true, and it will be invoked at intervals
+   * determined by the statistic-sampling-rate.
+   * <p>
+   * Get methods are not guaranteed to recompute a new value, they may return
+   * the last sampled value
+   * @param id a statistic id obtained with {@link #nameToId}
+   * or {@link StatisticsType#nameToId}.
+   * @param supplier a callback that will return the value of the stat. This
+   * replaces any previously registered supplier. If the passed in suppplier is null, it
+   * will remove any existing supplier
+   * @return the previously registered supplier, or null if there was no previously registered supplier
+   * @throws ArrayIndexOutOfBoundsException If the id is invalid.
+   * @throws IllegalArgumentException if the type of this
+   * stat is not int
+   */
+  public IntSupplier setIntSupplier(int id, IntSupplier supplier);
+
+  /**
+   * Provide a callback to compute the value of this statistic
+   * every sample interval and use that as the value of the stat.
+   * <p>
+   * The callback should return quickly because it is invoked on a shared thread.
+   * It should not do any expensive computations, network calls, or access any resources
+   * under locks that may be locked by long running processes.
+   * <p>
+   * This callback will only be invoked if the distributed system property
+   * statistic-sampling-enabled is set to true, and it will be invoked at intervals
+   * determined by the statistic-sampling-rate.
+   * <p>
+   * Get methods are not guaranteed to recompute a new value, they may return
+   * the last sampled value
+   * @param name the nane of the statistic to update
+   * @param supplier a callback that will return the value of the stat. This
+   * replaces any previously registered supplier. If the passed in suppplier is null, it
+   * will remove any existing supplier
+   * @return the previously registered supplier, or null if there was no previously registered supplier
+   * @throws IllegalArgumentException
+   *         If no statistic exists with name <code>name</code> or
+   *         if the statistic named <code>name</code> is not of
+   *         type <code>int</code>.
+   * @throws IllegalArgumentException if the type of this
+   * stat is not int
+   */
+  public IntSupplier setIntSupplier(String name, IntSupplier supplier);
+
+
+  /**
+   * Provide a callback to compute the value of this statistic
+   * every sample interval and use that as the value of the stat.
+   * <p>
+   * The callback should return quickly because it is invoked on a shared thread.
+   * It should not do any expensive computations, network calls, or access any resources
+   * under locks that may be locked by long running processes.
+   * <p>
+   * This callback will only be invoked if the distributed system property
+   * statistic-sampling-enabled is set to true, and it will be invoked at intervals
+   * determined by the statistic-sampling-rate.
+   * <p>
+   * Get methods are not guaranteed to recompute a new value, they may return
+   * the last sampled value
+   * @param descriptor the descriptor of the statistic to update
+   * @param supplier a callback that will return the value of the stat. This
+   * replaces any previously registered supplier. If the passed in suppplier is null, it
+   * will remove any existing supplier
+   * @return the previously registered supplier, or null if there was no previously registered supplier
+   * @throws IllegalArgumentException
+   *         If no statistic exists with the given <code>descriptor</code> or
+   *         if the described statistic is not of
+   *         type <code>int</code>.
+   * @throws IllegalArgumentException if the type of this
+   * stat is not int
+   */
+  public IntSupplier setIntSupplier(StatisticDescriptor descriptor, IntSupplier supplier);
+
+  /**
+   * Provide a callback to compute the value of this statistic
+   * every sample interval and use that as the value of the stat.
+   * <p>
+   * The callback should return quickly because it is invoked on a shared thread.
+   * It should not do any expensive computations, network calls, or access any resources
+   * under locks that may be locked by long running processes.
+   * <p>
+   * This callback will only be invoked if the distributed system property
+   * statistic-sampling-enabled is set to true, and it will be invoked at intervals
+   * determined by the statistic-sampling-rate.
+   * <p>
+   * Get methods are not guaranteed to recompute a new value, they may return
+   * the last sampled value
+   * @param id a statistic id obtained with {@link #nameToId}
+   * or {@link StatisticsType#nameToId}.
+   * @param supplier a callback that will return the value of the stat. This
+   * replaces any previously registered supplier. If the passed in suppplier is null, it
+   * will remove any existing supplier
+   * @return the previously registered supplier, or null if there was no previously registered supplier
+   * @throws ArrayIndexOutOfBoundsException If the id is invalid.
+   * @throws IllegalArgumentException if the type of this
+   * stat is not long
+   */
+  public LongSupplier setLongSupplier(int id, LongSupplier supplier);
+
+  /**
+   * Provide a callback to compute the value of this statistic
+   * every sample interval and use that as the value of the stat.
+   * <p>
+   * The callback should return quickly because it is invoked on a shared thread.
+   * It should not do any expensive computations, network calls, or access any resources
+   * under locks that may be locked by long running processes.
+   * <p>
+   * This callback will only be invoked if the distributed system property
+   * statistic-sampling-enabled is set to true, and it will be invoked at intervals
+   * determined by the statistic-sampling-rate.
+   * <p>
+   * Get methods are not guaranteed to recompute a new value, they may return
+   * the last sampled value
+   * @param name the nane of the statistic to update
+   * @param supplier a callback that will return the value of the stat. This
+   * replaces any previously registered supplier. If the passed in suppplier is null, it
+   * will remove any existing supplier
+   * @return the previously registered supplier, or null if there was no previously registered supplier
+   * @throws IllegalArgumentException
+   *         If no statistic exists with name <code>name</code> or
+   *         if the statistic named <code>name</code> is not of
+   *         type <code>long</code>.
+   * @throws IllegalArgumentException if the type of this
+   * stat is not long
+   */
+  public LongSupplier setLongSupplier(String name, LongSupplier supplier);
+
+
+  /**
+   * Provide a callback to compute the value of this statistic
+   * every sample interval and use that as the value of the stat.
+   * <p>
+   * The callback should return quickly because it is invoked on a shared thread.
+   * It should not do any expensive computations, network calls, or access any resources
+   * under locks that may be locked by long running processes.
+   * <p>
+   * This callback will only be invoked if the distributed system property
+   * statistic-sampling-enabled is set to true, and it will be invoked at intervals
+   * determined by the statistic-sampling-rate.
+   * <p>
+   * Get methods are not guaranteed to recompute a new value, they may return
+   * the last sampled value
+   * @param descriptor the descriptor of the statistic to update
+   * @param supplier a callback that will return the value of the stat. This
+   * replaces any previously registered supplier. If the passed in suppplier is null, it
+   * will remove any existing supplier
+   * @return the previously registered supplier, or null if there was no previously registered supplier
+   * @throws IllegalArgumentException
+   *         If no statistic exists with the given <code>descriptor</code> or
+   *         if the described statistic is not of
+   *         type <code>long</code>.
+   * @throws IllegalArgumentException if the type of this
+   * stat is not long
+   */
+  public LongSupplier setLongSupplier(StatisticDescriptor descriptor, LongSupplier supplier);
+
+  /**
+   * Provide a callback to compute the value of this statistic
+   * every sample interval and use that as the value of the stat.
+   * <p>
+   * The callback should return quickly because it is invoked on a shared thread.
+   * It should not do any expensive computations, network calls, or access any resources
+   * under locks that may be locked by double running processes.
+   * <p>
+   * This callback will only be invoked if the distributed system property
+   * statistic-sampling-enabled is set to true, and it will be invoked at intervals
+   * determined by the statistic-sampling-rate.
+   * <p>
+   * Get methods are not guaranteed to recompute a new value, they may return
+   * the last sampled value
+   * @param id a statistic id obtained with {@link #nameToId}
+   * or {@link StatisticsType#nameToId}.
+   * @param supplier a callback that will return the value of the stat. This
+   * replaces any previously registered supplier. If the passed in suppplier is null, it
+   * will remove any existing supplier
+   * @return the previously registered supplier, or null if there was no previously registered supplier
+   * @throws ArrayIndexOutOfBoundsException If the id is invalid.
+   * @throws IllegalArgumentException if the type of this
+   * stat is not double
+   */
+  public DoubleSupplier setDoubleSupplier(int id, DoubleSupplier supplier);
+
+  /**
+   * Provide a callback to compute the value of this statistic
+   * every sample interval and use that as the value of the stat.
+   * <p>
+   * The callback should return quickly because it is invoked on a shared thread.
+   * It should not do any expensive computations, network calls, or access any resources
+   * under locks that may be locked by double running processes.
+   * <p>
+   * This callback will only be invoked if the distributed system property
+   * statistic-sampling-enabled is set to true, and it will be invoked at intervals
+   * determined by the statistic-sampling-rate.
+   * <p>
+   * Get methods are not guaranteed to recompute a new value, they may return
+   * the last sampled value
+   * @param name the nane of the statistic to update
+   * @param supplier a callback that will return the value of the stat. This
+   * replaces any previously registered supplier. If the passed in suppplier is null, it
+   * will remove any existing supplier
+   * @return the previously registered supplier, or null if there was no previously registered supplier
+   * @throws IllegalArgumentException
+   *         If no statistic exists with name <code>name</code> or
+   *         if the statistic named <code>name</code> is not of
+   *         type <code>double</code>.
+   * @throws IllegalArgumentException if the type of this
+   * stat is not double
+   */
+  public DoubleSupplier setDoubleSupplier(String name, DoubleSupplier supplier);
+
+
+  /**
+   * Provide a callback to compute the value of this statistic
+   * every sample interval and use that as the value of the stat.
+   * <p>
+   * The callback should return quickly because it is invoked on a shared thread.
+   * It should not do any expensive computations, network calls, or access any resources
+   * under locks that may be locked by double running processes.
+   * <p>
+   * This callback will only be invoked if the distributed system property
+   * statistic-sampling-enabled is set to true, and it will be invoked at intervals
+   * determined by the statistic-sampling-rate.
+   * <p>
+   * Get methods are not guaranteed to recompute a new value, they may return
+   * the last sampled value
+   * @param descriptor the descriptor of the statistic to update
+   * @param supplier a callback that will return the value of the stat. This
+   * replaces any previously registered supplier. If the passed in suppplier is null, it
+   * will remove any existing supplier
+   * @return the previously registered supplier, or null if there was no previously registered supplier
+   * @throws IllegalArgumentException
+   *         If no statistic exists with the given <code>descriptor</code> or
+   *         if the described statistic is not of
+   *         type <code>double</code>.
+   * @throws IllegalArgumentException if the type of this
+   * stat is not double
+   */
+  public DoubleSupplier setDoubleSupplier(StatisticDescriptor descriptor, DoubleSupplier supplier);
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/01848b6e/geode-core/src/main/java/com/gemstone/gemfire/internal/DummyStatisticsImpl.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/DummyStatisticsImpl.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/DummyStatisticsImpl.java
index 83d4055..aa8da50 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/DummyStatisticsImpl.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/DummyStatisticsImpl.java
@@ -16,6 +16,10 @@
  */
 package com.gemstone.gemfire.internal;
 
+import java.util.function.DoubleSupplier;
+import java.util.function.IntSupplier;
+import java.util.function.LongSupplier;
+
 import com.gemstone.gemfire.*;
 
 /**
@@ -195,4 +199,44 @@ public class DummyStatisticsImpl implements Statistics {
 
   public final void incDouble(String name, double delta) {
   }
+
+  @Override
+  public IntSupplier setIntSupplier(final int id, final IntSupplier supplier) {
+    return null;
+  }
+
+  @Override
+  public IntSupplier setIntSupplier(final String name, final IntSupplier supplier) {
+    return null;
+  }
+
+  @Override
+  public IntSupplier setIntSupplier(final StatisticDescriptor descriptor, final IntSupplier supplier) {
+    return null;
+  }
+
+  @Override public LongSupplier setLongSupplier(final int id, final LongSupplier supplier) {
+    return null;
+  }
+
+  @Override public LongSupplier setLongSupplier(final String name, final LongSupplier supplier) {
+    return null;
+  }
+
+  @Override public LongSupplier setLongSupplier(final StatisticDescriptor descriptor, final LongSupplier supplier) {
+    return null;
+  }
+
+  @Override public DoubleSupplier setDoubleSupplier(final int id, final DoubleSupplier supplier) {
+    return null;
+  }
+
+  @Override public DoubleSupplier setDoubleSupplier(final String name, final DoubleSupplier supplier) {
+    return null;
+  }
+
+  @Override
+  public DoubleSupplier setDoubleSupplier(final StatisticDescriptor descriptor, final DoubleSupplier supplier) {
+    return null;
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/01848b6e/geode-core/src/main/java/com/gemstone/gemfire/internal/HostStatSampler.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/HostStatSampler.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/HostStatSampler.java
index a5a07ca..38fc83d 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/HostStatSampler.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/HostStatSampler.java
@@ -26,6 +26,7 @@ import com.gemstone.gemfire.internal.logging.LogService;
 import com.gemstone.gemfire.internal.logging.LoggingThreadGroup;
 import com.gemstone.gemfire.internal.logging.log4j.LocalizedMessage;
 import com.gemstone.gemfire.internal.logging.log4j.LogMarker;
+import com.gemstone.gemfire.internal.statistics.CallbackSampler;
 import com.gemstone.gemfire.internal.statistics.SampleCollector;
 import com.gemstone.gemfire.internal.statistics.StatArchiveHandlerConfig;
 import com.gemstone.gemfire.internal.statistics.StatisticsSampler;
@@ -35,6 +36,7 @@ import org.apache.logging.log4j.Logger;
 import java.io.File;
 import java.net.UnknownHostException;
 import java.util.List;
+import java.util.concurrent.TimeUnit;
 
 /**
  * HostStatSampler implements a thread which will monitor, sample, and archive
@@ -50,7 +52,7 @@ public abstract class HostStatSampler
   public static final String OS_STATS_DISABLED_PROPERTY = "osStatsDisabled";
 
   protected static final String INITIALIZATION_TIMEOUT_PROPERTY = DistributionConfig.GEMFIRE_PREFIX + "statSamplerInitializationTimeout";
-  protected static final int INITIALIZATION_TIMEOUT_DEFAULT = 3000;
+  protected static final int INITIALIZATION_TIMEOUT_DEFAULT = 30000;
   protected static final long INITIALIZATION_TIMEOUT_MILLIS = 
       Long.getLong(INITIALIZATION_TIMEOUT_PROPERTY, INITIALIZATION_TIMEOUT_DEFAULT);
   
@@ -82,6 +84,8 @@ public abstract class HostStatSampler
   private final StoppableCountDownLatch statSamplerInitializedLatch;
 
   private final CancelCriterion stopper;
+
+  private final CallbackSampler callbackSampler;
   
   protected HostStatSampler(CancelCriterion stopper, 
                             StatSamplerStats samplerStats) {
@@ -89,6 +93,7 @@ public abstract class HostStatSampler
     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);
   }
   
   public final StatSamplerStats getStatSamplerStats() {
@@ -276,6 +281,8 @@ public abstract class HostStatSampler
       }  
       ThreadGroup group = 
         LoggingThreadGroup.createThreadGroup("StatSampler Threads");
+
+      this.callbackSampler.start(getStatisticsManager(), group, getSampleRate(), TimeUnit.MILLISECONDS);
       statThread = new Thread(group, this);
       statThread.setName(statThread.getName() + " StatSampler");
       statThread.setPriority(Thread.MAX_PRIORITY);
@@ -298,6 +305,7 @@ public abstract class HostStatSampler
   }
   private final void stop(boolean interruptIfAlive) {
     synchronized (HostStatSampler.class) {
+      this.callbackSampler.stop();
       if ( statThread == null) {
         return; 
       }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/01848b6e/geode-core/src/main/java/com/gemstone/gemfire/internal/StatSamplerStats.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/StatSamplerStats.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/StatSamplerStats.java
index c702479..bbe333c 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/StatSamplerStats.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/StatSamplerStats.java
@@ -27,13 +27,19 @@ public class StatSamplerStats {
   public final static String DELAY_DURATION = "delayDuration"; // int
   public final static String STAT_RESOURCES = "statResources"; // int
   public final static String JVM_PAUSES = "jvmPauses"; // int
-  
+  public final static String SAMPLE_CALLBACKS = "sampleCallbacks"; // int
+  public final static String SAMPLE_CALLBACK_ERRORS = "sampleCallbackErrors"; // int
+  public final static String SAMPLE_CALLBACK_DURATION = "sampleCallbackDuration"; // long
+
   private final static StatisticsType samplerType;
   private final static int sampleCountId;
   private final static int sampleTimeId;
   private final static int delayDurationId;
   private final static int statResourcesId;
   private final static int jvmPausesId;
+  private final static int sampleCallbacksId;
+  private final static int sampleCallbackErrorsId;
+  private final static int sampleCallbackDurationId;
   static {
     StatisticsTypeFactory f = StatisticsTypeFactoryImpl.singleton();
     samplerType = f.createType("StatSampler",
@@ -54,12 +60,24 @@ public class StatSamplerStats {
                                  f.createIntCounter(JVM_PAUSES,
                                                     "Total number of JVM pauses (which may or may not be full GC pauses) detected by this sampler. A JVM pause is defined as a system event which kept the statistics sampler thread from sampling for 3000 or more milliseconds. This threshold can be customized by setting the system property gemfire.statSamplerDelayThreshold (units are milliseconds).",
                                                     "jvmPauses", false),
+                                 f.createIntGauge(SAMPLE_CALLBACKS,
+                                   "Total number of statistics that are sampled using callbacks.",
+                                   "resources", false),
+                                 f.createIntCounter(SAMPLE_CALLBACK_ERRORS,
+                                   "Total number of exceptions thrown by callbacks when performing sampling",
+                                   "errors", false),
+                                 f.createLongCounter(SAMPLE_CALLBACK_DURATION,
+                                   "Total amount of time invoking sampling callbacks",
+                                   "milliseconds", false),
                                });
     sampleCountId = samplerType.nameToId(SAMPLE_COUNT);
     sampleTimeId = samplerType.nameToId(SAMPLE_TIME);
     delayDurationId = samplerType.nameToId(DELAY_DURATION);
     statResourcesId = samplerType.nameToId(STAT_RESOURCES);
     jvmPausesId = samplerType.nameToId(JVM_PAUSES);
+    sampleCallbacksId = samplerType.nameToId(SAMPLE_CALLBACKS);
+    sampleCallbackErrorsId = samplerType.nameToId(SAMPLE_CALLBACK_ERRORS);
+    sampleCallbackDurationId = samplerType.nameToId(SAMPLE_CALLBACK_DURATION);
   }
 
   private final Statistics samplerStats;
@@ -78,7 +96,19 @@ public class StatSamplerStats {
   public void incJvmPauses() {
     this.samplerStats.incInt(jvmPausesId, 1);
   }
-  
+
+  public void incSampleCallbackErrors(int delta) {
+    this.samplerStats.incInt(sampleCallbackErrorsId, delta);
+  }
+
+  public void setSampleCallbacks(int count) {
+    this.samplerStats.setInt(sampleCallbacksId, count);
+  }
+
+  public void incSampleCallbackDuration(long delta) {
+    this.samplerStats.incLong(sampleCallbackDurationId, delta);
+  }
+
   public int getSampleCount() {
     return this.samplerStats.getInt(SAMPLE_COUNT);
   }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/01848b6e/geode-core/src/main/java/com/gemstone/gemfire/internal/StatisticsImpl.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/StatisticsImpl.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/StatisticsImpl.java
index 2344d93..04bfc01 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/StatisticsImpl.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/StatisticsImpl.java
@@ -17,11 +17,23 @@
 package com.gemstone.gemfire.internal;
 
 //import com.gemstone.gemfire.distributed.DistributedSystem;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.DoubleSupplier;
+import java.util.function.IntSupplier;
+import java.util.function.LongSupplier;
+import java.util.function.Supplier;
+
 import com.gemstone.gemfire.StatisticDescriptor;
 import com.gemstone.gemfire.Statistics;
 import com.gemstone.gemfire.StatisticsType;
 import com.gemstone.gemfire.internal.concurrent.Atomics;
 import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
+import com.gemstone.gemfire.internal.logging.LogService;
+import com.gemstone.gemfire.internal.util.concurrent.CopyOnWriteHashMap;
+
+import org.apache.logging.log4j.Logger;
 
 // @todo darrel Add statistics instances to archive when they are created. 
 /**
@@ -42,6 +54,8 @@ import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
  * @since GemFire 3.0
  */
 public abstract class StatisticsImpl implements Statistics {
+  /** logger - not private for tests */
+  static Logger logger = LogService.getLogger();
 
   /** The type of this statistics instance */
   private final StatisticsTypeImpl type;
@@ -61,6 +75,25 @@ public abstract class StatisticsImpl implements Statistics {
   /** Uniquely identifies this instance */
   private long uniqueId;
 
+  /**
+   * Suppliers of int sample values to be sampled every sample-interval
+   */
+  private final CopyOnWriteHashMap<Integer, IntSupplier> intSuppliers = new CopyOnWriteHashMap<>();
+  /**
+   * Suppliers of long sample values to be sampled every sample-interval
+   */
+  private final CopyOnWriteHashMap<Integer, LongSupplier> longSuppliers = new CopyOnWriteHashMap<>();
+  /**
+   * Suppliers of double sample values to be sampled every sample-interval
+   */
+  private final CopyOnWriteHashMap<Integer, DoubleSupplier> doubleSuppliers = new CopyOnWriteHashMap<>();
+
+  /**
+   * Suppliers that have previously failed. Tracked to avoid logging many messages about
+   * a failing supplier
+   */
+  private final Set<Object> flakySuppliers = new HashSet<Object>();
+
   ///////////////////////  Constructors  ///////////////////////
 
   /** factory method to create a class that implements Statistics
@@ -362,6 +395,12 @@ public abstract class StatisticsImpl implements Statistics {
   }
 
   /**
+   * Increments the value of the statistic of type <code>double</code> at
+   * the given offset by a given amount, but performs no type checking.
+   */
+  protected abstract void _incDouble(int offset, double delta);
+
+  /**
    * For internal use only.
    * Tells the implementation to prepare the data in this instance
    * for sampling.
@@ -372,10 +411,108 @@ public abstract class StatisticsImpl implements Statistics {
   }
 
   /**
-   * Increments the value of the statistic of type <code>double</code> at
-   * the given offset by a given amount, but performs no type checking.
+   * Invoke sample suppliers to retrieve the current value for
+   * the suppler controlled sets and update the stats to reflect
+   * the supplied values.
+   * @return the number of callback errors that occurred while
+   * sampling stats
    */
-  protected abstract void _incDouble(int offset, double delta);
+  public int invokeSuppliers() {
+    int errors = 0;
+    for(Map.Entry<Integer, IntSupplier> entry: intSuppliers.entrySet()) {
+      try {
+        _setInt(entry.getKey(), entry.getValue().getAsInt());
+      } catch(Throwable t) {
+        logSupplierError(t, entry.getKey(), entry.getValue());
+        errors++;
+      }
+    }
+    for(Map.Entry<Integer, LongSupplier> entry: longSuppliers.entrySet()) {
+      try {
+        _setLong(entry.getKey(), entry.getValue().getAsLong());
+      } catch(Throwable t) {
+        logSupplierError(t, entry.getKey(), entry.getValue());
+        errors++;
+      }
+    }
+    for(Map.Entry<Integer, DoubleSupplier> entry: doubleSuppliers.entrySet()) {
+      try {
+        _setDouble(entry.getKey(), entry.getValue().getAsDouble());
+      } catch(Throwable t) {
+        logSupplierError(t, entry.getKey(), entry.getValue());
+        errors++;
+      }
+    }
+
+    return errors;
+  }
+
+  private void logSupplierError(final Throwable t, int statId, Object supplier) {
+    if(flakySuppliers.add(supplier)) {
+      logger.warn("Error invoking supplier for stat {}, id {}", this.getTextId(), statId, t);
+    }
+  }
+
+  /**
+   * @return the number of statistics that are measured using supplier callbacks
+   */
+  public int getSupplierCount() {
+    return intSuppliers.size() + doubleSuppliers.size() + longSuppliers.size();
+  }
+
+  @Override
+  public IntSupplier setIntSupplier(final int id, final IntSupplier supplier) {
+    if(id >= type.getIntStatCount()) {
+      throw new IllegalArgumentException("Id " + id + " is not in range for stat" + type);
+    }
+    return intSuppliers.put(id, supplier);
+  }
+
+  @Override
+  public IntSupplier setIntSupplier(final String name, final IntSupplier supplier) {
+    return setIntSupplier(nameToId(name), supplier);
+  }
+
+  @Override
+  public IntSupplier setIntSupplier(final StatisticDescriptor descriptor, final IntSupplier supplier) {
+    return setIntSupplier(getIntId(descriptor), supplier);
+  }
+
+  @Override
+  public LongSupplier setLongSupplier(final int id, final LongSupplier supplier) {
+    if(id >= type.getLongStatCount()) {
+      throw new IllegalArgumentException("Id " + id + " is not in range for stat" + type);
+    }
+    return longSuppliers.put(id, supplier);
+  }
+
+  @Override
+  public LongSupplier setLongSupplier(final String name, final LongSupplier supplier) {
+    return setLongSupplier(nameToId(name), supplier);
+  }
+
+  @Override
+  public LongSupplier setLongSupplier(final StatisticDescriptor descriptor, final LongSupplier supplier) {
+    return setLongSupplier(getLongId(descriptor), supplier);
+  }
+
+  @Override
+  public DoubleSupplier setDoubleSupplier(final int id, final DoubleSupplier supplier) {
+    if(id >= type.getDoubleStatCount()) {
+      throw new IllegalArgumentException("Id " + id + " is not in range for stat" + type);
+    }
+    return doubleSuppliers.put(id, supplier);
+  }
+
+  @Override
+  public DoubleSupplier setDoubleSupplier(final String name, final DoubleSupplier supplier) {
+    return setDoubleSupplier(nameToId(name), supplier);
+  }
+
+  @Override
+  public DoubleSupplier setDoubleSupplier(final StatisticDescriptor descriptor, final DoubleSupplier supplier) {
+    return setDoubleSupplier(getDoubleId(descriptor), supplier);
+  }
 
   @Override
   public int hashCode() {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/01848b6e/geode-core/src/main/java/com/gemstone/gemfire/internal/StatisticsTypeImpl.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/StatisticsTypeImpl.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/StatisticsTypeImpl.java
index 2a29d46..ccd4d6c 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/StatisticsTypeImpl.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/StatisticsTypeImpl.java
@@ -192,19 +192,19 @@ public class StatisticsTypeImpl implements StatisticsType {
   /**
    * Gets the number of statistics in this type that are ints.
    */
-  public final int getIntStatCount() {
+  public int getIntStatCount() {
     return this.intStatCount;
   }
   /**
    * Gets the number of statistics in this type that are longs.
    */
-  public final int getLongStatCount() {
+  public int getLongStatCount() {
     return this.longStatCount;
   }
   /**
    * Gets the number of statistics that are doubles.
    */
-  public final int getDoubleStatCount() {
+  public int getDoubleStatCount() {
     return this.doubleStatCount;
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/01848b6e/geode-core/src/main/java/com/gemstone/gemfire/internal/statistics/CallbackSampler.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/statistics/CallbackSampler.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/statistics/CallbackSampler.java
new file mode 100644
index 0000000..0180fba
--- /dev/null
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/statistics/CallbackSampler.java
@@ -0,0 +1,96 @@
+/*
+ * 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 com.gemstone.gemfire.internal.statistics;
+
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+
+import com.gemstone.gemfire.CancelCriterion;
+import com.gemstone.gemfire.Statistics;
+import com.gemstone.gemfire.SystemFailure;
+import com.gemstone.gemfire.internal.StatSamplerStats;
+import com.gemstone.gemfire.internal.StatisticsImpl;
+import com.gemstone.gemfire.internal.StatisticsManager;
+import com.gemstone.gemfire.internal.logging.LogService;
+
+import org.apache.logging.log4j.Logger;
+
+public class CallbackSampler {
+  private static final Logger logger = LogService.getLogger();
+  private StatisticsManager statisticsManager;
+  private final CancelCriterion cancelCriterion;
+  private long sampleIntervalNanos;
+  private ScheduledExecutorService executor;
+  private final StatSamplerStats statSamplerStats;
+
+  public CallbackSampler( final CancelCriterion cancelCriterion,
+                         final StatSamplerStats statSamplerStats)
+  {
+    this.cancelCriterion = cancelCriterion;
+    this.statSamplerStats = statSamplerStats;
+  }
+
+  public void start(StatisticsManager statisticsManager, ThreadGroup threadGroup, int sampleInterval, TimeUnit timeUnit) {
+    ScheduledExecutorService executor = Executors.newSingleThreadScheduledExecutor(runnable -> {
+      Thread thread = new Thread(threadGroup, runnable, "CallbackSampler");
+      thread.setDaemon(true);
+      return thread;
+    });
+    start(executor, statisticsManager, sampleInterval, timeUnit);
+  }
+
+  void start(ScheduledExecutorService executor, StatisticsManager statisticsManager, int sampleInterval, TimeUnit timeUnit) {
+    stop();
+    this.statisticsManager = statisticsManager;
+    this.executor = executor;
+
+    executor.scheduleAtFixedRate(() -> sampleCallbacks(), sampleInterval, sampleInterval, timeUnit);
+  }
+
+  private void sampleCallbacks() {
+    if(cancelCriterion.cancelInProgress() != null) {
+      executor.shutdown();
+    }
+    int errors = 0;
+    int suppliers = 0;
+    long start = System.nanoTime();
+    try {
+      for (Statistics stats : statisticsManager.getStatsList()) {
+        StatisticsImpl statistics = (StatisticsImpl) stats;
+        errors += statistics.invokeSuppliers();
+        suppliers += statistics.getSupplierCount();
+      }
+    } catch(VirtualMachineError e) {
+      SystemFailure.initiateFailure(e);
+    } catch (Throwable throwable) {
+      logger.error("Error invoking statistic suppliers", throwable);
+    } finally {
+      long end = System.nanoTime();
+      statSamplerStats.incSampleCallbackDuration(TimeUnit.NANOSECONDS.toMillis(end - start));
+      statSamplerStats.incSampleCallbackErrors(errors);
+      statSamplerStats.setSampleCallbacks(suppliers);
+    }
+
+  }
+
+  public void stop() {
+    if(executor != null) {
+      this.executor.shutdown();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/01848b6e/geode-core/src/main/java/com/gemstone/gemfire/internal/statistics/SampleCollector.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/statistics/SampleCollector.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/statistics/SampleCollector.java
index dfb502b..6a3b421 100755
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/statistics/SampleCollector.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/statistics/SampleCollector.java
@@ -286,7 +286,9 @@ public class SampleCollector {
           handler.close();
         }
       }
-      instance = null;
+      if(instance == this) {
+        instance = null;
+      }
     }
   }
   

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/01848b6e/geode-core/src/test/java/com/gemstone/gemfire/internal/LocalStatisticsImplJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/LocalStatisticsImplJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/LocalStatisticsImplJUnitTest.java
new file mode 100644
index 0000000..a9b1ec7
--- /dev/null
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/LocalStatisticsImplJUnitTest.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 com.gemstone.gemfire.internal;
+
+import static org.junit.Assert.assertEquals;
+import static org.mockito.Mockito.*;
+
+import java.util.function.DoubleSupplier;
+import java.util.function.IntSupplier;
+import java.util.function.LongSupplier;
+
+import com.gemstone.gemfire.StatisticsType;
+import com.gemstone.gemfire.internal.LocalStatisticsImpl;
+import com.gemstone.gemfire.internal.StatisticsImpl;
+import com.gemstone.gemfire.internal.StatisticsManager;
+import com.gemstone.gemfire.internal.StatisticsTypeImpl;
+import com.gemstone.gemfire.test.junit.categories.UnitTest;
+
+import org.apache.commons.lang.ObjectUtils.Null;
+import org.apache.logging.log4j.Logger;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.rules.ExpectedException;
+
+@Category(UnitTest.class)
+public class LocalStatisticsImplJUnitTest {
+  @Rule
+  public ExpectedException thrown = ExpectedException.none();
+
+  private StatisticsImpl stats;
+
+  @Before
+  public void createStats() {
+    final StatisticsTypeImpl type = mock(StatisticsTypeImpl.class);
+    when(type.getIntStatCount()).thenReturn(5);
+    when(type.getDoubleStatCount()).thenReturn(5);
+    when(type.getLongStatCount()).thenReturn(5);
+    final String textId = "";
+    final long numbericId = 0;
+    final long uniqueId = 0;
+    final int osStatFlags = 0;
+    final boolean atomicIncrements = false;
+    final StatisticsManager system = mock(StatisticsManager.class);
+    stats = new LocalStatisticsImpl(type, textId, numbericId, uniqueId, atomicIncrements, osStatFlags, system);
+  }
+
+  @Test
+  public void invokeIntSuppliersShouldUpdateStats() {
+    IntSupplier supplier1 = mock(IntSupplier.class);
+    when(supplier1.getAsInt()).thenReturn(23);
+    stats.setIntSupplier(4, supplier1);
+    assertEquals(0, stats.invokeSuppliers());
+
+    verify(supplier1).getAsInt();
+    assertEquals(23, stats.getInt(4));
+  }
+
+  @Test
+  public void invokeLongSuppliersShouldUpdateStats() {
+    LongSupplier supplier1 = mock(LongSupplier.class);
+    when(supplier1.getAsLong()).thenReturn(23L);
+    stats.setLongSupplier(4, supplier1);
+    assertEquals(0, stats.invokeSuppliers());
+
+    verify(supplier1).getAsLong();
+    assertEquals(23L, stats.getLong(4));
+  }
+
+  @Test
+  public void invokeDoubleSuppliersShouldUpdateStats() {
+    DoubleSupplier supplier1 = mock(DoubleSupplier.class);
+    when(supplier1.getAsDouble()).thenReturn(23.3);
+    stats.setDoubleSupplier(4, supplier1);
+    assertEquals(0, stats.invokeSuppliers());
+
+    verify(supplier1).getAsDouble();
+    assertEquals(23.3, stats.getDouble(4), 0.1f);
+  }
+
+  @Test
+  public void getSupplierCountShouldReturnCorrectCount() {
+    IntSupplier supplier1 = mock(IntSupplier.class);
+    stats.setIntSupplier(4, supplier1);
+    assertEquals(1, stats.getSupplierCount());
+  }
+
+  @Test
+  public void invokeSuppliersShouldCatchSupplierErrorsAndReturnCount() {
+    IntSupplier supplier1 = mock(IntSupplier.class);
+    when(supplier1.getAsInt()).thenThrow(NullPointerException.class);
+    stats.setIntSupplier(4, supplier1);
+    assertEquals(1, stats.invokeSuppliers());
+
+    verify(supplier1).getAsInt();
+  }
+
+  @Test
+  public void invokeSuppliersShouldLogErrorOnlyOnce() {
+    final Logger originalLogger = StatisticsImpl.logger;
+    try {
+      final Logger logger = mock(Logger.class);
+      StatisticsImpl.logger = logger;
+      IntSupplier supplier1 = mock(IntSupplier.class);
+      when(supplier1.getAsInt()).thenThrow(NullPointerException.class);
+      stats.setIntSupplier(4, supplier1);
+      assertEquals(1, stats.invokeSuppliers());
+      verify(logger, times(1)).warn(anyString(), anyString(), anyInt(), isA(NullPointerException.class));
+      assertEquals(1, stats.invokeSuppliers());
+      //Make sure the logger isn't invoked again
+      verify(logger, times(1)).warn(anyString(), anyString(), anyInt(), isA(NullPointerException.class));
+    } finally {
+      StatisticsImpl.logger = originalLogger;
+    }
+  }
+
+  @Test
+  public void badSupplierParamShouldThrowError() {
+    IntSupplier supplier1 = mock(IntSupplier.class);
+    when(supplier1.getAsInt()).thenReturn(23);
+    thrown.expect(IllegalArgumentException.class);
+    stats.setIntSupplier(23, supplier1);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/01848b6e/geode-core/src/test/java/com/gemstone/gemfire/internal/StatArchiveWriterReaderJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/StatArchiveWriterReaderJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/StatArchiveWriterReaderJUnitTest.java
index bb15ff2..ea9ff4b 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/StatArchiveWriterReaderJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/StatArchiveWriterReaderJUnitTest.java
@@ -21,6 +21,7 @@ import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
+import static org.mockito.Mockito.*;
 
 import java.io.BufferedInputStream;
 import java.io.File;
@@ -47,7 +48,6 @@ import com.gemstone.gemfire.i18n.LogWriterI18n;
 import com.gemstone.gemfire.internal.StatArchiveReader.StatValue;
 import com.gemstone.gemfire.internal.logging.InternalLogWriter;
 import com.gemstone.gemfire.internal.logging.PureLogWriter;
-import com.gemstone.gemfire.internal.statistics.DummyStatistics;
 import com.gemstone.gemfire.internal.statistics.ResourceInstance;
 import com.gemstone.gemfire.internal.statistics.ResourceType;
 import com.gemstone.gemfire.internal.statistics.SampleCollector;
@@ -1575,7 +1575,8 @@ public class StatArchiveWriterReaderJUnitTest {
     
     final StatisticsType statsType = createDummyStatisticsType();
     final ResourceType rt = new ResourceType(0, statsType);
-    final Statistics statistics = new ClosedStatistics();
+    final Statistics statistics = mock(Statistics.class);
+    when(statistics.isClosed()).thenReturn(true);
     final ResourceInstance ri = new ResourceInstance(0, statistics, rt);
     
     // if bug #45377 still existed, this call would throw IllegalStateException
@@ -1602,7 +1603,7 @@ public class StatArchiveWriterReaderJUnitTest {
     
     final StatisticsType statsType = createDummyStatisticsType();
     final ResourceType rt = new ResourceType(0, statsType);
-    final Statistics statistics = new DummyStatistics();
+    final Statistics statistics = mock(Statistics.class);
     final ResourceInstance ri = new ResourceInstance(0, statistics, rt);
 
     writer.sampled(TestStatArchiveWriter.WRITER_INITIAL_DATE_MILLIS + 1000,
@@ -1709,14 +1710,7 @@ public class StatArchiveWriterReaderJUnitTest {
       }
     };
   }
-  
-  private static class ClosedStatistics extends DummyStatistics {
-    @Override
-    public boolean isClosed() {
-      return true;
-    }
-  }
-  
+
   /* [KEEP] alternative method for getting an expected golden file:
   Class clazz = getClass();
   assertNotNull(clazz);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/01848b6e/geode-core/src/test/java/com/gemstone/gemfire/internal/StatSamplerJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/StatSamplerJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/StatSamplerJUnitTest.java
index f2946b6..e16eeaf 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/StatSamplerJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/StatSamplerJUnitTest.java
@@ -117,11 +117,20 @@ public class StatSamplerJUnitTest {
         factory.createLongCounter(  "long_counter_9",   "d9",  "u9"),
         factory.createLongCounter(  "long_counter_10",  "d10", "u10", true),
         factory.createLongGauge(    "long_gauge_11",    "d11", "u11"),
-        factory.createLongGauge(    "long_gauge_12",    "d12", "u12", false)
+        factory.createLongGauge(    "long_gauge_12",    "d12", "u12", false),
+        factory.createLongGauge(    "sampled_long",    "d13", "u13", false),
+        factory.createIntGauge(    "sampled_int",    "d14", "u14", false),
+        factory.createDoubleGauge(    "sampled_double",    "d15", "u15", false)
     };
     final StatisticsType ST1 = factory.createType("ST1", "ST1", statsST1);
     final Statistics st1_1 = factory.createAtomicStatistics(ST1, "st1_1", 1);
-    
+    st1_1.setIntSupplier("sampled_int", () -> 5);
+    getOrCreateExpectedValueMap(st1_1).put("sampled_int", 5);
+    st1_1.setLongSupplier("sampled_long", () -> 6);
+    getOrCreateExpectedValueMap(st1_1).put("sampled_long", 6);
+    st1_1.setDoubleSupplier("sampled_double", () -> 7.0);
+    getOrCreateExpectedValueMap(st1_1).put("sampled_double", 7.0);
+
     boolean done = false;
 
     Statistics[] samplerStatsInstances = factory.findStatisticsByTextId("statSampler");
@@ -319,11 +328,7 @@ public class StatSamplerJUnitTest {
   
   private void incDouble(Statistics statistics, String stat, double value) {
     assertFalse(statistics.isClosed());
-    Map<String,Number> statValues = this.allStatistics.get(statistics.getTextId());
-    if (statValues == null) {
-      statValues = new HashMap<String,Number>();
-      this.allStatistics.put(statistics.getTextId(), statValues);
-    }
+    Map<String, Number> statValues = getOrCreateExpectedValueMap(statistics);
     statistics.incDouble(stat, value);
     statValues.put(stat, statistics.getDouble(stat));
     if (this.statisticTypes.get(statistics.getTextId()) == null) {
@@ -333,11 +338,7 @@ public class StatSamplerJUnitTest {
   
   private void incInt(Statistics statistics, String stat, int value) {
     assertFalse(statistics.isClosed());
-    Map<String,Number> statValues = this.allStatistics.get(statistics.getTextId());
-    if (statValues == null) {
-      statValues = new HashMap<String,Number>();
-      this.allStatistics.put(statistics.getTextId(), statValues);
-    }
+    Map<String, Number> statValues = getOrCreateExpectedValueMap(statistics);
     statistics.incInt(stat, value);
     statValues.put(stat, statistics.getInt(stat));
     if (this.statisticTypes.get(statistics.getTextId()) == null) {
@@ -345,13 +346,18 @@ public class StatSamplerJUnitTest {
     }
   }
 
-  private void incLong(Statistics statistics, String stat, long value) {
-    assertFalse(statistics.isClosed());
+  private Map<String, Number> getOrCreateExpectedValueMap(final Statistics statistics) {
     Map<String,Number> statValues = this.allStatistics.get(statistics.getTextId());
     if (statValues == null) {
       statValues = new HashMap<String,Number>();
       this.allStatistics.put(statistics.getTextId(), statValues);
     }
+    return statValues;
+  }
+
+  private void incLong(Statistics statistics, String stat, long value) {
+    assertFalse(statistics.isClosed());
+    Map<String, Number> statValues = getOrCreateExpectedValueMap(statistics);
     statistics.incLong(stat, value);
     statValues.put(stat, statistics.getLong(stat));
     if (this.statisticTypes.get(statistics.getTextId()) == null) {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/01848b6e/geode-core/src/test/java/com/gemstone/gemfire/internal/statistics/CallbackSamplerJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/statistics/CallbackSamplerJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/statistics/CallbackSamplerJUnitTest.java
new file mode 100644
index 0000000..9109ba7
--- /dev/null
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/statistics/CallbackSamplerJUnitTest.java
@@ -0,0 +1,98 @@
+/*
+ * 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 com.gemstone.gemfire.internal.statistics;
+
+import static org.junit.Assert.*;
+import static org.mockito.Mockito.*;
+
+import java.util.Arrays;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+
+import com.gemstone.gemfire.CancelCriterion;
+import com.gemstone.gemfire.Statistics;
+import com.gemstone.gemfire.internal.StatSamplerStats;
+import com.gemstone.gemfire.internal.StatisticsImpl;
+import com.gemstone.gemfire.internal.StatisticsManager;
+import com.gemstone.gemfire.test.junit.categories.UnitTest;
+
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.runner.RunWith;
+import org.mockito.ArgumentCaptor;
+import org.mockito.Mock;
+import org.mockito.runners.MockitoJUnitRunner;
+
+@Category(UnitTest.class)
+@RunWith(MockitoJUnitRunner.class)
+public class CallbackSamplerJUnitTest {
+  @Mock
+  CancelCriterion cancelCriterion;
+  @Mock
+  StatSamplerStats statSamplerStats;
+  @Mock
+  StatisticsManager statisticsManager;
+  @Mock
+  ScheduledExecutorService executorService;
+  private CallbackSampler sampler;
+
+  @Before
+  public void createSampler() {
+    sampler = new CallbackSampler(cancelCriterion, statSamplerStats);
+  }
+
+  @Test
+  public void taskShouldSampleStatistics() {
+    Runnable sampleTask = invokeStartAndGetTask();
+
+    StatisticsImpl stats1 = mock(StatisticsImpl.class);
+    StatisticsImpl stats2 = mock(StatisticsImpl.class);
+    when(stats1.invokeSuppliers()).thenReturn(3);
+    when(stats2.invokeSuppliers()).thenReturn(2);
+    when(stats1.getSupplierCount()).thenReturn(7);
+    when(stats2.getSupplierCount()).thenReturn(8);
+    when(statisticsManager.getStatsList()).thenReturn(Arrays.asList(stats1, stats2));
+    sampleTask.run();
+    verify(statSamplerStats).setSampleCallbacks(eq(15));
+    verify(statSamplerStats).incSampleCallbackErrors(5);
+    verify(statSamplerStats).incSampleCallbackDuration(anyLong());
+  }
+
+  @Test
+  public void stopShouldStopExecutor() {
+    sampler.start(executorService, statisticsManager, 1, TimeUnit.MILLISECONDS);
+    sampler.stop();
+    verify(executorService).shutdown();
+  }
+
+  @Test
+  public void cancelCriterionShouldStopExecutor() {
+    Runnable sampleTask = invokeStartAndGetTask();
+    when(cancelCriterion.cancelInProgress()).thenReturn("cancelled");
+    sampleTask.run();
+    verify(executorService).shutdown();
+  }
+
+  private Runnable invokeStartAndGetTask() {
+    sampler.start(executorService, statisticsManager, 1, TimeUnit.MILLISECONDS);
+    ArgumentCaptor<Runnable> runnableCaptor = ArgumentCaptor.forClass(Runnable.class);
+    verify(executorService).scheduleAtFixedRate(runnableCaptor.capture(), eq(1L), eq(1L), eq(TimeUnit.MILLISECONDS));
+    return runnableCaptor.getValue();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/01848b6e/geode-core/src/test/java/com/gemstone/gemfire/internal/statistics/DummyStatistics.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/statistics/DummyStatistics.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/statistics/DummyStatistics.java
deleted file mode 100755
index a1ea030..0000000
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/statistics/DummyStatistics.java
+++ /dev/null
@@ -1,208 +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 com.gemstone.gemfire.internal.statistics;
-
-import com.gemstone.gemfire.StatisticDescriptor;
-import com.gemstone.gemfire.Statistics;
-import com.gemstone.gemfire.StatisticsType;
-
-/**
- * @since GemFire 7.0
- */
-public class DummyStatistics implements Statistics {
-
-  @Override
-  public void close() {
-  }
-
-  @Override
-  public int nameToId(String name) {
-    return 0;
-  }
-  
-  @Override
-  public StatisticDescriptor nameToDescriptor(String name) {
-    return null;
-  }
-  
-  @Override
-  public long getUniqueId() {
-    return 0;
-  }
-
-  @Override
-  public StatisticsType getType() {
-    return null;
-  }
-
-  @Override
-  public String getTextId() {
-    return null;
-  }
-
-  @Override
-  public long getNumericId() {
-    return 0;
-  }
-
-  @Override
-  public boolean isAtomic() {
-    return false;
-  }
-
-  @Override
-  public boolean isClosed() {
-    return false;
-  }
-
-  @Override
-  public void setInt(int id, int value) {
-  }
-
-  @Override
-  public void setInt(String name, int value) {
-  }
-
-  @Override
-  public void setInt(StatisticDescriptor descriptor, int value) {
-  }
-
-  @Override
-  public void setLong(int id, long value) {
-  }
-
-  @Override
-  public void setLong(StatisticDescriptor descriptor, long value) {
-  }
-
-  @Override
-  public void setLong(String name, long value) {
-  }
-
-  @Override
-  public void setDouble(int id, double value) {
-  }
-
-  @Override
-  public void setDouble(StatisticDescriptor descriptor, double value) {
-  }
-
-  @Override
-  public void setDouble(String name, double value) {
-  }
-
-  @Override
-  public int getInt(int id) {
-    return 0;
-  }
-
-  @Override
-  public int getInt(StatisticDescriptor descriptor) {
-    return 0;
-  }
-
-  @Override
-  public int getInt(String name) {
-    return 0;
-  }
-
-  @Override
-  public long getLong(int id) {
-    return 0;
-  }
-
-  @Override
-  public long getLong(StatisticDescriptor descriptor) {
-    return 0;
-  }
-
-  @Override
-  public long getLong(String name) {
-    return 0;
-  }
-
-  @Override
-  public double getDouble(int id) {
-    return 0;
-  }
-
-  @Override
-  public double getDouble(StatisticDescriptor descriptor) {
-    return 0;
-  }
-
-  @Override
-  public double getDouble(String name) {
-    return 0;
-  }
-
-  @Override
-  public Number get(StatisticDescriptor descriptor) {
-    return null;
-  }
-
-  @Override
-  public Number get(String name) {
-    return null;
-  }
-
-  @Override
-  public long getRawBits(StatisticDescriptor descriptor) {
-    return 0;
-  }
-
-  @Override
-  public long getRawBits(String name) {
-    return 0;
-  }
-
-  @Override
-  public void incInt(int id, int delta) {
-  }
-
-  @Override
-  public void incInt(StatisticDescriptor descriptor, int delta) {
-  }
-
-  @Override
-  public void incInt(String name, int delta) {
-  }
-
-  @Override
-  public void incLong(int id, long delta) {
-  }
-
-  @Override
-  public void incLong(StatisticDescriptor descriptor, long delta) {
-  }
-
-  @Override
-  public void incLong(String name, long delta) {
-  }
-
-  @Override
-  public void incDouble(int id, double delta) {
-  }
-
-  @Override
-  public void incDouble(StatisticDescriptor descriptor, double delta) {
-  }
-
-  @Override
-  public void incDouble(String name, double delta) {
-  }
-}