You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@geode.apache.org by jb...@apache.org on 2021/11/19 19:44:31 UTC

[geode] 11/16: Extract legacy OS stats.

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

jbarrett pushed a commit to branch wip/oshi-multios-stats-module
in repository https://gitbox.apache.org/repos/asf/geode.git

commit cf79e27c0c8f88f1bc911f3a89c2003546243e7b
Author: Jacob Barrett <jb...@pivotal.io>
AuthorDate: Wed Nov 17 08:07:17 2021 -0800

    Extract legacy OS stats.
---
 .../internal/MemberHealthEvaluatorJUnitTest.java   |   4 +-
 .../GemFireStatSamplerIntegrationTest.java         |   3 +-
 ...csBenchmark.java => OsStatisticsBenchmark.java} |  47 +++-
 ...arkTest.java => OsStatisticsBenchmarkTest.java} |   2 +-
 .../statistics/AbstractStatisticsFactory.java      |  13 +-
 .../statistics/DummyStatisticsFactory.java         |   4 +-
 .../statistics/DummyStatisticsRegistry.java        |   2 +-
 .../internal/statistics/GemFireStatSampler.java    |  40 +--
 .../geode/internal/statistics/HostStatSampler.java |   1 -
 .../statistics/LocalStatisticsFactory.java         |   5 +-
 .../internal/statistics/LocalStatisticsImpl.java   |  12 +-
 .../{platform => }/OsStatisticsFactory.java        |   5 +-
 .../internal/statistics/OsStatisticsProvider.java  | 127 +--------
 .../statistics/OsStatisticsProviderException.java} |  10 +-
 .../geode/internal/statistics/StatisticsImpl.java  |  22 +-
 .../internal/statistics/StatisticsManager.java     |   1 -
 .../internal/statistics/StatisticsRegistry.java    |  20 +-
 .../internal/statistics/StripedStatisticsImpl.java |   2 +-
 .../LegacyOsStatisticsProvider.java}               | 109 ++++----
 .../statistics/oshi/OshiStatisticsProvider.java    | 307 ++++++++++++++++++++-
 .../oshi/OshiStatisticsProviderException.java      |   4 -
 .../oshi/OshiStatisticsProviderImpl.java           | 307 ---------------------
 .../statistics/platform/LinuxProcFsStatistics.java |  31 +--
 .../internal/beans/MemberMBeanBridge.java          |   5 +-
 .../statistics/LocalStatisticsImplTest.java        |   4 +-
 .../statistics/OsStatisticsProviderTest.java       |  58 ----
 .../internal/statistics/StatisticsImplTest.java    |   4 +-
 .../statistics/StatisticsRegistryTest.java         |  49 ++--
 .../legacy/LegacyOsStatisticsProviderTest.java}    |  14 +-
 ...plTest.java => OshiStatisticsProviderTest.java} |  34 +--
 .../internal/statistics/TestStatisticsManager.java |   3 +-
 31 files changed, 511 insertions(+), 738 deletions(-)

diff --git a/geode-core/src/integrationTest/java/org/apache/geode/admin/internal/MemberHealthEvaluatorJUnitTest.java b/geode-core/src/integrationTest/java/org/apache/geode/admin/internal/MemberHealthEvaluatorJUnitTest.java
index 6a574e6..4809b40 100644
--- a/geode-core/src/integrationTest/java/org/apache/geode/admin/internal/MemberHealthEvaluatorJUnitTest.java
+++ b/geode-core/src/integrationTest/java/org/apache/geode/admin/internal/MemberHealthEvaluatorJUnitTest.java
@@ -26,7 +26,7 @@ import org.junit.Test;
 import org.apache.geode.admin.GemFireHealth;
 import org.apache.geode.admin.GemFireHealthConfig;
 import org.apache.geode.internal.statistics.GemFireStatSampler;
-import org.apache.geode.internal.statistics.OsStatisticsProvider;
+import org.apache.geode.internal.statistics.legacy.LegacyOsStatisticsProvider;
 import org.apache.geode.internal.statistics.platform.ProcessStats;
 
 /**
@@ -46,7 +46,7 @@ public class MemberHealthEvaluatorJUnitTest extends HealthEvaluatorTestCase {
    */
   @Test
   public void testCheckVMProcessSize() throws InterruptedException {
-    if (OsStatisticsProvider.build().osStatsSupported()) {
+    if (LegacyOsStatisticsProvider.build().osStatsSupported()) {
       GemFireStatSampler sampler = system.getStatSampler();
       assertNotNull(sampler);
 
diff --git a/geode-core/src/integrationTest/java/org/apache/geode/internal/statistics/GemFireStatSamplerIntegrationTest.java b/geode-core/src/integrationTest/java/org/apache/geode/internal/statistics/GemFireStatSamplerIntegrationTest.java
index 8128567..d799b32 100644
--- a/geode-core/src/integrationTest/java/org/apache/geode/internal/statistics/GemFireStatSamplerIntegrationTest.java
+++ b/geode-core/src/integrationTest/java/org/apache/geode/internal/statistics/GemFireStatSamplerIntegrationTest.java
@@ -61,6 +61,7 @@ import org.apache.geode.StatisticsType;
 import org.apache.geode.distributed.DistributedSystem;
 import org.apache.geode.distributed.internal.InternalDistributedSystem;
 import org.apache.geode.internal.statistics.GemFireStatSampler.LocalStatListenerImpl;
+import org.apache.geode.internal.statistics.legacy.LegacyOsStatisticsProvider;
 import org.apache.geode.internal.statistics.platform.ProcessStats;
 import org.apache.geode.internal.stats50.VMStats50;
 import org.apache.geode.logging.internal.log4j.api.LogService;
@@ -171,7 +172,7 @@ public class GemFireStatSamplerIntegrationTest extends StatSamplerTestCase {
       assertThat(processStats)
           .withFailMessage("ProcessStats were not created on" + osName)
           .isNotNull();
-      assertThat(OsStatisticsProvider.build().osStatsSupported())
+      assertThat(LegacyOsStatisticsProvider.build().osStatsSupported())
           .as("os stats are available on Linux")
           .isTrue();
       assertThat(allStats.containsStatisticsType("LinuxProcessStats"))
diff --git a/geode-core/src/jmh/java/org/apache/geode/internal/statistics/oshi/OshiStatisticsBenchmark.java b/geode-core/src/jmh/java/org/apache/geode/internal/statistics/OsStatisticsBenchmark.java
similarity index 85%
rename from geode-core/src/jmh/java/org/apache/geode/internal/statistics/oshi/OshiStatisticsBenchmark.java
rename to geode-core/src/jmh/java/org/apache/geode/internal/statistics/OsStatisticsBenchmark.java
index a2f1746..9ec47f1 100644
--- a/geode-core/src/jmh/java/org/apache/geode/internal/statistics/oshi/OshiStatisticsBenchmark.java
+++ b/geode-core/src/jmh/java/org/apache/geode/internal/statistics/OsStatisticsBenchmark.java
@@ -13,7 +13,7 @@
  * the License.
  */
 
-package org.apache.geode.internal.statistics.oshi;
+package org.apache.geode.internal.statistics;
 
 import java.util.concurrent.TimeUnit;
 import java.util.function.DoubleSupplier;
@@ -24,6 +24,7 @@ import org.openjdk.jmh.annotations.Benchmark;
 import org.openjdk.jmh.annotations.BenchmarkMode;
 import org.openjdk.jmh.annotations.Mode;
 import org.openjdk.jmh.annotations.OutputTimeUnit;
+import org.openjdk.jmh.annotations.Param;
 import org.openjdk.jmh.annotations.Scope;
 import org.openjdk.jmh.annotations.Setup;
 import org.openjdk.jmh.annotations.State;
@@ -31,28 +32,45 @@ import org.openjdk.jmh.annotations.State;
 import org.apache.geode.StatisticDescriptor;
 import org.apache.geode.Statistics;
 import org.apache.geode.StatisticsType;
-import org.apache.geode.internal.statistics.SuppliableStatistics;
-import org.apache.geode.internal.statistics.platform.OsStatisticsFactory;
+import org.apache.geode.internal.process.ProcessUtils;
+import org.apache.geode.internal.statistics.legacy.LegacyOsStatisticsProvider;
+import org.apache.geode.internal.statistics.oshi.OshiStatisticsProvider;
 
 @State(Scope.Benchmark)
 @BenchmarkMode(Mode.SampleTime)
 @OutputTimeUnit(TimeUnit.MILLISECONDS)
-public class OshiStatisticsBenchmark {
+public class OsStatisticsBenchmark {
 
-  private final OshiStatisticsProviderImpl oshiStatisticsProvider =
-      new OshiStatisticsProviderImpl();
+  public enum Impl {
+    Legacy,
+    Oshi
+  }
+
+  @Param
+  public Impl impl;
+  private OsStatisticsProvider osStatisticsProvider;
 
   @Setup
-  public void setup() throws OshiStatisticsProviderException {
-    oshiStatisticsProvider.init(new NoopStatisticsProvider(), 0);
+  public void setup() throws OsStatisticsProviderException {
+    switch (impl) {
+      case Legacy:
+        osStatisticsProvider = new LegacyOsStatisticsProvider();
+        break;
+      case Oshi:
+        osStatisticsProvider = new OshiStatisticsProvider();
+        break;
+      default:
+        throw new IllegalArgumentException(impl.name());
+    }
+    osStatisticsProvider.init(new NoopStatisticsProvider(), ProcessUtils.identifyPidAsUnchecked());
   }
 
-//  @Benchmark
-//  public void noop() {}
+  // @Benchmark
+  // public void noop() {}
 
   @Benchmark
   public void sampleProcess() {
-    oshiStatisticsProvider.sampleProcess();
+    osStatisticsProvider.sample();
   }
 
   private static class NoopStatistics implements SuppliableStatistics {
@@ -310,12 +328,11 @@ public class OshiStatisticsBenchmark {
     }
   }
 
-  private static class NoopStatisticsProvider implements OsStatisticsFactory {
+  public static class NoopStatisticsProvider implements OsStatisticsFactory {
     @Override
-    public Statistics createOsStatistics(final StatisticsType type, final String textId,
-        final long numericId,
-        final int osStatFlags) {
+    public Statistics createOsStatistics(final StatisticsType type, final String textId, final long numericId) {
       return new NoopStatistics();
     }
   }
+
 }
diff --git a/geode-core/src/jmhTest/java/org/apache/geode/internal/statistics/platform/OshiStatisticsBenchmarkTest.java b/geode-core/src/jmhTest/java/org/apache/geode/internal/statistics/platform/OsStatisticsBenchmarkTest.java
similarity index 95%
copy from geode-core/src/jmhTest/java/org/apache/geode/internal/statistics/platform/OshiStatisticsBenchmarkTest.java
copy to geode-core/src/jmhTest/java/org/apache/geode/internal/statistics/platform/OsStatisticsBenchmarkTest.java
index db0d965..3221bd4 100644
--- a/geode-core/src/jmhTest/java/org/apache/geode/internal/statistics/platform/OshiStatisticsBenchmarkTest.java
+++ b/geode-core/src/jmhTest/java/org/apache/geode/internal/statistics/platform/OsStatisticsBenchmarkTest.java
@@ -17,7 +17,7 @@ package org.apache.geode.internal.statistics.platform;
 
 import org.junit.Test;
 
-public class OshiStatisticsBenchmarkTest {
+public class OsStatisticsBenchmarkTest {
   @Test
   public void getsValidValues() {
 
diff --git a/geode-core/src/main/java/org/apache/geode/internal/statistics/AbstractStatisticsFactory.java b/geode-core/src/main/java/org/apache/geode/internal/statistics/AbstractStatisticsFactory.java
index 3f42408..f36e2cd 100755
--- a/geode-core/src/main/java/org/apache/geode/internal/statistics/AbstractStatisticsFactory.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/statistics/AbstractStatisticsFactory.java
@@ -111,28 +111,27 @@ public abstract class AbstractStatisticsFactory implements StatisticsFactory, St
 
   @Override
   public Statistics createStatistics(StatisticsType type) {
-    return createOsStatistics(type, null, 0, 0);
+    return createOsStatistics(type, null, 0);
   }
 
   @Override
   public Statistics createStatistics(StatisticsType type, String textId) {
-    return createOsStatistics(type, textId, 0, 0);
+    return createOsStatistics(type, textId, 0);
   }
 
   @Override
   public Statistics createStatistics(StatisticsType type, String textId, long numericId) {
-    return createOsStatistics(type, textId, 0, 0);
+    return createOsStatistics(type, textId, 0);
   }
 
   @Override
-  public Statistics createOsStatistics(StatisticsType type, String textId, long numericId,
-      int osStatFlags) {
+  public Statistics createOsStatistics(StatisticsType type, String textId, long numericId) {
     long myUniqueId;
     synchronized (statsListUniqueIdLock) {
-      myUniqueId = statsListUniqueId++; // fix for bug 30597
+      myUniqueId = statsListUniqueId++;
     }
     Statistics result =
-        new LocalStatisticsImpl(type, textId, numericId, myUniqueId, false, osStatFlags, this);
+        new LocalStatisticsImpl(type, textId, numericId, myUniqueId, false, this);
     synchronized (statsList) {
       statsList.add(result);
       statsListModCount++;
diff --git a/geode-core/src/main/java/org/apache/geode/internal/statistics/DummyStatisticsFactory.java b/geode-core/src/main/java/org/apache/geode/internal/statistics/DummyStatisticsFactory.java
index 167fd36..be74890 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/statistics/DummyStatisticsFactory.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/statistics/DummyStatisticsFactory.java
@@ -49,7 +49,7 @@ public class DummyStatisticsFactory implements StatisticsFactory {
 
   @Override
   public Statistics createStatistics(StatisticsType type, String textId, long numericId) {
-    Statistics result = new LocalStatisticsImpl(type, textId, numericId, 1, false, 0, null);
+    Statistics result = new LocalStatisticsImpl(type, textId, numericId, 1, false, null);
     return result;
   }
 
@@ -72,7 +72,7 @@ public class DummyStatisticsFactory implements StatisticsFactory {
 
   @Override
   public Statistics createAtomicStatistics(StatisticsType type, String textId, long numericId) {
-    Statistics result = new LocalStatisticsImpl(type, textId, numericId, 1, true, 0, null);
+    Statistics result = new LocalStatisticsImpl(type, textId, numericId, 1, true, null);
     return result;
   }
 
diff --git a/geode-core/src/main/java/org/apache/geode/internal/statistics/DummyStatisticsRegistry.java b/geode-core/src/main/java/org/apache/geode/internal/statistics/DummyStatisticsRegistry.java
index 7ad599e..8c65187 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/statistics/DummyStatisticsRegistry.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/statistics/DummyStatisticsRegistry.java
@@ -56,7 +56,7 @@ public class DummyStatisticsRegistry extends StatisticsRegistry {
 
   @Override
   protected Statistics newOsStatistics(StatisticsType type, long uniqueId, long numericId,
-      String textId, int osStatFlags) {
+      String textId) {
     return new DummyStatisticsImpl(type, textId, numericId);
   }
 }
diff --git a/geode-core/src/main/java/org/apache/geode/internal/statistics/GemFireStatSampler.java b/geode-core/src/main/java/org/apache/geode/internal/statistics/GemFireStatSampler.java
index b39ff2f..6ceb510 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/statistics/GemFireStatSampler.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/statistics/GemFireStatSampler.java
@@ -36,9 +36,6 @@ import org.apache.geode.internal.admin.ListenerIdMap;
 import org.apache.geode.internal.admin.remote.StatListenerMessage;
 import org.apache.geode.internal.logging.log4j.LogMarker;
 import org.apache.geode.internal.statistics.oshi.OshiStatisticsProvider;
-import org.apache.geode.internal.statistics.oshi.OshiStatisticsProviderException;
-import org.apache.geode.internal.statistics.oshi.OshiStatisticsProviderImpl;
-import org.apache.geode.internal.statistics.platform.OsStatisticsFactory;
 import org.apache.geode.internal.statistics.platform.ProcessStats;
 import org.apache.geode.logging.internal.log4j.api.LogService;
 import org.apache.geode.logging.internal.spi.LogFile;
@@ -69,8 +66,7 @@ public class GemFireStatSampler extends HostStatSampler {
   private int nextListenerId = 1;
   private ProcessStats processStats;
 
-  // private OsStatisticsProvider osStatisticsProvider = OsStatisticsProvider.build();
-  private OshiStatisticsProvider oshiStatisticsProvider = new OshiStatisticsProviderImpl();
+  private OsStatisticsProvider osStatisticsProvider = new OshiStatisticsProvider();
 
   public GemFireStatSampler(InternalDistributedSystem internalDistributedSystem) {
     this(internalDistributedSystem, null);
@@ -279,20 +275,12 @@ public class GemFireStatSampler extends HostStatSampler {
     }
 
     try {
-      oshiStatisticsProvider.init(getOsStatisticsFactory(), pid);
-    } catch (OshiStatisticsProviderException e) {
+      osStatisticsProvider.init(getOsStatisticsFactory(), pid);
+    } catch (OsStatisticsProviderException e) {
       logger.error(LogMarker.STATISTICS_MARKER, "Failed to initialize OS statistics.", e);
     }
 
-    // osStatisticsProvider.newSystem(getOsStatisticsFactory(), pid);
-    // String statName = getStatisticsManager().getName();
-    // if (statName == null || statName.length() == 0) {
-    // statName = "javaApp" + getSystemId();
-    // }
-    // Statistics stats =
-    // osStatisticsProvider.newProcess(getOsStatisticsFactory(), id, statName + "-proc");
     processStats = null; // osStatisticsProvider.newProcessStats(stats);
-
   }
 
   @Override
@@ -300,30 +288,12 @@ public class GemFireStatSampler extends HostStatSampler {
     if (prepareOnly || osStatsDisabled() || stopRequested()) {
       return;
     }
-    oshiStatisticsProvider.sample();
-    // List<Statistics> statisticsList = getStatisticsManager().getStatsList();
-    // for (Statistics statistics : statisticsList) {
-    // if (stopRequested()) {
-    // return;
-    // }
-    // StatisticsImpl statisticsImpl = (StatisticsImpl) statistics;
-    // if (statisticsImpl.usesSystemCalls()) {
-    // osStatisticsProvider.refresh((LocalStatisticsImpl) statisticsImpl);
-    // }
-    // }
+    osStatisticsProvider.sample();
   }
 
   @Override
   protected void closeProcessStats() {
-    oshiStatisticsProvider.destroy();
-    // if (osStatisticsProvider.osStatsSupported()) {
-    // if (!osStatsDisabled()) {
-    // if (processStats != null) {
-    // processStats.close();
-    // }
-    // osStatisticsProvider.closeOSStats();
-    // }
-    // }
+    osStatisticsProvider.destroy();
   }
 
   private void checkLocalListeners() {
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 6eb6c4f..7e0f4ee 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
@@ -31,7 +31,6 @@ import org.apache.geode.internal.inet.LocalHostUtil;
 import org.apache.geode.internal.io.MainWithChildrenRollingFileHandler;
 import org.apache.geode.internal.logging.log4j.LogMarker;
 import org.apache.geode.internal.process.UncheckedPidUnavailableException;
-import org.apache.geode.internal.statistics.platform.OsStatisticsFactory;
 import org.apache.geode.internal.util.concurrent.StoppableCountDownLatch;
 import org.apache.geode.logging.internal.executors.LoggingThread;
 import org.apache.geode.logging.internal.log4j.api.LogService;
diff --git a/geode-core/src/main/java/org/apache/geode/internal/statistics/LocalStatisticsFactory.java b/geode-core/src/main/java/org/apache/geode/internal/statistics/LocalStatisticsFactory.java
index d8efd83..1fafbe5 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/statistics/LocalStatisticsFactory.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/statistics/LocalStatisticsFactory.java
@@ -79,12 +79,11 @@ public class LocalStatisticsFactory extends AbstractStatisticsFactory
   }
 
   @Override
-  public Statistics createOsStatistics(StatisticsType type, String textId, long numericId,
-      int osStatFlags) {
+  public Statistics createOsStatistics(StatisticsType type, String textId, long numericId) {
     if (this.statsDisabled) {
       return new DummyStatisticsImpl(type, textId, numericId);
     }
-    return super.createOsStatistics(type, textId, numericId, osStatFlags);
+    return super.createOsStatistics(type, textId, numericId);
   }
 
   @Override
diff --git a/geode-core/src/main/java/org/apache/geode/internal/statistics/LocalStatisticsImpl.java b/geode-core/src/main/java/org/apache/geode/internal/statistics/LocalStatisticsImpl.java
index 58ddaab..7610219 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/statistics/LocalStatisticsImpl.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/statistics/LocalStatisticsImpl.java
@@ -56,13 +56,11 @@ public class LocalStatisticsImpl extends StatisticsImpl {
    * @param uniqueId A number that uniquely identifies this instance
    * @param atomicIncrements Are increment operations atomic? If only one application thread
    *        increments a statistic, then a <code>false</code> value may yield better performance.
-   * @param osStatFlags Non-zero if stats require system calls to collect them; for internal use
-   *        only
    * @param statisticsManager The statistics manager that is creating this instance
    */
   public LocalStatisticsImpl(StatisticsType type, String textId, long numericId, long uniqueId,
-      boolean atomicIncrements, int osStatFlags, StatisticsManager statisticsManager) {
-    super(type, textId, numericId, uniqueId, osStatFlags, statisticsManager);
+      boolean atomicIncrements, StatisticsManager statisticsManager) {
+    super(type, textId, numericId, uniqueId, statisticsManager);
 
     StatisticsTypeImpl realType = (StatisticsTypeImpl) type;
     longCount = realType.getLongStatCount();
@@ -107,15 +105,13 @@ public class LocalStatisticsImpl extends StatisticsImpl {
    * @param numericId A number that displayed when this statistic is monitored
    * @param uniqueId A number that uniquely identifies this instance
    *        increments a statistic, then a <code>false</code> value may yield better performance.
-   * @param osStatFlags Non-zero if stats require system calls to collect them; for internal use
-   *        only
    * @param statisticsManager The distributed system that determines whether or not these statistics
    *        are stored
    *        (and collected) in GemFire shared memory or in the local VM
    */
   public static Statistics createNonAtomic(StatisticsType type, String textId, long numericId,
-      long uniqueId, int osStatFlags, StatisticsManager statisticsManager) {
-    return new LocalStatisticsImpl(type, textId, numericId, uniqueId, false, osStatFlags,
+      long uniqueId, StatisticsManager statisticsManager) {
+    return new LocalStatisticsImpl(type, textId, numericId, uniqueId, false,
         statisticsManager);
   }
 
diff --git a/geode-core/src/main/java/org/apache/geode/internal/statistics/platform/OsStatisticsFactory.java b/geode-core/src/main/java/org/apache/geode/internal/statistics/OsStatisticsFactory.java
similarity index 93%
rename from geode-core/src/main/java/org/apache/geode/internal/statistics/platform/OsStatisticsFactory.java
rename to geode-core/src/main/java/org/apache/geode/internal/statistics/OsStatisticsFactory.java
index 2ba2a4c..1a05dc3 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/statistics/platform/OsStatisticsFactory.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/statistics/OsStatisticsFactory.java
@@ -12,7 +12,7 @@
  * or implied. See the License for the specific language governing permissions and limitations under
  * the License.
  */
-package org.apache.geode.internal.statistics.platform;
+package org.apache.geode.internal.statistics;
 
 import org.apache.geode.Statistics;
 import org.apache.geode.StatisticsType;
@@ -32,6 +32,5 @@ public interface OsStatisticsFactory {
    * <p>
    * The created instance may not be {@link Statistics#isAtomic atomic}.
    */
-  Statistics createOsStatistics(StatisticsType type, String textId, long numericId,
-      int osStatFlags);
+  Statistics createOsStatistics(StatisticsType type, String textId, long numericId);
 }
diff --git a/geode-core/src/main/java/org/apache/geode/internal/statistics/OsStatisticsProvider.java b/geode-core/src/main/java/org/apache/geode/internal/statistics/OsStatisticsProvider.java
index d274bf1..79a96b8 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/statistics/OsStatisticsProvider.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/statistics/OsStatisticsProvider.java
@@ -12,129 +12,16 @@
  * or implied. See the License for the specific language governing permissions and limitations under
  * the License.
  */
-package org.apache.geode.internal.statistics;
-
-import java.net.UnknownHostException;
-
-import org.apache.geode.Statistics;
-import org.apache.geode.internal.inet.LocalHostUtil;
-import org.apache.geode.internal.lang.SystemUtils;
-import org.apache.geode.internal.statistics.platform.LinuxProcFsStatistics;
-import org.apache.geode.internal.statistics.platform.LinuxProcessStats;
-import org.apache.geode.internal.statistics.platform.LinuxSystemStats;
-import org.apache.geode.internal.statistics.platform.OsStatisticsFactory;
-import org.apache.geode.internal.statistics.platform.ProcessStats;
-
-/**
- * Provides methods which fetch operating system statistics.
- * Only Linux OS is currently allowed.
- */
-public class OsStatisticsProvider {
-  private static final int PROCESS_STAT_FLAG = 1;
-  private static final int SYSTEM_STAT_FLAG = 2;
-  private final boolean osStatsSupported;
-
-  public boolean osStatsSupported() {
-    return osStatsSupported;
-  }
-
-  private OsStatisticsProvider() {
-    osStatsSupported = SystemUtils.isLinux();
-  }
 
-  public static OsStatisticsProvider build() {
-    return new OsStatisticsProvider();
-  }
-
-  int initOSStats() {
-    return LinuxProcFsStatistics.init();
-  }
-
-  void closeOSStats() {
-    LinuxProcFsStatistics.close();
-  }
-
-  void readyRefreshOSStats() {
-    LinuxProcFsStatistics.readyRefresh();
-  }
-
-  /**
-   * Refreshes the specified process stats instance by fetching the current OS values for the given
-   * stats and storing them in the instance.
-   */
-  private void refreshProcess(LocalStatisticsImpl statistics) {
-    int pid = (int) statistics.getNumericId();
-    LinuxProcFsStatistics.refreshProcess(pid, statistics);
-  }
-
-  /**
-   * Refreshes the specified system stats instance by fetching the current OS values for the local
-   * machine and storing them in the instance.
-   */
-  private void refreshSystem(LocalStatisticsImpl statistics) {
-    LinuxProcFsStatistics.refreshSystem(statistics);
-  }
-
-  /**
-   * The call should have already checked to make sure usesSystemCalls returns true.
-   */
-  public void refresh(LocalStatisticsImpl statistics) {
-    int flags = statistics.getOsStatFlags();
-    if ((flags & PROCESS_STAT_FLAG) != 0) {
-      refreshProcess(statistics);
-    } else if ((flags & SYSTEM_STAT_FLAG) != 0) {
-      refreshSystem(statistics);
-    } else {
-      throw new RuntimeException(String.format("Unexpected os stats flags %s", flags));
-    }
-  }
+package org.apache.geode.internal.statistics;
 
-  /**
-   * Creates and returns a {@link Statistics} with the given pid and name. The resource's stats will
-   * contain a snapshot of the current statistic values for the specified process.
-   */
-  Statistics newProcess(OsStatisticsFactory osStatisticsFactory, long pid, String name) {
-    Statistics statistics;
-    statistics = osStatisticsFactory.createOsStatistics(LinuxProcessStats.getType(), name, pid,
-        PROCESS_STAT_FLAG);
-    // Note we don't call refreshProcess since we only want the manager to do that
-    return statistics;
-  }
+import org.jetbrains.annotations.NotNull;
 
-  /**
-   * Creates a new <code>ProcessStats</code> instance that wraps the given <code>Statistics</code>.
-   *
-   * @see #newProcess
-   * @since GemFire 3.5
-   */
-  ProcessStats newProcessStats(Statistics statistics) {
-    if (statistics instanceof LocalStatisticsImpl) {
-      refresh((LocalStatisticsImpl) statistics);
-    } // otherwise its a Dummy implementation so do nothing
-    return LinuxProcessStats.createProcessStats(statistics);
-  }
+public interface OsStatisticsProvider {
+  void init(final @NotNull OsStatisticsFactory osStatisticsFactory,
+      final long pid) throws OsStatisticsProviderException;
 
-  /**
-   * Creates a {@link Statistics} with the current machine's stats. The resource's stats
-   * will contain a snapshot of the current statistic values for the local machine.
-   */
-  void newSystem(OsStatisticsFactory osStatisticsFactory, long id) {
-    Statistics statistics;
-    statistics = osStatisticsFactory.createOsStatistics(LinuxSystemStats.getType(),
-        getHostSystemName(), id, SYSTEM_STAT_FLAG);
-    if (statistics instanceof LocalStatisticsImpl) {
-      refreshSystem((LocalStatisticsImpl) statistics);
-    } // otherwise its a Dummy implementation so do nothing
-  }
+  void sample();
 
-  /**
-   * @return this machine's fully qualified hostname or "unknownHostName" if one cannot be found.
-   */
-  private String getHostSystemName() {
-    try {
-      return LocalHostUtil.getCanonicalLocalHostName();
-    } catch (UnknownHostException ignored) {
-    }
-    return "unknownHostName";
-  }
+  void destroy();
 }
diff --git a/geode-core/src/jmhTest/java/org/apache/geode/internal/statistics/platform/OshiStatisticsBenchmarkTest.java b/geode-core/src/main/java/org/apache/geode/internal/statistics/OsStatisticsProviderException.java
similarity index 82%
copy from geode-core/src/jmhTest/java/org/apache/geode/internal/statistics/platform/OshiStatisticsBenchmarkTest.java
copy to geode-core/src/main/java/org/apache/geode/internal/statistics/OsStatisticsProviderException.java
index db0d965..5e64e93 100644
--- a/geode-core/src/jmhTest/java/org/apache/geode/internal/statistics/platform/OshiStatisticsBenchmarkTest.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/statistics/OsStatisticsProviderException.java
@@ -13,13 +13,7 @@
  * the License.
  */
 
-package org.apache.geode.internal.statistics.platform;
+package org.apache.geode.internal.statistics;
 
-import org.junit.Test;
-
-public class OshiStatisticsBenchmarkTest {
-  @Test
-  public void getsValidValues() {
-
-  }
+public class OsStatisticsProviderException extends Exception {
 }
diff --git a/geode-core/src/main/java/org/apache/geode/internal/statistics/StatisticsImpl.java b/geode-core/src/main/java/org/apache/geode/internal/statistics/StatisticsImpl.java
index a524ca3..270cd43 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/statistics/StatisticsImpl.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/statistics/StatisticsImpl.java
@@ -53,9 +53,6 @@ public abstract class StatisticsImpl implements SuppliableStatistics {
   /** Numeric information display with these statistics */
   private final long numericId;
 
-  /** Non-zero if stats values come from operating system system calls */
-  private final int osStatFlags;
-
   /** Uniquely identifies this instance */
   private final long uniqueId;
 
@@ -103,13 +100,11 @@ public abstract class StatisticsImpl implements SuppliableStatistics {
    * @param textId Text that helps identifies this instance
    * @param numericId A number that helps identify this instance
    * @param uniqueId A number that uniquely identifies this instance
-   * @param osStatFlags Non-zero if stats require system calls to collect them; for internal use
-   *        only
    * @param statisticsManager The StatisticsManager responsible for creating this instance
    */
   StatisticsImpl(StatisticsType type, String textId, long numericId, long uniqueId,
-      int osStatFlags, StatisticsManager statisticsManager) {
-    this(type, textId, numericId, uniqueId, osStatFlags, statisticsManager,
+      StatisticsManager statisticsManager) {
+    this(type, textId, numericId, uniqueId, statisticsManager,
         logger::warn);
   }
 
@@ -120,18 +115,15 @@ public abstract class StatisticsImpl implements SuppliableStatistics {
    * @param textId Text that helps identifies this instance
    * @param numericId A number that helps identify this instance
    * @param uniqueId A number that uniquely identifies this instance
-   * @param osStatFlags Non-zero if stats require system calls to collect them; for internal use
-   *        only
    * @param statisticsManager The StatisticsManager responsible for creating this instance
    * @param statisticsLogger The StatisticsLogger to log warning about flaky suppliers
    */
   StatisticsImpl(StatisticsType type, String textId, long numericId, long uniqueId,
-      int osStatFlags, StatisticsManager statisticsManager, StatisticsLogger statisticsLogger) {
+      StatisticsManager statisticsManager, StatisticsLogger statisticsLogger) {
     this.type = (ValidatingStatisticsType) type;
     this.textId = StringUtils.isEmpty(textId) ? statisticsManager.getName() : textId;
     this.numericId = numericId == 0 ? statisticsManager.getPid() : numericId;
     this.uniqueId = uniqueId;
-    this.osStatFlags = osStatFlags;
     this.statisticsManager = statisticsManager;
     this.statisticsLogger = statisticsLogger;
     closed = false;
@@ -540,14 +532,6 @@ public abstract class StatisticsImpl implements SuppliableStatistics {
     return intSuppliers.size() + doubleSuppliers.size() + longSuppliers.size();
   }
 
-  boolean usesSystemCalls() {
-    return osStatFlags != 0;
-  }
-
-  int getOsStatFlags() {
-    return osStatFlags;
-  }
-
   private void logSupplierError(final Throwable throwable, int statId, Object supplier) {
     if (flakySuppliers.add(supplier)) {
       statisticsLogger.logWarning("Error invoking supplier for stat {}, id {}", getTextId(), statId,
diff --git a/geode-core/src/main/java/org/apache/geode/internal/statistics/StatisticsManager.java b/geode-core/src/main/java/org/apache/geode/internal/statistics/StatisticsManager.java
index f53c2c5..457c71f 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/statistics/StatisticsManager.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/statistics/StatisticsManager.java
@@ -19,7 +19,6 @@ import java.util.List;
 
 import org.apache.geode.Statistics;
 import org.apache.geode.StatisticsFactory;
-import org.apache.geode.internal.statistics.platform.OsStatisticsFactory;
 
 /**
  * Defines the contract that a statistics factory must implement for its Statistics instances. This
diff --git a/geode-core/src/main/java/org/apache/geode/internal/statistics/StatisticsRegistry.java b/geode-core/src/main/java/org/apache/geode/internal/statistics/StatisticsRegistry.java
index 3fdbfda..04db506 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/statistics/StatisticsRegistry.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/statistics/StatisticsRegistry.java
@@ -51,9 +51,7 @@ public class StatisticsRegistry implements StatisticsManager {
    * Creates an instance of OS Statistics for this registry.
    */
   public interface OsStatisticsFactory {
-
-    Statistics create(StatisticsType type, String textId, long numericId, long uniqueId,
-        int osStatFlags, StatisticsManager manager);
+    Statistics create(StatisticsType type, String textId, long numericId, long uniqueId, StatisticsManager manager);
   }
 
   /**
@@ -158,7 +156,7 @@ public class StatisticsRegistry implements StatisticsManager {
 
   @Override
   public Statistics createStatistics(StatisticsType type) {
-    return createOsStatistics(type, null, 0, 0);
+    return createOsStatistics(type, null, 0);
   }
 
   @Override
@@ -168,7 +166,7 @@ public class StatisticsRegistry implements StatisticsManager {
 
   @Override
   public Statistics createStatistics(StatisticsType type, String textId) {
-    return createOsStatistics(type, textId, 0, 0);
+    return createOsStatistics(type, textId, 0);
   }
 
   @Override
@@ -178,7 +176,7 @@ public class StatisticsRegistry implements StatisticsManager {
 
   @Override
   public Statistics createStatistics(StatisticsType type, String textId, long numericId) {
-    return createOsStatistics(type, textId, numericId, 0);
+    return createOsStatistics(type, textId, numericId);
   }
 
   @Override
@@ -188,10 +186,9 @@ public class StatisticsRegistry implements StatisticsManager {
   }
 
   @Override
-  public Statistics createOsStatistics(StatisticsType type, String textId, long numericId,
-      int osStatFlags) {
+  public Statistics createOsStatistics(StatisticsType type, String textId, long numericId) {
     long uniqueId = nextUniqueId.getAndIncrement();
-    return newOsStatistics(type, uniqueId, numericId, textId, osStatFlags);
+    return newOsStatistics(type, uniqueId, numericId, textId);
   }
 
   @Override
@@ -304,9 +301,8 @@ public class StatisticsRegistry implements StatisticsManager {
   }
 
   protected Statistics newOsStatistics(StatisticsType type, long uniqueId, long numericId,
-      String textId, int osStatFlags) {
-    Statistics statistics = osStatisticsFactory.create(type, textId, numericId, uniqueId,
-        osStatFlags, this);
+      String textId) {
+    Statistics statistics = osStatisticsFactory.create(type, textId, numericId, uniqueId, this);
     registerNewStatistics(statistics);
     return statistics;
   }
diff --git a/geode-core/src/main/java/org/apache/geode/internal/statistics/StripedStatisticsImpl.java b/geode-core/src/main/java/org/apache/geode/internal/statistics/StripedStatisticsImpl.java
index 8634cd4..b46c4a2 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/statistics/StripedStatisticsImpl.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/statistics/StripedStatisticsImpl.java
@@ -33,7 +33,7 @@ public class StripedStatisticsImpl extends StatisticsImpl {
 
   public StripedStatisticsImpl(StatisticsType type, String textId, long numericId,
       long uniqueId, StatisticsManager statisticsManager) {
-    super(type, textId, numericId, uniqueId, 0, statisticsManager);
+    super(type, textId, numericId, uniqueId, statisticsManager);
 
     StatisticsTypeImpl realType = (StatisticsTypeImpl) type;
 
diff --git a/geode-core/src/main/java/org/apache/geode/internal/statistics/OsStatisticsProvider.java b/geode-core/src/main/java/org/apache/geode/internal/statistics/legacy/LegacyOsStatisticsProvider.java
similarity index 58%
copy from geode-core/src/main/java/org/apache/geode/internal/statistics/OsStatisticsProvider.java
copy to geode-core/src/main/java/org/apache/geode/internal/statistics/legacy/LegacyOsStatisticsProvider.java
index d274bf1..1578178 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/statistics/OsStatisticsProvider.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/statistics/legacy/LegacyOsStatisticsProvider.java
@@ -12,57 +12,59 @@
  * or implied. See the License for the specific language governing permissions and limitations under
  * the License.
  */
-package org.apache.geode.internal.statistics;
+package org.apache.geode.internal.statistics.legacy;
 
 import java.net.UnknownHostException;
 
+import org.jetbrains.annotations.NotNull;
+
 import org.apache.geode.Statistics;
 import org.apache.geode.internal.inet.LocalHostUtil;
 import org.apache.geode.internal.lang.SystemUtils;
+import org.apache.geode.internal.statistics.OsStatisticsFactory;
+import org.apache.geode.internal.statistics.OsStatisticsProvider;
 import org.apache.geode.internal.statistics.platform.LinuxProcFsStatistics;
 import org.apache.geode.internal.statistics.platform.LinuxProcessStats;
 import org.apache.geode.internal.statistics.platform.LinuxSystemStats;
-import org.apache.geode.internal.statistics.platform.OsStatisticsFactory;
 import org.apache.geode.internal.statistics.platform.ProcessStats;
 
 /**
  * Provides methods which fetch operating system statistics.
  * Only Linux OS is currently allowed.
  */
-public class OsStatisticsProvider {
+public class LegacyOsStatisticsProvider implements OsStatisticsProvider {
   private static final int PROCESS_STAT_FLAG = 1;
   private static final int SYSTEM_STAT_FLAG = 2;
   private final boolean osStatsSupported;
+  private Statistics systemStatistics;
+  private Statistics processStatistics;
+  private ProcessStats processStats;
 
   public boolean osStatsSupported() {
     return osStatsSupported;
   }
 
-  private OsStatisticsProvider() {
+  public LegacyOsStatisticsProvider() {
     osStatsSupported = SystemUtils.isLinux();
   }
 
-  public static OsStatisticsProvider build() {
-    return new OsStatisticsProvider();
+  public static LegacyOsStatisticsProvider build() {
+    return new LegacyOsStatisticsProvider();
   }
 
-  int initOSStats() {
-    return LinuxProcFsStatistics.init();
+  private void initOSStats() {
+    LinuxProcFsStatistics.init();
   }
 
-  void closeOSStats() {
+  private void closeOSStats() {
     LinuxProcFsStatistics.close();
   }
 
-  void readyRefreshOSStats() {
-    LinuxProcFsStatistics.readyRefresh();
-  }
-
   /**
    * Refreshes the specified process stats instance by fetching the current OS values for the given
    * stats and storing them in the instance.
    */
-  private void refreshProcess(LocalStatisticsImpl statistics) {
+  private void refreshProcess(@NotNull final Statistics statistics) {
     int pid = (int) statistics.getNumericId();
     LinuxProcFsStatistics.refreshProcess(pid, statistics);
   }
@@ -71,34 +73,16 @@ public class OsStatisticsProvider {
    * Refreshes the specified system stats instance by fetching the current OS values for the local
    * machine and storing them in the instance.
    */
-  private void refreshSystem(LocalStatisticsImpl statistics) {
+  private void refreshSystem(@NotNull final Statistics statistics) {
     LinuxProcFsStatistics.refreshSystem(statistics);
   }
 
   /**
-   * The call should have already checked to make sure usesSystemCalls returns true.
-   */
-  public void refresh(LocalStatisticsImpl statistics) {
-    int flags = statistics.getOsStatFlags();
-    if ((flags & PROCESS_STAT_FLAG) != 0) {
-      refreshProcess(statistics);
-    } else if ((flags & SYSTEM_STAT_FLAG) != 0) {
-      refreshSystem(statistics);
-    } else {
-      throw new RuntimeException(String.format("Unexpected os stats flags %s", flags));
-    }
-  }
-
-  /**
    * Creates and returns a {@link Statistics} with the given pid and name. The resource's stats will
    * contain a snapshot of the current statistic values for the specified process.
    */
-  Statistics newProcess(OsStatisticsFactory osStatisticsFactory, long pid, String name) {
-    Statistics statistics;
-    statistics = osStatisticsFactory.createOsStatistics(LinuxProcessStats.getType(), name, pid,
-        PROCESS_STAT_FLAG);
-    // Note we don't call refreshProcess since we only want the manager to do that
-    return statistics;
+  private Statistics newProcess(OsStatisticsFactory osStatisticsFactory, long pid, String name) {
+    return osStatisticsFactory.createOsStatistics(LinuxProcessStats.getType(), name, pid);
   }
 
   /**
@@ -107,10 +91,8 @@ public class OsStatisticsProvider {
    * @see #newProcess
    * @since GemFire 3.5
    */
-  ProcessStats newProcessStats(Statistics statistics) {
-    if (statistics instanceof LocalStatisticsImpl) {
-      refresh((LocalStatisticsImpl) statistics);
-    } // otherwise its a Dummy implementation so do nothing
+  private @NotNull ProcessStats newProcessStats(@NotNull Statistics statistics) {
+    refreshProcess(statistics);
     return LinuxProcessStats.createProcessStats(statistics);
   }
 
@@ -118,23 +100,56 @@ public class OsStatisticsProvider {
    * Creates a {@link Statistics} with the current machine's stats. The resource's stats
    * will contain a snapshot of the current statistic values for the local machine.
    */
-  void newSystem(OsStatisticsFactory osStatisticsFactory, long id) {
-    Statistics statistics;
-    statistics = osStatisticsFactory.createOsStatistics(LinuxSystemStats.getType(),
-        getHostSystemName(), id, SYSTEM_STAT_FLAG);
-    if (statistics instanceof LocalStatisticsImpl) {
-      refreshSystem((LocalStatisticsImpl) statistics);
-    } // otherwise its a Dummy implementation so do nothing
+  private Statistics newSystem(@NotNull OsStatisticsFactory osStatisticsFactory, long id) {
+    final Statistics statistics = osStatisticsFactory.createOsStatistics(LinuxSystemStats.getType(),
+        getHostSystemName(), id);
+    refreshSystem(statistics);
+    return statistics;
   }
 
   /**
    * @return this machine's fully qualified hostname or "unknownHostName" if one cannot be found.
    */
-  private String getHostSystemName() {
+  private @NotNull String getHostSystemName() {
     try {
       return LocalHostUtil.getCanonicalLocalHostName();
     } catch (UnknownHostException ignored) {
     }
     return "unknownHostName";
   }
+
+  private void sampleSystem() {
+    refreshSystem(systemStatistics);
+  }
+
+  void sampleProcess() {
+    refreshSystem(processStatistics);
+  }
+
+  @Override
+  public void init(final @NotNull OsStatisticsFactory osStatisticsFactory, final long pid) {
+    initOSStats();
+    systemStatistics = newSystem(osStatisticsFactory, pid);
+
+    // TODO jbarrett
+    // String statName = getStatisticsManager().getName();
+    // if (statName == null || statName.length() == 0) {
+    // statName = "javaApp" + getSystemId();
+    // }
+    processStatistics = newProcess(osStatisticsFactory, pid, "javaApp-proc");
+    processStats = newProcessStats(processStatistics);
+  }
+
+  @Override
+  public void sample() {
+    sampleSystem();
+    sampleProcess();
+  }
+
+  @Override
+  public void destroy() {
+    processStats.close();
+    closeOSStats();
+  }
+
 }
diff --git a/geode-core/src/main/java/org/apache/geode/internal/statistics/oshi/OshiStatisticsProvider.java b/geode-core/src/main/java/org/apache/geode/internal/statistics/oshi/OshiStatisticsProvider.java
index b1ec885..35f67bb 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/statistics/oshi/OshiStatisticsProvider.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/statistics/oshi/OshiStatisticsProvider.java
@@ -1,14 +1,309 @@
 package org.apache.geode.internal.statistics.oshi;
 
+import java.util.List;
+
+import org.apache.logging.log4j.Logger;
 import org.jetbrains.annotations.NotNull;
+import oshi.SystemInfo;
+import oshi.hardware.CentralProcessor;
+import oshi.hardware.CentralProcessor.LogicalProcessor;
+import oshi.hardware.CentralProcessor.TickType;
+import oshi.hardware.GlobalMemory;
+import oshi.hardware.HardwareAbstractionLayer;
+import oshi.hardware.NetworkIF;
+import oshi.hardware.VirtualMemory;
+import oshi.software.os.FileSystem;
+import oshi.software.os.InternetProtocolStats;
+import oshi.software.os.InternetProtocolStats.TcpStats;
+import oshi.software.os.InternetProtocolStats.UdpStats;
+import oshi.software.os.OSProcess;
+import oshi.software.os.OperatingSystem;
+
+import org.apache.geode.Statistics;
+import org.apache.geode.internal.statistics.OsStatisticsFactory;
+import org.apache.geode.internal.statistics.OsStatisticsProvider;
+import org.apache.geode.internal.statistics.OsStatisticsProviderException;
+import org.apache.geode.logging.internal.log4j.api.LogService;
+
+public class OshiStatisticsProvider implements OsStatisticsProvider {
+  private static final Logger log = LogService.getLogger();
+
+  final SystemInfo systemInfo;
+
+  private int processId;
+  private CentralProcessor processor;
+  private OperatingSystem operatingSystem;
+  private HardwareAbstractionLayer hardware;
+  private List<NetworkIF> networkIFs;
+
+  private long[] systemCpuLoadTicks;
+  private long[][] processorCpuLoadTicks;
+  private OSProcess process;
+
+  private Statistics processStats;
+  private Statistics systemStats;
+  private Statistics[] processorStats;
+  private Statistics[] networkInterfaceStats;
+
+  public OshiStatisticsProvider() {
+    this(new SystemInfo());
+  }
+
+  OshiStatisticsProvider(final SystemInfo systemInfo) {
+    this.systemInfo = systemInfo;
+  }
+
+  @Override
+  public void init(final @NotNull OsStatisticsFactory osStatisticsFactory,
+      final long id) throws OsStatisticsProviderException {
+
+    operatingSystem = systemInfo.getOperatingSystem();
+    processId = operatingSystem.getProcessId();
+    hardware = systemInfo.getHardware();
+    processor = hardware.getProcessor();
+
+    process = operatingSystem.getProcess(processId);
+    final String processIdentity = process.toString();
+    processStats = osStatisticsFactory.createOsStatistics(ProcessStats.getType(),
+        processIdentity, id);
+
+    final String systemIdentity = operatingSystem.toString();
+    systemCpuLoadTicks = new long[TickType.values().length];
+    systemStats = osStatisticsFactory.createOsStatistics(OperatingSystemStats.getType(),
+        systemIdentity, id);
+
+
+    final List<LogicalProcessor> logicalProcessors = processor.getLogicalProcessors();
+    processorCpuLoadTicks = new long[logicalProcessors.size()][TickType.values().length];
+    processorStats = new Statistics[logicalProcessors.size()];
+    for (int i = 0, size = logicalProcessors.size(); i < size; i++) {
+      final LogicalProcessor logicalProcessor = logicalProcessors.get(i);
+      final String processorIdentity = logicalProcessor.toString();
+      processorStats[i] = osStatisticsFactory.createOsStatistics(ProcessorStats.getType(),
+          processorIdentity, id);
+    }
+
+    networkIFs = hardware.getNetworkIFs();
+    networkInterfaceStats = new Statistics[networkIFs.size()];
+    for (int i = 0, size = networkIFs.size(); i < size; i++) {
+      final NetworkIF networkIF = networkIFs.get(i);
+      final String displayName = networkIF.getDisplayName();
+      log.info("Creating network interfaces stats for {}", displayName);
+      networkInterfaceStats[i] =
+          osStatisticsFactory.createOsStatistics(NetworkInterfaceStats.getType(),
+              displayName, id);
+    }
+  }
+
+  @Override
+  public void sample() {
+    sampleProcess();
+    sampleSystem();
+    sampleProcessors();
+    sampleNetworkInterfaces();
+  }
+
+  @Override
+  public void destroy() {}
+
+  void sampleProcess() {
+    final OSProcess process = operatingSystem.getProcess(processId);
+
+    final double processCpuLoadBetweenTicks = process.getProcessCpuLoadBetweenTicks(this.process);
+    processStats.setDouble(ProcessStats.cpuLoad, processCpuLoadBetweenTicks);
+    this.process = process;
+
+    processStats.setLong(ProcessStats.virtualSize, process.getVirtualSize());
+    processStats.setLong(ProcessStats.residentSetSize, process.getResidentSetSize());
+    processStats.setLong(ProcessStats.threadCount, process.getThreadCount());
+    processStats.setLong(ProcessStats.kernelTime, process.getKernelTime());
+    processStats.setLong(ProcessStats.userTime, process.getUserTime());
+    processStats.setLong(ProcessStats.bytesRead, process.getBytesRead());
+    processStats.setLong(ProcessStats.bytesWritten, process.getBytesWritten());
+    processStats.setLong(ProcessStats.openFiles, process.getOpenFiles());
+    processStats.setDouble(ProcessStats.cpuLoadCumulative, process.getProcessCpuLoadCumulative());
+    processStats.setLong(ProcessStats.minorFaults, process.getMinorFaults());
+    processStats.setLong(ProcessStats.majorFaults, process.getMajorFaults());
+    processStats.setLong(ProcessStats.contextSwitches, process.getContextSwitches());
+  }
+
+  private void sampleSystem() {
+    systemStats.setLong(OperatingSystemStats.processCount, operatingSystem.getProcessCount());
+    systemStats.setLong(OperatingSystemStats.threadCount, operatingSystem.getThreadCount());
+
+    final CentralProcessor processor = hardware.getProcessor();
+    systemStats.setLong(OperatingSystemStats.contextSwitches, processor.getContextSwitches());
+    systemStats.setLong(OperatingSystemStats.interrupts, processor.getInterrupts());
+    systemStats.setLong(OperatingSystemStats.physicalProcessorCount,
+        processor.getPhysicalProcessorCount());
+    systemStats.setLong(OperatingSystemStats.logicalProcessorCount,
+        processor.getLogicalProcessorCount());
+    systemStats.setLong(OperatingSystemStats.maxFreq, processor.getMaxFreq());
+
+    final double[] systemLoadAverage = processor.getSystemLoadAverage(3);
+    systemStats.setDouble(OperatingSystemStats.systemLoadAverage1, systemLoadAverage[0]);
+    systemStats.setDouble(OperatingSystemStats.systemLoadAverage5, systemLoadAverage[1]);
+    systemStats.setDouble(OperatingSystemStats.systemLoadAverage15, systemLoadAverage[2]);
+
+    final double systemCpuLoadBetweenTicks =
+        processor.getSystemCpuLoadBetweenTicks(systemCpuLoadTicks);
+    systemStats.setDouble(OperatingSystemStats.systemCpuLoad, systemCpuLoadBetweenTicks);
+
+    systemCpuLoadTicks = processor.getSystemCpuLoadTicks();
+    systemStats.setLong(OperatingSystemStats.systemCpuLoadTicksUSER,
+        systemCpuLoadTicks[TickType.USER.getIndex()]);
+    systemStats.setLong(OperatingSystemStats.systemCpuLoadTicksNICE,
+        systemCpuLoadTicks[TickType.NICE.getIndex()]);
+    systemStats.setLong(OperatingSystemStats.systemCpuLoadTicksSYSTEM,
+        systemCpuLoadTicks[TickType.SYSTEM.getIndex()]);
+    systemStats.setLong(OperatingSystemStats.systemCpuLoadTicksIDLE,
+        systemCpuLoadTicks[TickType.IDLE.getIndex()]);
+    systemStats.setLong(OperatingSystemStats.systemCpuLoadTicksIOWAIT,
+        systemCpuLoadTicks[TickType.IOWAIT.getIndex()]);
+    systemStats.setLong(OperatingSystemStats.systemCpuLoadTicksIRQ,
+        systemCpuLoadTicks[TickType.IRQ.getIndex()]);
+    systemStats.setLong(OperatingSystemStats.systemCpuLoadTicksSOFTIRQ,
+        systemCpuLoadTicks[TickType.SOFTIRQ.getIndex()]);
+    systemStats.setLong(OperatingSystemStats.systemCpuLoadTicksSTEAL,
+        systemCpuLoadTicks[TickType.STEAL.getIndex()]);
+
+    final GlobalMemory memory = hardware.getMemory();
+    systemStats.setLong(OperatingSystemStats.memoryTotal, memory.getTotal());
+    systemStats.setLong(OperatingSystemStats.memoryAvailable, memory.getAvailable());
+    systemStats.setLong(OperatingSystemStats.memoryPageSize, memory.getPageSize());
+
+    final VirtualMemory virtualMemory = memory.getVirtualMemory();
+    systemStats.setLong(OperatingSystemStats.swapTotal, virtualMemory.getSwapTotal());
+    systemStats.setLong(OperatingSystemStats.swapUsed, virtualMemory.getSwapUsed());
+    systemStats.setLong(OperatingSystemStats.virtualMax, virtualMemory.getVirtualMax());
+    systemStats.setLong(OperatingSystemStats.virtualInUse, virtualMemory.getVirtualInUse());
+    systemStats.setLong(OperatingSystemStats.swapPagesIn, virtualMemory.getSwapPagesIn());
+    systemStats.setLong(OperatingSystemStats.swapPagesOut, virtualMemory.getSwapPagesOut());
+
+    final InternetProtocolStats internetProtocolStats = operatingSystem.getInternetProtocolStats();
+    final TcpStats tcPv4Stats = internetProtocolStats.getTCPv4Stats();
+    systemStats.setLong(OperatingSystemStats.tcpv4ConnectionsEstablished,
+        tcPv4Stats.getConnectionsEstablished());
+    systemStats.setLong(OperatingSystemStats.tcpv4ConnectionsActive,
+        tcPv4Stats.getConnectionsActive());
+    systemStats.setLong(OperatingSystemStats.tcpv4ConnectionsPassive,
+        tcPv4Stats.getConnectionsPassive());
+    systemStats.setLong(OperatingSystemStats.tcpv4ConnectionFailures,
+        tcPv4Stats.getConnectionFailures());
+    systemStats.setLong(OperatingSystemStats.tcpv4ConnectionsReset,
+        tcPv4Stats.getConnectionsReset());
+    systemStats.setLong(OperatingSystemStats.tcpv4SegmentsSent,
+        tcPv4Stats.getSegmentsSent());
+    systemStats.setLong(OperatingSystemStats.tcpv4SegmentsReceived,
+        tcPv4Stats.getSegmentsReceived());
+    systemStats.setLong(OperatingSystemStats.tcpv4SegmentsRetransmitted,
+        tcPv4Stats.getSegmentsRetransmitted());
+    systemStats.setLong(OperatingSystemStats.tcpv4InErrors,
+        tcPv4Stats.getInErrors());
+    systemStats.setLong(OperatingSystemStats.tcpv4OutResets,
+        tcPv4Stats.getOutResets());
+
+    final UdpStats udPv4Stats = internetProtocolStats.getUDPv4Stats();
+    systemStats.setLong(OperatingSystemStats.udpv4DatagramsSent,
+        udPv4Stats.getDatagramsSent());
+    systemStats.setLong(OperatingSystemStats.udpv4DatagramsReceived,
+        udPv4Stats.getDatagramsReceived());
+    systemStats.setLong(OperatingSystemStats.udpv4DatagramsNoPort,
+        udPv4Stats.getDatagramsNoPort());
+    systemStats.setLong(OperatingSystemStats.udpv4DatagramsReceivedErrors,
+        udPv4Stats.getDatagramsReceivedErrors());
+
+    final TcpStats tcPv6Stats = internetProtocolStats.getTCPv6Stats();
+    systemStats.setLong(OperatingSystemStats.tcpv6ConnectionsEstablished,
+        tcPv6Stats.getConnectionsEstablished());
+    systemStats.setLong(OperatingSystemStats.tcpv6ConnectionsActive,
+        tcPv6Stats.getConnectionsActive());
+    systemStats.setLong(OperatingSystemStats.tcpv6ConnectionsPassive,
+        tcPv6Stats.getConnectionsPassive());
+    systemStats.setLong(OperatingSystemStats.tcpv6ConnectionFailures,
+        tcPv6Stats.getConnectionFailures());
+    systemStats.setLong(OperatingSystemStats.tcpv6ConnectionsReset,
+        tcPv6Stats.getConnectionsReset());
+    systemStats.setLong(OperatingSystemStats.tcpv6SegmentsSent,
+        tcPv6Stats.getSegmentsSent());
+    systemStats.setLong(OperatingSystemStats.tcpv6SegmentsReceived,
+        tcPv6Stats.getSegmentsReceived());
+    systemStats.setLong(OperatingSystemStats.tcpv6SegmentsRetransmitted,
+        tcPv6Stats.getSegmentsRetransmitted());
+    systemStats.setLong(OperatingSystemStats.tcpv6InErrors,
+        tcPv6Stats.getInErrors());
+    systemStats.setLong(OperatingSystemStats.tcpv6OutResets,
+        tcPv6Stats.getOutResets());
+
+    final UdpStats udPv6Stats = internetProtocolStats.getUDPv6Stats();
+    systemStats.setLong(OperatingSystemStats.udpv6DatagramsSent,
+        udPv6Stats.getDatagramsSent());
+    systemStats.setLong(OperatingSystemStats.udpv6DatagramsReceived,
+        udPv6Stats.getDatagramsReceived());
+    systemStats.setLong(OperatingSystemStats.udpv6DatagramsNoPort,
+        udPv6Stats.getDatagramsNoPort());
+    systemStats.setLong(OperatingSystemStats.udpv6DatagramsReceivedErrors,
+        udPv6Stats.getDatagramsReceivedErrors());
+
+    final FileSystem fileSystem = operatingSystem.getFileSystem();
+    systemStats.setLong(OperatingSystemStats.openFileDescriptors,
+        fileSystem.getOpenFileDescriptors());
+    systemStats.setLong(OperatingSystemStats.maxFileDescriptors,
+        fileSystem.getMaxFileDescriptors());
+
+  }
+
+  private void sampleProcessors() {
+    final long[] currentFreq = processor.getCurrentFreq();
+    final double[] processorCpuLoad =
+        processor.getProcessorCpuLoadBetweenTicks(processorCpuLoadTicks);
+    processorCpuLoadTicks = processor.getProcessorCpuLoadTicks();
+
+    for (int i = 0; i < processorStats.length; i++) {
+      final Statistics processorStat = processorStats[i];
+      processorStat.setLong(ProcessorStats.currentFreq, currentFreq[i]);
+      processorStat.setDouble(ProcessorStats.processorCpuLoad, processorCpuLoad[i]);
 
-import org.apache.geode.internal.statistics.platform.OsStatisticsFactory;
+      long[] processorCpuLoadTick = processorCpuLoadTicks[i];
+      processorStat.setLong(ProcessorStats.processorCpuLoadTicksUSER,
+          processorCpuLoadTick[TickType.USER.getIndex()]);
+      processorStat.setLong(ProcessorStats.processorCpuLoadTicksNICE,
+          processorCpuLoadTick[TickType.NICE.getIndex()]);
+      processorStat.setLong(ProcessorStats.processorCpuLoadTicksSYSTEM,
+          processorCpuLoadTick[TickType.SYSTEM.getIndex()]);
+      processorStat.setLong(ProcessorStats.processorCpuLoadTicksIDLE,
+          processorCpuLoadTick[TickType.IDLE.getIndex()]);
+      processorStat.setLong(ProcessorStats.processorCpuLoadTicksIOWAIT,
+          processorCpuLoadTick[TickType.IOWAIT.getIndex()]);
+      processorStat.setLong(ProcessorStats.processorCpuLoadTicksIRQ,
+          processorCpuLoadTick[TickType.IRQ.getIndex()]);
+      processorStat.setLong(ProcessorStats.processorCpuLoadTicksSOFTIRQ,
+          processorCpuLoadTick[TickType.SOFTIRQ.getIndex()]);
+      processorStat.setLong(ProcessorStats.processorCpuLoadTicksSTEAL,
+          processorCpuLoadTick[TickType.STEAL.getIndex()]);
+    }
+  }
 
-public interface OshiStatisticsProvider {
-  void init(final @NotNull OsStatisticsFactory osStatisticsFactory,
-      final long pid) throws OshiStatisticsProviderException;
+  private void sampleNetworkInterfaces() {
+    for (int i = 0, size = networkIFs.size(); i < size; i++) {
+      final NetworkIF networkIF = networkIFs.get(i);
+      if (!networkIF.updateAttributes()) {
+        continue;
+      }
 
-  void sample();
+      final Statistics networkInterfaceStat = networkInterfaceStats[i];
+      networkInterfaceStat.setLong(NetworkInterfaceStats.mtu, networkIF.getMTU());
+      networkInterfaceStat.setLong(NetworkInterfaceStats.bytesReceived, networkIF.getBytesRecv());
+      networkInterfaceStat.setLong(NetworkInterfaceStats.bytesSent, networkIF.getBytesSent());
+      networkInterfaceStat.setLong(NetworkInterfaceStats.packetsReceived,
+          networkIF.getPacketsRecv());
+      networkInterfaceStat.setLong(NetworkInterfaceStats.packetsSent, networkIF.getPacketsSent());
+      networkInterfaceStat.setLong(NetworkInterfaceStats.inErrors, networkIF.getInErrors());
+      networkInterfaceStat.setLong(NetworkInterfaceStats.outErrors, networkIF.getOutErrors());
+      networkInterfaceStat.setLong(NetworkInterfaceStats.inDrops, networkIF.getInDrops());
+      networkInterfaceStat.setLong(NetworkInterfaceStats.collisions, networkIF.getCollisions());
+      networkInterfaceStat.setLong(NetworkInterfaceStats.speed, networkIF.getSpeed());
+    }
 
-  void destroy();
+  }
 }
diff --git a/geode-core/src/main/java/org/apache/geode/internal/statistics/oshi/OshiStatisticsProviderException.java b/geode-core/src/main/java/org/apache/geode/internal/statistics/oshi/OshiStatisticsProviderException.java
deleted file mode 100644
index 146a716..0000000
--- a/geode-core/src/main/java/org/apache/geode/internal/statistics/oshi/OshiStatisticsProviderException.java
+++ /dev/null
@@ -1,4 +0,0 @@
-package org.apache.geode.internal.statistics.oshi;
-
-public class OshiStatisticsProviderException extends Exception {
-}
diff --git a/geode-core/src/main/java/org/apache/geode/internal/statistics/oshi/OshiStatisticsProviderImpl.java b/geode-core/src/main/java/org/apache/geode/internal/statistics/oshi/OshiStatisticsProviderImpl.java
deleted file mode 100644
index 71201fc..0000000
--- a/geode-core/src/main/java/org/apache/geode/internal/statistics/oshi/OshiStatisticsProviderImpl.java
+++ /dev/null
@@ -1,307 +0,0 @@
-package org.apache.geode.internal.statistics.oshi;
-
-import java.util.List;
-
-import org.apache.logging.log4j.Logger;
-import org.jetbrains.annotations.NotNull;
-import oshi.SystemInfo;
-import oshi.hardware.CentralProcessor;
-import oshi.hardware.CentralProcessor.LogicalProcessor;
-import oshi.hardware.CentralProcessor.TickType;
-import oshi.hardware.GlobalMemory;
-import oshi.hardware.HardwareAbstractionLayer;
-import oshi.hardware.NetworkIF;
-import oshi.hardware.VirtualMemory;
-import oshi.software.os.FileSystem;
-import oshi.software.os.InternetProtocolStats;
-import oshi.software.os.InternetProtocolStats.TcpStats;
-import oshi.software.os.InternetProtocolStats.UdpStats;
-import oshi.software.os.OSProcess;
-import oshi.software.os.OperatingSystem;
-
-import org.apache.geode.Statistics;
-import org.apache.geode.internal.statistics.platform.OsStatisticsFactory;
-import org.apache.geode.logging.internal.log4j.api.LogService;
-
-public class OshiStatisticsProviderImpl implements OshiStatisticsProvider {
-  private static final Logger log = LogService.getLogger();
-
-  final SystemInfo systemInfo;
-
-  private int processId;
-  private CentralProcessor processor;
-  private OperatingSystem operatingSystem;
-  private HardwareAbstractionLayer hardware;
-  private List<NetworkIF> networkIFs;
-
-  private long[] systemCpuLoadTicks;
-  private long[][] processorCpuLoadTicks;
-  private OSProcess process;
-
-  private Statistics processStats;
-  private Statistics systemStats;
-  private Statistics[] processorStats;
-  private Statistics[] networkInterfaceStats;
-
-  public OshiStatisticsProviderImpl() {
-    this(new SystemInfo());
-  }
-
-  OshiStatisticsProviderImpl(final SystemInfo systemInfo) {
-    this.systemInfo = systemInfo;
-  }
-
-  @Override
-  public void init(final @NotNull OsStatisticsFactory osStatisticsFactory,
-      final long id) throws OshiStatisticsProviderException {
-
-    operatingSystem = systemInfo.getOperatingSystem();
-    processId = operatingSystem.getProcessId();
-    hardware = systemInfo.getHardware();
-    processor = hardware.getProcessor();
-
-    process = operatingSystem.getProcess(processId);
-    final String processIdentity = process.toString();
-    processStats = osStatisticsFactory.createOsStatistics(ProcessStats.getType(),
-        processIdentity, id, 0);
-
-    final String systemIdentity = operatingSystem.toString();
-    systemCpuLoadTicks = new long[TickType.values().length];
-    systemStats = osStatisticsFactory.createOsStatistics(OperatingSystemStats.getType(),
-        systemIdentity, id, 0);
-
-
-    final List<LogicalProcessor> logicalProcessors = processor.getLogicalProcessors();
-    processorCpuLoadTicks = new long[logicalProcessors.size()][TickType.values().length];
-    processorStats = new Statistics[logicalProcessors.size()];
-    for (int i = 0, size = logicalProcessors.size(); i < size; i++) {
-      final LogicalProcessor logicalProcessor = logicalProcessors.get(i);
-      final String processorIdentity = logicalProcessor.toString();
-      processorStats[i] = osStatisticsFactory.createOsStatistics(ProcessorStats.getType(),
-          processorIdentity, id, 0);
-    }
-
-    networkIFs = hardware.getNetworkIFs();
-    networkInterfaceStats = new Statistics[networkIFs.size()];
-    for (int i = 0, size = networkIFs.size(); i < size; i++) {
-      final NetworkIF networkIF = networkIFs.get(i);
-      final String displayName = networkIF.getDisplayName();
-      log.info("Creating network interfaces stats for {}", displayName);
-      networkInterfaceStats[i] =
-          osStatisticsFactory.createOsStatistics(NetworkInterfaceStats.getType(),
-              displayName, id, 0);
-    }
-  }
-
-  @Override
-  public void sample() {
-    sampleProcess();
-    sampleSystem();
-    sampleProcessors();
-    sampleNetworkInterfaces();
-  }
-
-  @Override
-  public void destroy() {}
-
-  void sampleProcess() {
-    final OSProcess process = operatingSystem.getProcess(processId);
-
-    final double processCpuLoadBetweenTicks = process.getProcessCpuLoadBetweenTicks(this.process);
-    processStats.setDouble(ProcessStats.cpuLoad, processCpuLoadBetweenTicks);
-    this.process = process;
-
-    processStats.setLong(ProcessStats.virtualSize, process.getVirtualSize());
-    processStats.setLong(ProcessStats.residentSetSize, process.getResidentSetSize());
-    processStats.setLong(ProcessStats.threadCount, process.getThreadCount());
-    processStats.setLong(ProcessStats.kernelTime, process.getKernelTime());
-    processStats.setLong(ProcessStats.userTime, process.getUserTime());
-    processStats.setLong(ProcessStats.bytesRead, process.getBytesRead());
-    processStats.setLong(ProcessStats.bytesWritten, process.getBytesWritten());
-    processStats.setLong(ProcessStats.openFiles, process.getOpenFiles());
-    processStats.setDouble(ProcessStats.cpuLoadCumulative, process.getProcessCpuLoadCumulative());
-    processStats.setLong(ProcessStats.minorFaults, process.getMinorFaults());
-    processStats.setLong(ProcessStats.majorFaults, process.getMajorFaults());
-    processStats.setLong(ProcessStats.contextSwitches, process.getContextSwitches());
-  }
-
-  private void sampleSystem() {
-    systemStats.setLong(OperatingSystemStats.processCount, operatingSystem.getProcessCount());
-    systemStats.setLong(OperatingSystemStats.threadCount, operatingSystem.getThreadCount());
-
-    final CentralProcessor processor = hardware.getProcessor();
-    systemStats.setLong(OperatingSystemStats.contextSwitches, processor.getContextSwitches());
-    systemStats.setLong(OperatingSystemStats.interrupts, processor.getInterrupts());
-    systemStats.setLong(OperatingSystemStats.physicalProcessorCount,
-        processor.getPhysicalProcessorCount());
-    systemStats.setLong(OperatingSystemStats.logicalProcessorCount,
-        processor.getLogicalProcessorCount());
-    systemStats.setLong(OperatingSystemStats.maxFreq, processor.getMaxFreq());
-
-    final double[] systemLoadAverage = processor.getSystemLoadAverage(3);
-    systemStats.setDouble(OperatingSystemStats.systemLoadAverage1, systemLoadAverage[0]);
-    systemStats.setDouble(OperatingSystemStats.systemLoadAverage5, systemLoadAverage[1]);
-    systemStats.setDouble(OperatingSystemStats.systemLoadAverage15, systemLoadAverage[2]);
-
-    final double systemCpuLoadBetweenTicks =
-        processor.getSystemCpuLoadBetweenTicks(systemCpuLoadTicks);
-    systemStats.setDouble(OperatingSystemStats.systemCpuLoad, systemCpuLoadBetweenTicks);
-
-    systemCpuLoadTicks = processor.getSystemCpuLoadTicks();
-    systemStats.setLong(OperatingSystemStats.systemCpuLoadTicksUSER,
-        systemCpuLoadTicks[TickType.USER.getIndex()]);
-    systemStats.setLong(OperatingSystemStats.systemCpuLoadTicksNICE,
-        systemCpuLoadTicks[TickType.NICE.getIndex()]);
-    systemStats.setLong(OperatingSystemStats.systemCpuLoadTicksSYSTEM,
-        systemCpuLoadTicks[TickType.SYSTEM.getIndex()]);
-    systemStats.setLong(OperatingSystemStats.systemCpuLoadTicksIDLE,
-        systemCpuLoadTicks[TickType.IDLE.getIndex()]);
-    systemStats.setLong(OperatingSystemStats.systemCpuLoadTicksIOWAIT,
-        systemCpuLoadTicks[TickType.IOWAIT.getIndex()]);
-    systemStats.setLong(OperatingSystemStats.systemCpuLoadTicksIRQ,
-        systemCpuLoadTicks[TickType.IRQ.getIndex()]);
-    systemStats.setLong(OperatingSystemStats.systemCpuLoadTicksSOFTIRQ,
-        systemCpuLoadTicks[TickType.SOFTIRQ.getIndex()]);
-    systemStats.setLong(OperatingSystemStats.systemCpuLoadTicksSTEAL,
-        systemCpuLoadTicks[TickType.STEAL.getIndex()]);
-
-    final GlobalMemory memory = hardware.getMemory();
-    systemStats.setLong(OperatingSystemStats.memoryTotal, memory.getTotal());
-    systemStats.setLong(OperatingSystemStats.memoryAvailable, memory.getAvailable());
-    systemStats.setLong(OperatingSystemStats.memoryPageSize, memory.getPageSize());
-
-    final VirtualMemory virtualMemory = memory.getVirtualMemory();
-    systemStats.setLong(OperatingSystemStats.swapTotal, virtualMemory.getSwapTotal());
-    systemStats.setLong(OperatingSystemStats.swapUsed, virtualMemory.getSwapUsed());
-    systemStats.setLong(OperatingSystemStats.virtualMax, virtualMemory.getVirtualMax());
-    systemStats.setLong(OperatingSystemStats.virtualInUse, virtualMemory.getVirtualInUse());
-    systemStats.setLong(OperatingSystemStats.swapPagesIn, virtualMemory.getSwapPagesIn());
-    systemStats.setLong(OperatingSystemStats.swapPagesOut, virtualMemory.getSwapPagesOut());
-
-    final InternetProtocolStats internetProtocolStats = operatingSystem.getInternetProtocolStats();
-    final TcpStats tcPv4Stats = internetProtocolStats.getTCPv4Stats();
-    systemStats.setLong(OperatingSystemStats.tcpv4ConnectionsEstablished,
-        tcPv4Stats.getConnectionsEstablished());
-    systemStats.setLong(OperatingSystemStats.tcpv4ConnectionsActive,
-        tcPv4Stats.getConnectionsActive());
-    systemStats.setLong(OperatingSystemStats.tcpv4ConnectionsPassive,
-        tcPv4Stats.getConnectionsPassive());
-    systemStats.setLong(OperatingSystemStats.tcpv4ConnectionFailures,
-        tcPv4Stats.getConnectionFailures());
-    systemStats.setLong(OperatingSystemStats.tcpv4ConnectionsReset,
-        tcPv4Stats.getConnectionsReset());
-    systemStats.setLong(OperatingSystemStats.tcpv4SegmentsSent,
-        tcPv4Stats.getSegmentsSent());
-    systemStats.setLong(OperatingSystemStats.tcpv4SegmentsReceived,
-        tcPv4Stats.getSegmentsReceived());
-    systemStats.setLong(OperatingSystemStats.tcpv4SegmentsRetransmitted,
-        tcPv4Stats.getSegmentsRetransmitted());
-    systemStats.setLong(OperatingSystemStats.tcpv4InErrors,
-        tcPv4Stats.getInErrors());
-    systemStats.setLong(OperatingSystemStats.tcpv4OutResets,
-        tcPv4Stats.getOutResets());
-
-    final UdpStats udPv4Stats = internetProtocolStats.getUDPv4Stats();
-    systemStats.setLong(OperatingSystemStats.udpv4DatagramsSent,
-        udPv4Stats.getDatagramsSent());
-    systemStats.setLong(OperatingSystemStats.udpv4DatagramsReceived,
-        udPv4Stats.getDatagramsReceived());
-    systemStats.setLong(OperatingSystemStats.udpv4DatagramsNoPort,
-        udPv4Stats.getDatagramsNoPort());
-    systemStats.setLong(OperatingSystemStats.udpv4DatagramsReceivedErrors,
-        udPv4Stats.getDatagramsReceivedErrors());
-
-    final TcpStats tcPv6Stats = internetProtocolStats.getTCPv6Stats();
-    systemStats.setLong(OperatingSystemStats.tcpv6ConnectionsEstablished,
-        tcPv6Stats.getConnectionsEstablished());
-    systemStats.setLong(OperatingSystemStats.tcpv6ConnectionsActive,
-        tcPv6Stats.getConnectionsActive());
-    systemStats.setLong(OperatingSystemStats.tcpv6ConnectionsPassive,
-        tcPv6Stats.getConnectionsPassive());
-    systemStats.setLong(OperatingSystemStats.tcpv6ConnectionFailures,
-        tcPv6Stats.getConnectionFailures());
-    systemStats.setLong(OperatingSystemStats.tcpv6ConnectionsReset,
-        tcPv6Stats.getConnectionsReset());
-    systemStats.setLong(OperatingSystemStats.tcpv6SegmentsSent,
-        tcPv6Stats.getSegmentsSent());
-    systemStats.setLong(OperatingSystemStats.tcpv6SegmentsReceived,
-        tcPv6Stats.getSegmentsReceived());
-    systemStats.setLong(OperatingSystemStats.tcpv6SegmentsRetransmitted,
-        tcPv6Stats.getSegmentsRetransmitted());
-    systemStats.setLong(OperatingSystemStats.tcpv6InErrors,
-        tcPv6Stats.getInErrors());
-    systemStats.setLong(OperatingSystemStats.tcpv6OutResets,
-        tcPv6Stats.getOutResets());
-
-    final UdpStats udPv6Stats = internetProtocolStats.getUDPv6Stats();
-    systemStats.setLong(OperatingSystemStats.udpv6DatagramsSent,
-        udPv6Stats.getDatagramsSent());
-    systemStats.setLong(OperatingSystemStats.udpv6DatagramsReceived,
-        udPv6Stats.getDatagramsReceived());
-    systemStats.setLong(OperatingSystemStats.udpv6DatagramsNoPort,
-        udPv6Stats.getDatagramsNoPort());
-    systemStats.setLong(OperatingSystemStats.udpv6DatagramsReceivedErrors,
-        udPv6Stats.getDatagramsReceivedErrors());
-
-    final FileSystem fileSystem = operatingSystem.getFileSystem();
-    systemStats.setLong(OperatingSystemStats.openFileDescriptors,
-        fileSystem.getOpenFileDescriptors());
-    systemStats.setLong(OperatingSystemStats.maxFileDescriptors,
-        fileSystem.getMaxFileDescriptors());
-
-  }
-
-  private void sampleProcessors() {
-    final long[] currentFreq = processor.getCurrentFreq();
-    final double[] processorCpuLoad =
-        processor.getProcessorCpuLoadBetweenTicks(processorCpuLoadTicks);
-    processorCpuLoadTicks = processor.getProcessorCpuLoadTicks();
-
-    for (int i = 0; i < processorStats.length; i++) {
-      final Statistics processorStat = processorStats[i];
-      processorStat.setLong(ProcessorStats.currentFreq, currentFreq[i]);
-      processorStat.setDouble(ProcessorStats.processorCpuLoad, processorCpuLoad[i]);
-
-      long[] processorCpuLoadTick = processorCpuLoadTicks[i];
-      processorStat.setLong(ProcessorStats.processorCpuLoadTicksUSER,
-          processorCpuLoadTick[TickType.USER.getIndex()]);
-      processorStat.setLong(ProcessorStats.processorCpuLoadTicksNICE,
-          processorCpuLoadTick[TickType.NICE.getIndex()]);
-      processorStat.setLong(ProcessorStats.processorCpuLoadTicksSYSTEM,
-          processorCpuLoadTick[TickType.SYSTEM.getIndex()]);
-      processorStat.setLong(ProcessorStats.processorCpuLoadTicksIDLE,
-          processorCpuLoadTick[TickType.IDLE.getIndex()]);
-      processorStat.setLong(ProcessorStats.processorCpuLoadTicksIOWAIT,
-          processorCpuLoadTick[TickType.IOWAIT.getIndex()]);
-      processorStat.setLong(ProcessorStats.processorCpuLoadTicksIRQ,
-          processorCpuLoadTick[TickType.IRQ.getIndex()]);
-      processorStat.setLong(ProcessorStats.processorCpuLoadTicksSOFTIRQ,
-          processorCpuLoadTick[TickType.SOFTIRQ.getIndex()]);
-      processorStat.setLong(ProcessorStats.processorCpuLoadTicksSTEAL,
-          processorCpuLoadTick[TickType.STEAL.getIndex()]);
-    }
-  }
-
-  private void sampleNetworkInterfaces() {
-    for (int i = 0, size = networkIFs.size(); i < size; i++) {
-      final NetworkIF networkIF = networkIFs.get(i);
-      if (!networkIF.updateAttributes()) {
-        continue;
-      }
-
-      final Statistics networkInterfaceStat = networkInterfaceStats[i];
-      networkInterfaceStat.setLong(NetworkInterfaceStats.mtu, networkIF.getMTU());
-      networkInterfaceStat.setLong(NetworkInterfaceStats.bytesReceived, networkIF.getBytesRecv());
-      networkInterfaceStat.setLong(NetworkInterfaceStats.bytesSent, networkIF.getBytesSent());
-      networkInterfaceStat.setLong(NetworkInterfaceStats.packetsReceived,
-          networkIF.getPacketsRecv());
-      networkInterfaceStat.setLong(NetworkInterfaceStats.packetsSent, networkIF.getPacketsSent());
-      networkInterfaceStat.setLong(NetworkInterfaceStats.inErrors, networkIF.getInErrors());
-      networkInterfaceStat.setLong(NetworkInterfaceStats.outErrors, networkIF.getOutErrors());
-      networkInterfaceStat.setLong(NetworkInterfaceStats.inDrops, networkIF.getInDrops());
-      networkInterfaceStat.setLong(NetworkInterfaceStats.collisions, networkIF.getCollisions());
-      networkInterfaceStat.setLong(NetworkInterfaceStats.speed, networkIF.getSpeed());
-    }
-
-  }
-}
diff --git a/geode-core/src/main/java/org/apache/geode/internal/statistics/platform/LinuxProcFsStatistics.java b/geode-core/src/main/java/org/apache/geode/internal/statistics/platform/LinuxProcFsStatistics.java
index b418ef4..e8f3720 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/statistics/platform/LinuxProcFsStatistics.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/statistics/platform/LinuxProcFsStatistics.java
@@ -28,11 +28,12 @@ import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
 import org.apache.logging.log4j.Logger;
+import org.jetbrains.annotations.NotNull;
 
+import org.apache.geode.Statistics;
 import org.apache.geode.annotations.Immutable;
 import org.apache.geode.annotations.VisibleForTesting;
 import org.apache.geode.annotations.internal.MakeNotStatic;
-import org.apache.geode.internal.statistics.LocalStatisticsImpl;
 import org.apache.geode.util.internal.GeodeGlossary;
 
 public class LinuxProcFsStatistics {
@@ -110,16 +111,13 @@ public class LinuxProcFsStatistics {
     cpuStatSingleton = null;
   }
 
-  public static void readyRefresh() { // TODO: was package-protected
-  }
-
   /*
    * get the statistics for the specified process. ( pid_rssSize, pid_imageSize ) vsize is assumed
    * to be in units of kbytes System property gemfire.statistics.pagesSize can be used to configure
    * pageSize. This is the mem_unit member of the struct returned by sysinfo()
    *
    */
-  public static void refreshProcess(int pid, LocalStatisticsImpl stats) {
+  public static void refreshProcess(int pid, @NotNull final Statistics stats) {
     // Just incase a pid is not available
     if (pid == 0) {
       return;
@@ -151,13 +149,14 @@ public class LinuxProcFsStatistics {
     }
   }
 
-  public static void refreshSystem(LocalStatisticsImpl stats) {
+  public static void refreshSystem(@NotNull final Statistics stats) {
     refreshSystem(stats, "/proc/stat", "/proc/net/netstat");
   }
 
   @VisibleForTesting
-  public static void refreshSystem(LocalStatisticsImpl stats, String statFilePath,
-      String netstatStatsFilePath) {
+  public static void refreshSystem(@NotNull final Statistics stats,
+      final @NotNull String statFilePath,
+      final @NotNull String netstatStatsFilePath) {
     if (cpuStatSingleton == null) {
       // stats have been closed or haven't been properly initialized
       return;
@@ -224,7 +223,7 @@ public class LinuxProcFsStatistics {
 
   // Example of /proc/loadavg
   // 0.00 0.00 0.07 1/218 7907
-  private static void getLoadAvg(LocalStatisticsImpl stats) {
+  private static void getLoadAvg(@NotNull final Statistics stats) {
     try (FileInputStream fileInputStream = new FileInputStream("/proc/loadavg");
         InputStreamReader isr = new InputStreamReader(fileInputStream);
         BufferedReader br = new BufferedReader(isr, 512)) {
@@ -277,7 +276,7 @@ public class LinuxProcFsStatistics {
   // total: used: free: shared: buffers: cached:
   // Mem: 4118380544 3816050688 302329856 0 109404160 3060326400
   // Swap: 4194881536 127942656 4066938880
-  private static void getMemInfo(LocalStatisticsImpl stats) {
+  private static void getMemInfo(@NotNull final Statistics stats) {
     try (FileInputStream fileInputStream = new FileInputStream("/proc/meminfo");
         InputStreamReader isr = new InputStreamReader(fileInputStream);
         BufferedReader br = new BufferedReader(isr)) {
@@ -334,8 +333,8 @@ public class LinuxProcFsStatistics {
     }
   }
 
-  private static void getNetStatStats(LocalStatisticsImpl stats,
-      String netstatStatsFilePath) {
+  private static void getNetStatStats(@NotNull final Statistics stats,
+      final @NotNull String netstatStatsFilePath) {
     SpaceTokenizer headerTokenizer = new SpaceTokenizer();
     try (FileInputStream netstatStatsFileInputStream = new FileInputStream(netstatStatsFilePath);
         InputStreamReader isr = new InputStreamReader(netstatStatsFileInputStream);
@@ -409,7 +408,7 @@ public class LinuxProcFsStatistics {
    * 326949246 0 0 0 0 0 0
    */
 
-  private static void getNetStats(LocalStatisticsImpl stats) {
+  private static void getNetStats(@NotNull final Statistics stats) {
     try (FileInputStream fileInputStream = new FileInputStream("/proc/net/dev");
         InputStreamReader isr = new InputStreamReader(fileInputStream);
         BufferedReader br = new BufferedReader(isr)) {
@@ -499,7 +498,7 @@ public class LinuxProcFsStatistics {
   // 2024138028
   // 8 17 sdb1 12601113 213085114 216407197 1731257800
   // 3 0 hda 0 0 0 0 0 0 0 0 0 0 0
-  private static void getDiskStats(LocalStatisticsImpl stats) {
+  private static void getDiskStats(@NotNull final Statistics stats) {
     InputStreamReader isr = null;
     BufferedReader br = null;
     String line;
@@ -611,7 +610,7 @@ public class LinuxProcFsStatistics {
   // pgpgout 1057420300
   // pswpin 19422
   // pswpout 14495
-  private static void getVmStats(LocalStatisticsImpl stats) {
+  private static void getVmStats(@NotNull final Statistics stats) {
     assert hasProcVmStat : "getVmStats called when hasVmStat was false";
     try (FileInputStream fileInputStream = new FileInputStream("/proc/vmstat");
         InputStreamReader isr = new InputStreamReader(fileInputStream);
@@ -843,8 +842,8 @@ public class LinuxProcFsStatistics {
 
     protected void skipTokens(int numberToSkip) {
       int remaining = numberToSkip + 1;
+      // noinspection StatementWithEmptyBody
       while (--remaining > 0 && skipToken()) {
-        ;
       }
     }
 
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/beans/MemberMBeanBridge.java b/geode-core/src/main/java/org/apache/geode/management/internal/beans/MemberMBeanBridge.java
index 993c617..e2e8aa6 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/beans/MemberMBeanBridge.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/beans/MemberMBeanBridge.java
@@ -73,10 +73,10 @@ import org.apache.geode.internal.offheap.OffHeapMemoryStats;
 import org.apache.geode.internal.process.PidUnavailableException;
 import org.apache.geode.internal.process.ProcessUtils;
 import org.apache.geode.internal.serialization.KnownVersion;
-import org.apache.geode.internal.statistics.OsStatisticsProvider;
 import org.apache.geode.internal.statistics.StatSamplerStats;
 import org.apache.geode.internal.statistics.StatisticsManager;
 import org.apache.geode.internal.statistics.VMStatsContract;
+import org.apache.geode.internal.statistics.legacy.LegacyOsStatisticsProvider;
 import org.apache.geode.internal.statistics.platform.LinuxSystemStats;
 import org.apache.geode.internal.statistics.platform.ProcessStats;
 import org.apache.geode.internal.stats50.VMStats50;
@@ -110,7 +110,8 @@ public class MemberMBeanBridge {
   private static final String MEMBER_LEVEL_REGION_MONITOR = "MemberLevelRegionMonitor";
   private static final long MBFactor = 1024 * 1024;
 
-  private final OsStatisticsProvider osStatisticsProvider = OsStatisticsProvider.build();
+  private final LegacyOsStatisticsProvider osStatisticsProvider =
+      LegacyOsStatisticsProvider.build();
 
   private InternalCache cache;
   private DistributionConfig config;
diff --git a/geode-core/src/test/java/org/apache/geode/internal/statistics/LocalStatisticsImplTest.java b/geode-core/src/test/java/org/apache/geode/internal/statistics/LocalStatisticsImplTest.java
index d1a6032..54bc66f 100644
--- a/geode-core/src/test/java/org/apache/geode/internal/statistics/LocalStatisticsImplTest.java
+++ b/geode-core/src/test/java/org/apache/geode/internal/statistics/LocalStatisticsImplTest.java
@@ -37,7 +37,7 @@ public class LocalStatisticsImplTest {
     };
     StatisticsType statisticsType = new StatisticsTypeImpl("abc", "mock stats", stats);
     LocalStatisticsImpl localStatistics = new LocalStatisticsImpl(statisticsType, "abc", 123L, 123L,
-        false, 90, mockStatisticsManager);
+        false, mockStatisticsManager);
 
     localStatistics.incInt("intCount", 7);
     localStatistics.incLong("longCount", 15);
@@ -65,7 +65,7 @@ public class LocalStatisticsImplTest {
     };
     StatisticsType statisticsType = new StatisticsTypeImpl("abc", "mock stats", stats);
     LocalStatisticsImpl localStatistics = new LocalStatisticsImpl(statisticsType, "abc", 123L, 123L,
-        false, 90, mockStatisticsManager);
+        false, mockStatisticsManager);
 
     localStatistics.incInt("intCount", 7);
     localStatistics.incLong("longCount", 15);
diff --git a/geode-core/src/test/java/org/apache/geode/internal/statistics/OsStatisticsProviderTest.java b/geode-core/src/test/java/org/apache/geode/internal/statistics/OsStatisticsProviderTest.java
deleted file mode 100644
index 4c3e8f2..0000000
--- a/geode-core/src/test/java/org/apache/geode/internal/statistics/OsStatisticsProviderTest.java
+++ /dev/null
@@ -1,58 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
- * agreements. See the NOTICE file distributed with this work for additional information regarding
- * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License. You may obtain a
- * copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software distributed under the License
- * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
- * or implied. See the License for the specific language governing permissions and limitations under
- * the License.
- */
-package org.apache.geode.internal.statistics;
-
-import static org.assertj.core.api.Assertions.assertThat;
-
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.contrib.java.lang.system.RestoreSystemProperties;
-
-public class OsStatisticsProviderTest {
-
-  @Rule
-  public RestoreSystemProperties restoreSystemProperties = new RestoreSystemProperties();
-
-  @Test
-  public void givenLinuxOs_thenOsStatsAreSupported() {
-    System.setProperty("os.name", "Linux");
-    assertThat(OsStatisticsProvider.build().osStatsSupported()).isTrue();
-  }
-
-  @Test
-  public void givenWindowsOs_thenOsStatsAreNotSupported() {
-    System.setProperty("os.name", "Windows");
-    assertThat(OsStatisticsProvider.build().osStatsSupported()).isFalse();
-  }
-
-  @Test
-  public void givenMacOs_thenOsStatsAreNotSupported() {
-    System.setProperty("os.name", "Mac OS X");
-    assertThat(OsStatisticsProvider.build().osStatsSupported()).isFalse();
-  }
-
-  @Test
-  public void givenSolarisOs_thenOsStatsAreNotSupported() {
-    System.setProperty("os.name", "SunOS");
-    assertThat(OsStatisticsProvider.build().osStatsSupported()).isFalse();
-  }
-
-  @Test
-  public void givenUnknownOs_thenOsStatsAreNotSupported() {
-    System.setProperty("os.name", "AnyOtherOS");
-    assertThat(OsStatisticsProvider.build().osStatsSupported()).isFalse();
-  }
-
-}
diff --git a/geode-core/src/test/java/org/apache/geode/internal/statistics/StatisticsImplTest.java b/geode-core/src/test/java/org/apache/geode/internal/statistics/StatisticsImplTest.java
index 0d3f9c3..aa75ca6 100644
--- a/geode-core/src/test/java/org/apache/geode/internal/statistics/StatisticsImplTest.java
+++ b/geode-core/src/test/java/org/apache/geode/internal/statistics/StatisticsImplTest.java
@@ -219,12 +219,12 @@ public class StatisticsImplTest {
 
     SimpleStatistics(StatisticsType type, String textId, long numericId, long uniqueId,
         int osStatFlags, StatisticsManager statisticsManager) {
-      super(type, textId, numericId, uniqueId, osStatFlags, statisticsManager);
+      super(type, textId, numericId, uniqueId, statisticsManager);
     }
 
     SimpleStatistics(StatisticsType type, String textId, long numericId, long uniqueId,
         int osStatFlags, StatisticsManager statisticsManager, StatisticsLogger statisticsLogger) {
-      super(type, textId, numericId, uniqueId, osStatFlags, statisticsManager, statisticsLogger);
+      super(type, textId, numericId, uniqueId, statisticsManager, statisticsLogger);
     }
 
     @Override
diff --git a/geode-core/src/test/java/org/apache/geode/internal/statistics/StatisticsRegistryTest.java b/geode-core/src/test/java/org/apache/geode/internal/statistics/StatisticsRegistryTest.java
index 3e6ae29..0f8932a 100644
--- a/geode-core/src/test/java/org/apache/geode/internal/statistics/StatisticsRegistryTest.java
+++ b/geode-core/src/test/java/org/apache/geode/internal/statistics/StatisticsRegistryTest.java
@@ -18,7 +18,6 @@ import static org.assertj.core.api.Assertions.assertThat;
 import static org.assertj.core.api.Assertions.catchThrowable;
 import static org.mockito.ArgumentMatchers.any;
 import static org.mockito.ArgumentMatchers.anyBoolean;
-import static org.mockito.ArgumentMatchers.anyInt;
 import static org.mockito.ArgumentMatchers.anyLong;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.verify;
@@ -312,11 +311,10 @@ public class StatisticsRegistryTest {
   public void createsOsStatisticsViaFactory() {
     Statistics statisticsCreatedByFactory = mock(Statistics.class);
 
-    when(osStatisticsFactory.create(any(), any(), anyLong(), anyLong(), anyInt(), any()))
+    when(osStatisticsFactory.create(any(), any(), anyLong(), anyLong(), any()))
         .thenReturn(statisticsCreatedByFactory);
 
-    Statistics result = registry.createOsStatistics(type, STATISTICS_TEXT_ID, STATISTICS_NUMERIC_ID,
-        STATISTICS_OS_FLAGS);
+    Statistics result = registry.createOsStatistics(type, STATISTICS_TEXT_ID, STATISTICS_NUMERIC_ID);
 
     assertThat(result)
         .isSameAs(statisticsCreatedByFactory);
@@ -338,20 +336,20 @@ public class StatisticsRegistryTest {
 
   @Test
   public void incrementsUniqueIdForEachCreatedStatistics() {
-    registry.createOsStatistics(type, STATISTICS_TEXT_ID, 0, 0);
-    verify(osStatisticsFactory).create(type, STATISTICS_TEXT_ID, 0L, 1, 0, registry);
+    registry.createOsStatistics(type, STATISTICS_TEXT_ID, 0);
+    verify(osStatisticsFactory).create(type, STATISTICS_TEXT_ID, 0L, 1, registry);
 
-    registry.createOsStatistics(type, STATISTICS_TEXT_ID, 0, 0);
-    verify(osStatisticsFactory).create(type, STATISTICS_TEXT_ID, 0, 2, 0, registry);
+    registry.createOsStatistics(type, STATISTICS_TEXT_ID, 0);
+    verify(osStatisticsFactory).create(type, STATISTICS_TEXT_ID, 0, 2, registry);
 
-    registry.createOsStatistics(type, STATISTICS_TEXT_ID, 0, 0);
-    verify(osStatisticsFactory).create(type, STATISTICS_TEXT_ID, 0, 3, 0, registry);
+    registry.createOsStatistics(type, STATISTICS_TEXT_ID, 0);
+    verify(osStatisticsFactory).create(type, STATISTICS_TEXT_ID, 0, 3, registry);
 
     registry.createAtomicStatistics(type, STATISTICS_TEXT_ID, 0);
     verify(atomicStatisticsFactory).create(type, STATISTICS_TEXT_ID, 0, 4, registry);
 
-    registry.createOsStatistics(type, STATISTICS_TEXT_ID, 0, 0);
-    verify(osStatisticsFactory).create(type, STATISTICS_TEXT_ID, 0, 5, 0, registry);
+    registry.createOsStatistics(type, STATISTICS_TEXT_ID, 0);
+    verify(osStatisticsFactory).create(type, STATISTICS_TEXT_ID, 0, 5, registry);
 
     registry.createAtomicStatistics(type, STATISTICS_TEXT_ID, 0);
     verify(atomicStatisticsFactory).create(type, STATISTICS_TEXT_ID, 0, 6, registry);
@@ -369,7 +367,7 @@ public class StatisticsRegistryTest {
     Statistics osStatistics2 = mock(Statistics.class, "os 1");
     Statistics osStatistics3 = mock(Statistics.class, "os 1");
 
-    when(osStatisticsFactory.create(any(), any(), anyLong(), anyLong(), anyInt(), any()))
+    when(osStatisticsFactory.create(any(), any(), anyLong(), anyLong(), any()))
         .thenReturn(osStatistics1)
         .thenReturn(osStatistics2)
         .thenReturn(osStatistics3);
@@ -380,13 +378,10 @@ public class StatisticsRegistryTest {
         .thenReturn(atomicStatistics3);
 
     registry.createAtomicStatistics(type, STATISTICS_TEXT_ID, STATISTICS_NUMERIC_ID);
-    registry.createOsStatistics(type, STATISTICS_TEXT_ID, STATISTICS_NUMERIC_ID,
-        STATISTICS_OS_FLAGS);
-    registry.createOsStatistics(type, STATISTICS_TEXT_ID, STATISTICS_NUMERIC_ID,
-        STATISTICS_OS_FLAGS);
+    registry.createOsStatistics(type, STATISTICS_TEXT_ID, STATISTICS_NUMERIC_ID);
+    registry.createOsStatistics(type, STATISTICS_TEXT_ID, STATISTICS_NUMERIC_ID);
     registry.createAtomicStatistics(type, STATISTICS_TEXT_ID, STATISTICS_NUMERIC_ID);
-    registry.createOsStatistics(type, STATISTICS_TEXT_ID, STATISTICS_NUMERIC_ID,
-        STATISTICS_OS_FLAGS);
+    registry.createOsStatistics(type, STATISTICS_TEXT_ID, STATISTICS_NUMERIC_ID);
     registry.createAtomicStatistics(type, STATISTICS_TEXT_ID, STATISTICS_NUMERIC_ID);
 
     assertThat(registry.getStatsList())
@@ -404,7 +399,7 @@ public class StatisticsRegistryTest {
     Statistics osStatistics1 = mock(Statistics.class, "os 1");
     Statistics osStatistics2 = mock(Statistics.class, "os 2");
     Statistics osStatistics3 = mock(Statistics.class, "os 3");
-    when(osStatisticsFactory.create(any(), any(), anyLong(), anyLong(), anyInt(), any()))
+    when(osStatisticsFactory.create(any(), any(), anyLong(), anyLong(), any()))
         .thenReturn(osStatistics1)
         .thenReturn(osStatistics2)
         .thenReturn(osStatistics3);
@@ -418,13 +413,10 @@ public class StatisticsRegistryTest {
         .thenReturn(atomicStatistics3);
 
     registry.createAtomicStatistics(type, STATISTICS_TEXT_ID, STATISTICS_NUMERIC_ID);
-    registry.createOsStatistics(type, STATISTICS_TEXT_ID, STATISTICS_NUMERIC_ID,
-        STATISTICS_OS_FLAGS);
-    registry.createOsStatistics(type, STATISTICS_TEXT_ID, STATISTICS_NUMERIC_ID,
-        STATISTICS_OS_FLAGS);
+    registry.createOsStatistics(type, STATISTICS_TEXT_ID, STATISTICS_NUMERIC_ID);
+    registry.createOsStatistics(type, STATISTICS_TEXT_ID, STATISTICS_NUMERIC_ID);
     registry.createAtomicStatistics(type, STATISTICS_TEXT_ID, STATISTICS_NUMERIC_ID);
-    registry.createOsStatistics(type, STATISTICS_TEXT_ID, STATISTICS_NUMERIC_ID,
-        STATISTICS_OS_FLAGS);
+    registry.createOsStatistics(type, STATISTICS_TEXT_ID, STATISTICS_NUMERIC_ID);
     registry.createAtomicStatistics(type, STATISTICS_TEXT_ID, STATISTICS_NUMERIC_ID);
 
     registry.destroyStatistics(osStatistics2);
@@ -449,7 +441,7 @@ public class StatisticsRegistryTest {
     Statistics osStatistics = mock(Statistics.class, "os");
     Statistics atomicStatistics = mock(Statistics.class, "atomic");
 
-    when(osStatisticsFactory.create(any(), any(), anyLong(), anyLong(), anyInt(), any()))
+    when(osStatisticsFactory.create(any(), any(), anyLong(), anyLong(), any()))
         .thenReturn(osStatistics);
 
     when(atomicStatisticsFactory.create(any(), any(), anyLong(), anyLong(), any()))
@@ -460,8 +452,7 @@ public class StatisticsRegistryTest {
         .as("modification count after first modification")
         .isEqualTo(1);
 
-    registry.createOsStatistics(type, STATISTICS_TEXT_ID, STATISTICS_NUMERIC_ID,
-        STATISTICS_OS_FLAGS);
+    registry.createOsStatistics(type, STATISTICS_TEXT_ID, STATISTICS_NUMERIC_ID);
     assertThat(registry.getStatListModCount())
         .as("modification count after second modification")
         .isEqualTo(2);
diff --git a/geode-core/src/jmhTest/java/org/apache/geode/internal/statistics/platform/OshiStatisticsBenchmarkTest.java b/geode-core/src/test/java/org/apache/geode/internal/statistics/legacy/LegacyOsStatisticsProviderTest.java
similarity index 69%
rename from geode-core/src/jmhTest/java/org/apache/geode/internal/statistics/platform/OshiStatisticsBenchmarkTest.java
rename to geode-core/src/test/java/org/apache/geode/internal/statistics/legacy/LegacyOsStatisticsProviderTest.java
index db0d965..90e2037 100644
--- a/geode-core/src/jmhTest/java/org/apache/geode/internal/statistics/platform/OshiStatisticsBenchmarkTest.java
+++ b/geode-core/src/test/java/org/apache/geode/internal/statistics/legacy/LegacyOsStatisticsProviderTest.java
@@ -13,13 +13,19 @@
  * the License.
  */
 
-package org.apache.geode.internal.statistics.platform;
+package org.apache.geode.internal.statistics.legacy;
 
+import org.assertj.core.api.Assertions;
 import org.junit.Test;
 
-public class OshiStatisticsBenchmarkTest {
-  @Test
-  public void getsValidValues() {
+import org.apache.geode.internal.lang.SystemUtils;
+
+public class LegacyOsStatisticsProviderTest {
 
+  @Test
+  public void osSystemSupportedOnLinuxOnly() {
+    Assertions.assertThat(LegacyOsStatisticsProvider.build().osStatsSupported()).isEqualTo(
+        SystemUtils.isLinux());
   }
+
 }
diff --git a/geode-core/src/test/java/org/apache/geode/internal/statistics/oshi/OshiStatisticsProviderImplTest.java b/geode-core/src/test/java/org/apache/geode/internal/statistics/oshi/OshiStatisticsProviderTest.java
similarity index 89%
rename from geode-core/src/test/java/org/apache/geode/internal/statistics/oshi/OshiStatisticsProviderImplTest.java
rename to geode-core/src/test/java/org/apache/geode/internal/statistics/oshi/OshiStatisticsProviderTest.java
index c5244f3..f8c4d01 100644
--- a/geode-core/src/test/java/org/apache/geode/internal/statistics/oshi/OshiStatisticsProviderImplTest.java
+++ b/geode-core/src/test/java/org/apache/geode/internal/statistics/oshi/OshiStatisticsProviderTest.java
@@ -17,7 +17,6 @@ package org.apache.geode.internal.statistics.oshi;
 
 import static java.util.Arrays.asList;
 import static org.mockito.ArgumentMatchers.any;
-import static org.mockito.ArgumentMatchers.anyInt;
 import static org.mockito.ArgumentMatchers.anyLong;
 import static org.mockito.ArgumentMatchers.eq;
 import static org.mockito.Mockito.mock;
@@ -34,9 +33,10 @@ import oshi.software.os.OSProcess;
 import oshi.software.os.OperatingSystem;
 
 import org.apache.geode.Statistics;
-import org.apache.geode.internal.statistics.platform.OsStatisticsFactory;
+import org.apache.geode.internal.statistics.OsStatisticsFactory;
+import org.apache.geode.internal.statistics.OsStatisticsProviderException;
 
-public class OshiStatisticsProviderImplTest {
+public class OshiStatisticsProviderTest {
 
   private static final String SYSTEM_IDENTITY = "mock-operating-system";
   private static final String PROCESS_IDENTITY = "mock-process";
@@ -45,12 +45,12 @@ public class OshiStatisticsProviderImplTest {
   private static final String NETWORK_IF_0_DISPLAY_NAME = "mock-if0";
   private static final String NETWORK_IF_1_DISPLAY_NAME = "mock-if1";
 
-  private final OshiStatisticsProviderImpl oshiStatisticsProvider;
+  private final OshiStatisticsProvider oshiStatisticsProvider;
   private final OSProcess osProcess;
 
-  public OshiStatisticsProviderImplTest() {
+  public OshiStatisticsProviderTest() {
     SystemInfo systemInfo = mock(SystemInfo.class);
-    oshiStatisticsProvider = new OshiStatisticsProviderImpl(systemInfo);
+    oshiStatisticsProvider = new OshiStatisticsProvider(systemInfo);
 
     final OperatingSystem operatingSystem = mock(OperatingSystem.class);
     when(systemInfo.getOperatingSystem()).thenReturn(operatingSystem);
@@ -89,40 +89,36 @@ public class OshiStatisticsProviderImplTest {
   }
 
   @Test
-  public void initCreatesOsStatistics() throws OshiStatisticsProviderException {
+  public void initCreatesOsStatistics() throws OsStatisticsProviderException {
     final OsStatisticsFactory osStatisticsFactory = mock(OsStatisticsFactory.class);
     final long id = 13;
     oshiStatisticsProvider.init(osStatisticsFactory, id);
 
     verify(osStatisticsFactory).createOsStatistics(eq(ProcessStats.getType()), eq(PROCESS_IDENTITY),
-        eq(id),
-        eq(0));
+        eq(id));
     verify(osStatisticsFactory).createOsStatistics(eq(OperatingSystemStats.getType()),
         eq(SYSTEM_IDENTITY),
-        eq(id), eq(0));
+        eq(id));
     verify(osStatisticsFactory)
-        .createOsStatistics(eq(ProcessorStats.getType()), eq(LOGICAL_PROCESSOR_0_IDENTITY), eq(id),
-            eq(0));
+        .createOsStatistics(eq(ProcessorStats.getType()), eq(LOGICAL_PROCESSOR_0_IDENTITY), eq(id));
     verify(osStatisticsFactory)
-        .createOsStatistics(eq(ProcessorStats.getType()), eq(LOGICAL_PROCESSOR_1_IDENTITY), eq(id),
-            eq(0));
+        .createOsStatistics(eq(ProcessorStats.getType()), eq(LOGICAL_PROCESSOR_1_IDENTITY), eq(id));
     verify(osStatisticsFactory)
         .createOsStatistics(eq(NetworkInterfaceStats.getType()), eq(NETWORK_IF_0_DISPLAY_NAME),
-            eq(id), eq(0));
+            eq(id));
     verify(osStatisticsFactory)
         .createOsStatistics(eq(NetworkInterfaceStats.getType()), eq(NETWORK_IF_1_DISPLAY_NAME),
-            eq(id), eq(0));
+            eq(id));
 
     verifyNoMoreInteractions(osStatisticsFactory);
   }
 
   @Test
-  public void sampleProcessUpdatesStats() throws OshiStatisticsProviderException {
+  public void sampleProcessUpdatesStats() throws OsStatisticsProviderException {
     final OsStatisticsFactory osStatisticsFactory = mock(OsStatisticsFactory.class);
     final Statistics statistics = mock(Statistics.class);
     when(osStatisticsFactory.createOsStatistics(eq(ProcessStats.getType()), eq(PROCESS_IDENTITY),
-        anyLong(),
-        anyInt())).thenReturn(statistics);
+        anyLong())).thenReturn(statistics);
 
     when(osProcess.getProcessCpuLoadBetweenTicks(any())).thenReturn(0.123D);
     when(osProcess.getVirtualSize()).thenReturn(456L);
diff --git a/geode-junit/src/main/java/org/apache/geode/internal/statistics/TestStatisticsManager.java b/geode-junit/src/main/java/org/apache/geode/internal/statistics/TestStatisticsManager.java
index 7449416..cb214fd 100755
--- a/geode-junit/src/main/java/org/apache/geode/internal/statistics/TestStatisticsManager.java
+++ b/geode-junit/src/main/java/org/apache/geode/internal/statistics/TestStatisticsManager.java
@@ -16,7 +16,6 @@ package org.apache.geode.internal.statistics;
 
 import org.apache.geode.Statistics;
 import org.apache.geode.StatisticsType;
-import org.apache.geode.internal.statistics.platform.OsStatisticsFactory;
 
 /**
  * @since GemFire 7.0
@@ -30,7 +29,7 @@ public class TestStatisticsManager extends AbstractStatisticsFactory
 
   @Override
   public Statistics createOsStatistics(final StatisticsType type, final String textId,
-      final long numericId, final int osStatFlags) {
+      final long numericId) {
     return null;
   }
 }