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:23 UTC

[geode] 03/16: Refactor interface with more 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 0f965eb98d6ace4a862d0e94db73bba5cabc3350
Author: Jacob Barrett <jb...@pivotal.io>
AuthorDate: Sat Jun 12 07:39:02 2021 -0700

    Refactor interface with more stats.
---
 .../platform/OshiStatisticsBenchmark.java          |  41 +++--
 .../internal/statistics/GemFireStatSampler.java    |  94 +++++------
 .../geode/internal/statistics/HostStatSampler.java |   6 +-
 .../internal/statistics/OsStatisticsProvider.java  |  62 ++-----
 .../OperatingSystemStats.java}                     |  63 +++++++-
 .../statistics/oshi/OshiStatisticsProvider.java    |  14 ++
 .../oshi/OshiStatisticsProviderException.java      |   4 +
 .../oshi/OshiStatisticsProviderImpl.java           | 179 +++++++++++++++++++++
 .../ProcessStats.java}                             |  59 +++++--
 .../internal/statistics/oshi/ProcessorStats.java   |  98 +++++++++++
 .../statistics/platform/OshiStatistics.java        | 109 -------------
 .../internal/beans/MemberMBeanBridge.java          |   2 +-
 .../statistics/oshi/OshiStatisticsTest.java        | 103 ++++++++++++
 .../statistics/platform/OshiStatisticsTest.java    | 103 ------------
 14 files changed, 582 insertions(+), 355 deletions(-)

diff --git a/geode-core/src/jmh/java/org/apache/geode/internal/statistics/platform/OshiStatisticsBenchmark.java b/geode-core/src/jmh/java/org/apache/geode/internal/statistics/platform/OshiStatisticsBenchmark.java
index d6fbb07..6bfb079 100644
--- a/geode-core/src/jmh/java/org/apache/geode/internal/statistics/platform/OshiStatisticsBenchmark.java
+++ b/geode-core/src/jmh/java/org/apache/geode/internal/statistics/platform/OshiStatisticsBenchmark.java
@@ -22,14 +22,11 @@ import java.util.function.LongSupplier;
 
 import org.openjdk.jmh.annotations.Benchmark;
 import org.openjdk.jmh.annotations.BenchmarkMode;
-import org.openjdk.jmh.annotations.Fork;
-import org.openjdk.jmh.annotations.Measurement;
 import org.openjdk.jmh.annotations.Mode;
 import org.openjdk.jmh.annotations.OutputTimeUnit;
 import org.openjdk.jmh.annotations.Scope;
 import org.openjdk.jmh.annotations.Setup;
 import org.openjdk.jmh.annotations.State;
-import org.openjdk.jmh.annotations.Warmup;
 import oshi.SystemInfo;
 
 import org.apache.geode.StatisticDescriptor;
@@ -44,25 +41,25 @@ public class OshiStatisticsBenchmark {
   private final int pid = new SystemInfo().getOperatingSystem().getProcessId();
   private final SuppliableStatistics noopStatistics = new NoopStatistics();
 
-  @Setup
-  public void setup() {
-    OshiStatistics.init();
-  }
-
-  @Benchmark
-  public void noop() {
-
-  }
-
-  @Benchmark
-  public void refreshProcess() {
-    OshiStatistics.refreshProcess(pid, noopStatistics);
-  }
-
-  @Benchmark
-  public void refreshSystem() {
-    OshiStatistics.refreshSystem(noopStatistics);
-  }
+//  @Setup
+//  public void setup() {
+//    OshiStatistics.init();
+//  }
+//
+//  @Benchmark
+//  public void noop() {
+//
+//  }
+//
+//  @Benchmark
+//  public void refreshProcess() {
+//    OshiStatistics.refreshProcess(pid, noopStatistics);
+//  }
+//
+//  @Benchmark
+//  public void refreshSystem() {
+//    OshiStatistics.refreshSystem(noopStatistics);
+//  }
 
   private static class NoopStatistics implements SuppliableStatistics {
     @Override
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 f7ef22a..a6d8e97 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
@@ -35,6 +35,9 @@ import org.apache.geode.internal.GemFireVersion;
 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;
@@ -66,7 +69,8 @@ public class GemFireStatSampler extends HostStatSampler {
   private int nextListenerId = 1;
   private ProcessStats processStats;
 
-  private OsStatisticsProvider osStatisticsProvider = OsStatisticsProvider.build();
+//  private OsStatisticsProvider osStatisticsProvider = OsStatisticsProvider.build();
+  private OshiStatisticsProvider oshiStatisticsProvider = new OshiStatisticsProviderImpl();
 
   public GemFireStatSampler(InternalDistributedSystem internalDistributedSystem) {
     this(internalDistributedSystem, null);
@@ -268,63 +272,59 @@ public class GemFireStatSampler extends HostStatSampler {
   }
 
   @Override
-  protected void initProcessStats(long id) {
-    if (osStatisticsProvider.osStatsSupported()) {
-      if (osStatsDisabled()) {
-        logger.info(LogMarker.STATISTICS_MARKER,
-            "OS statistic collection disabled by setting the osStatsDisabled system property to true.");
-      } else {
-        int retVal = osStatisticsProvider.initOSStats();
-        if (retVal != 0) {
-          logger.error(LogMarker.STATISTICS_MARKER,
-              "OS statistics failed to initialize properly, some stats may be missing. See bugnote #37160.");
-        }
-        osStatisticsProvider.newSystem(getOsStatisticsFactory(), id);
-        String statName = getStatisticsManager().getName();
-        if (statName == null || statName.length() == 0) {
-          statName = "javaApp" + getSystemId();
-        }
-        Statistics stats =
-            osStatisticsProvider.newProcess(getOsStatisticsFactory(), id, statName + "-proc");
-        processStats = osStatisticsProvider.newProcessStats(stats);
-      }
+  protected void initProcessStats(long pid) {
+    if (osStatsDisabled()) {
+      logger.info(LogMarker.STATISTICS_MARKER,
+          "OS statistic collection disabled by setting the osStatsDisabled system property to true.");
+      return;
     }
+
+    try {
+      oshiStatisticsProvider.init(getOsStatisticsFactory(), pid);
+    } catch (OshiStatisticsProviderException 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
   protected void sampleProcessStats(boolean prepareOnly) {
-    if (prepareOnly || osStatsDisabled() || !osStatisticsProvider.osStatsSupported()) {
-      return;
-    }
-    List<Statistics> statisticsList = getStatisticsManager().getStatsList();
-    if (statisticsList == null) {
-      return;
-    }
-    if (stopRequested()) {
+    if (prepareOnly || osStatsDisabled() || stopRequested()) {
       return;
     }
-    osStatisticsProvider.readyRefreshOSStats();
-    for (Statistics statistics : statisticsList) {
-      if (stopRequested()) {
-        return;
-      }
-      StatisticsImpl statisticsImpl = (StatisticsImpl) statistics;
-      if (statisticsImpl.usesSystemCalls()) {
-        osStatisticsProvider.refresh((LocalStatisticsImpl) statisticsImpl);
-      }
-    }
+    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);
+//      }
+//    }
   }
 
   @Override
   protected void closeProcessStats() {
-    if (osStatisticsProvider.osStatsSupported()) {
-      if (!osStatsDisabled()) {
-        if (processStats != null) {
-          processStats.close();
-        }
-        osStatisticsProvider.closeOSStats();
-      }
-    }
+    oshiStatisticsProvider.destroy();
+//    if (osStatisticsProvider.osStatsSupported()) {
+//      if (!osStatsDisabled()) {
+//        if (processStats != null) {
+//          processStats.close();
+//        }
+//        osStatisticsProvider.closeOSStats();
+//      }
+//    }
   }
 
   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 588034d..6eb6c4f 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
@@ -437,7 +437,11 @@ public abstract class HostStatSampler
     return null;
   }
 
-  protected void initProcessStats(long id) {
+  /**
+   *
+   * @param pid process id if this process
+   */
+  protected void initProcessStats(long pid) {
     // do nothing by default
   }
 
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 f2c0011..d274bf1 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
@@ -14,24 +14,15 @@
  */
 package org.apache.geode.internal.statistics;
 
-import static org.apache.geode.internal.lang.SystemUtils.isLinux;
-import static org.apache.geode.internal.lang.SystemUtils.isMacOS;
-
 import java.net.UnknownHostException;
 
-import org.jetbrains.annotations.NotNull;
-
 import org.apache.geode.Statistics;
-import org.apache.geode.StatisticsType;
 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.OshiProcessStats;
-import org.apache.geode.internal.statistics.platform.OshiStatistics;
-import org.apache.geode.internal.statistics.platform.OshiSystemStats;
 import org.apache.geode.internal.statistics.platform.ProcessStats;
 
 /**
@@ -48,7 +39,7 @@ public class OsStatisticsProvider {
   }
 
   private OsStatisticsProvider() {
-    osStatsSupported = isLinux() || SystemUtils.isMacOS();
+    osStatsSupported = SystemUtils.isLinux();
   }
 
   public static OsStatisticsProvider build() {
@@ -56,12 +47,7 @@ public class OsStatisticsProvider {
   }
 
   int initOSStats() {
-    if (isLinux()) {
-      return LinuxProcFsStatistics.init();
-    } else if (isMacOS()) {
-      return OshiStatistics.init();
-    }
-    return 1;
+    return LinuxProcFsStatistics.init();
   }
 
   void closeOSStats() {
@@ -69,11 +55,7 @@ public class OsStatisticsProvider {
   }
 
   void readyRefreshOSStats() {
-    if (isLinux()) {
-      LinuxProcFsStatistics.readyRefresh();
-    } else if (isMacOS()) {
-      OshiStatistics.readyRefresh();
-    }
+    LinuxProcFsStatistics.readyRefresh();
   }
 
   /**
@@ -82,11 +64,7 @@ public class OsStatisticsProvider {
    */
   private void refreshProcess(LocalStatisticsImpl statistics) {
     int pid = (int) statistics.getNumericId();
-    if (isLinux()) {
-      LinuxProcFsStatistics.refreshProcess(pid, statistics);
-    } else if (isMacOS()) {
-      OshiStatistics.refreshProcess(pid, statistics);
-    }
+    LinuxProcFsStatistics.refreshProcess(pid, statistics);
   }
 
   /**
@@ -94,11 +72,7 @@ public class OsStatisticsProvider {
    * machine and storing them in the instance.
    */
   private void refreshSystem(LocalStatisticsImpl statistics) {
-    if (isLinux()) {
-      LinuxProcFsStatistics.refreshSystem(statistics);
-    } else if (isMacOS()) {
-      OshiStatistics.refreshSystem(statistics);
-    }
+    LinuxProcFsStatistics.refreshSystem(statistics);
   }
 
   /**
@@ -121,19 +95,12 @@ public class OsStatisticsProvider {
    */
   Statistics newProcess(OsStatisticsFactory osStatisticsFactory, long pid, String name) {
     Statistics statistics;
-    statistics = osStatisticsFactory.createOsStatistics(getProcessStatType(), name, pid,
+    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 static StatisticsType getProcessStatType() {
-    if (isLinux()) {
-      return LinuxProcessStats.getType();
-    }
-    return OshiProcessStats.getType();
-  }
-
   /**
    * Creates a new <code>ProcessStats</code> instance that wraps the given <code>Statistics</code>.
    *
@@ -144,31 +111,22 @@ public class OsStatisticsProvider {
     if (statistics instanceof LocalStatisticsImpl) {
       refresh((LocalStatisticsImpl) statistics);
     } // otherwise its a Dummy implementation so do nothing
-    if (isLinux()) {
-      return LinuxProcessStats.createProcessStats(statistics);
-    }
-    return OshiProcessStats.createProcessStats(statistics);
+    return LinuxProcessStats.createProcessStats(statistics);
   }
 
   /**
    * 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(final @NotNull OsStatisticsFactory osStatisticsFactory, long id) {
-    final Statistics statistics = osStatisticsFactory.createOsStatistics(getSystemStatType(),
+  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
   }
 
-  public static StatisticsType getSystemStatType() {
-    if (isLinux()) {
-      return LinuxSystemStats.getType();
-    }
-    return OshiSystemStats.getType();
-  }
-
   /**
    * @return this machine's fully qualified hostname or "unknownHostName" if one cannot be found.
    */
diff --git a/geode-core/src/main/java/org/apache/geode/internal/statistics/platform/OshiSystemStats.java b/geode-core/src/main/java/org/apache/geode/internal/statistics/oshi/OperatingSystemStats.java
similarity index 68%
rename from geode-core/src/main/java/org/apache/geode/internal/statistics/platform/OshiSystemStats.java
rename to geode-core/src/main/java/org/apache/geode/internal/statistics/oshi/OperatingSystemStats.java
index 7693cd0..49f819d 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/statistics/platform/OshiSystemStats.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/statistics/oshi/OperatingSystemStats.java
@@ -13,7 +13,7 @@
  * the License.
  */
 
-package org.apache.geode.internal.statistics.platform;
+package org.apache.geode.internal.statistics.oshi;
 
 import org.jetbrains.annotations.NotNull;
 
@@ -23,7 +23,7 @@ import org.apache.geode.StatisticsTypeFactory;
 import org.apache.geode.annotations.Immutable;
 import org.apache.geode.internal.statistics.StatisticsTypeFactoryImpl;
 
-public class OshiSystemStats {
+public class OperatingSystemStats {
   static final int processCount;
   static final int threadCount;
   static final int contextSwitches;
@@ -42,6 +42,16 @@ public class OshiSystemStats {
   static final int systemCpuLoadTicksIRQ;
   static final int systemCpuLoadTicksSOFTIRQ;
   static final int systemCpuLoadTicksSTEAL;
+  static final int systemCpuLoad;
+  static final int memoryTotal;
+  static final int memoryAvailable;
+  static final int memoryPageSize;
+  static final int swapTotal;
+  static final int swapUsed;
+  static final int virtualMax;
+  static final int virtualInUse;
+  static final int swapPagesIn;
+  static final int swapPagesOut;
 
   @Immutable
   private static final StatisticsType statisticsType;
@@ -95,15 +105,45 @@ public class OshiSystemStats {
             f.createLongCounter("systemCpuLoadTicksSTEAL",
                 "Time spent in Steal",
                 "milliseconds"),
+            f.createDoubleGauge("systemCpuLoad",
+                "Time spent in Steal",
+                "percent"),
             f.createDoubleGauge("systemLoadAverage1",
                 "The system load average is the sum of the number of runnable entities queued to the available processors and the number of runnable entities running on the available processors averaged over 1 minute.",
-                "processors"),
+                "load"),
             f.createDoubleGauge("systemLoadAverage5",
                 "The system load average is the sum of the number of runnable entities queued to the available processors and the number of runnable entities running on the available processors averaged over 5 minutes.",
-                "processors"),
+                "load"),
             f.createDoubleGauge("systemLoadAverage15",
                 "The system load average is the sum of the number of runnable entities queued to the available processors and the number of runnable entities running on the available processors averaged over 15 minutes.",
-                "processors"),
+                "load"),
+            f.createLongGauge("memoryTotal",
+                "The amount of actual physical memory, in bytes.",
+                "bytes"),
+            f.createLongGauge("memoryAvailable",
+                "The amount of physical memory currently available, in bytes.",
+                "bytes"),
+            f.createLongGauge("memoryPageSize",
+                "The number of bytes in a memory page",
+                "bytes"),
+            f.createLongGauge("swapTotal",
+                "The current size of the paging/swap file(s), in bytes. If the paging/swap file can be extended, this is a soft limit.",
+                "bytes"),
+            f.createLongGauge("swapUsed",
+                "The current memory committed to the paging/swap file(s), in bytes.",
+                "bytes"),
+            f.createLongGauge("virtualMax",
+                "The maximum memory that can be committed by the system without extending the paging file(s), in bytes. Also called the Commit Limit. If the paging/swap file can be extended, this is a soft limit. This is generally equal to the sum of the sizes of physical memory and paging/swap file(s).",
+                "bytes"),
+            f.createLongGauge("virtualInUse",
+                "The memory currently committed by the system, in bytes. Also called the Commit Total. This is generally equal to the sum of the bytes used of physical memory and paging/swap file(s).",
+                "bytes"),
+            f.createLongCounter("swapPagesIn",
+                "Number of pages read from paging/swap file(s) to resolve hard page faults.",
+                "pages"),
+            f.createLongCounter("swapPagesOut",
+                "Number of pages read from paging/swap file(s) to resolve hard page faults.",
+                "pages"),
     });
 
     processCount = statisticsType.nameToId("processCount");
@@ -124,9 +164,20 @@ public class OshiSystemStats {
     systemCpuLoadTicksIRQ = statisticsType.nameToId("systemCpuLoadTicksIRQ");
     systemCpuLoadTicksSOFTIRQ = statisticsType.nameToId("systemCpuLoadTicksSOFTIRQ");
     systemCpuLoadTicksSTEAL = statisticsType.nameToId("systemCpuLoadTicksSTEAL");
+    systemCpuLoad = statisticsType.nameToId("systemCpuLoad");
+    memoryTotal = statisticsType.nameToId("memoryTotal");
+    memoryAvailable = statisticsType.nameToId("memoryAvailable");
+    memoryPageSize = statisticsType.nameToId("memoryPageSize");
+    swapTotal = statisticsType.nameToId("swapTotal");
+    swapUsed = statisticsType.nameToId("swapUsed");
+    virtualMax = statisticsType.nameToId("virtualMax");
+    virtualInUse = statisticsType.nameToId("virtualInUse");
+    swapPagesIn = statisticsType.nameToId("swapPagesIn");
+    swapPagesOut = statisticsType.nameToId("swapPagesOut");
+
   }
 
-  private OshiSystemStats() {
+  private OperatingSystemStats() {
     // no instances allowed
   }
 
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
new file mode 100644
index 0000000..d60be5c
--- /dev/null
+++ b/geode-core/src/main/java/org/apache/geode/internal/statistics/oshi/OshiStatisticsProvider.java
@@ -0,0 +1,14 @@
+package org.apache.geode.internal.statistics.oshi;
+
+import org.jetbrains.annotations.NotNull;
+
+import org.apache.geode.internal.statistics.platform.OsStatisticsFactory;
+
+public interface OshiStatisticsProvider {
+  void init(final @NotNull OsStatisticsFactory osStatisticsFactory,
+            final long pid) throws OshiStatisticsProviderException;
+
+  void sample();
+
+  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
new file mode 100644
index 0000000..146a716
--- /dev/null
+++ b/geode-core/src/main/java/org/apache/geode/internal/statistics/oshi/OshiStatisticsProviderException.java
@@ -0,0 +1,4 @@
+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
new file mode 100644
index 0000000..dcaf5b4
--- /dev/null
+++ b/geode-core/src/main/java/org/apache/geode/internal/statistics/oshi/OshiStatisticsProviderImpl.java
@@ -0,0 +1,179 @@
+package org.apache.geode.internal.statistics.oshi;
+
+import java.util.List;
+
+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.VirtualMemory;
+import oshi.software.os.OSProcess;
+import oshi.software.os.OperatingSystem;
+
+import org.apache.geode.Statistics;
+import org.apache.geode.internal.statistics.platform.OsStatisticsFactory;
+
+public class OshiStatisticsProviderImpl implements OshiStatisticsProvider {
+
+  final SystemInfo systemInfo = new SystemInfo();
+  
+  private int processId;
+  private CentralProcessor processor;
+  private OperatingSystem operatingSystem;
+  private HardwareAbstractionLayer hardware;
+  private long[] systemCpuLoadTicks;
+  private long[][] processorCpuLoadTicks;
+
+  private Statistics processStats;
+  private Statistics systemStats;
+  private Statistics[] processorStats;
+  private OSProcess process;
+
+  @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, logicalProcessorsSize = logicalProcessors.size(); i < logicalProcessorsSize; i++) {
+      final LogicalProcessor logicalProcessor = logicalProcessors.get(i);
+      final String processorIdentity = logicalProcessor.toString();
+      processorStats[i] = osStatisticsFactory.createOsStatistics(ProcessorStats.getType(),
+          processorIdentity, id, 0);
+    }
+  }
+
+  @Override
+  public void sample() {
+    sampleProcess();
+    sampleSystem();
+    sampleProcessors();
+  }
+
+  @Override
+  public void destroy() {
+  }
+
+  public 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());
+  }
+
+  public 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());
+  }
+
+  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()]);
+    }
+  }
+  
+}
diff --git a/geode-core/src/main/java/org/apache/geode/internal/statistics/platform/OshiProcessStats.java b/geode-core/src/main/java/org/apache/geode/internal/statistics/oshi/ProcessStats.java
similarity index 55%
rename from geode-core/src/main/java/org/apache/geode/internal/statistics/platform/OshiProcessStats.java
rename to geode-core/src/main/java/org/apache/geode/internal/statistics/oshi/ProcessStats.java
index 23e6541..1e6199e 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/statistics/platform/OshiProcessStats.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/statistics/oshi/ProcessStats.java
@@ -13,23 +13,30 @@
  * the License.
  */
 
-package org.apache.geode.internal.statistics.platform;
+package org.apache.geode.internal.statistics.oshi;
 
 import org.jetbrains.annotations.NotNull;
 
 import org.apache.geode.StatisticDescriptor;
-import org.apache.geode.Statistics;
 import org.apache.geode.StatisticsType;
 import org.apache.geode.StatisticsTypeFactory;
 import org.apache.geode.annotations.Immutable;
 import org.apache.geode.internal.statistics.StatisticsTypeFactoryImpl;
 
-public class OshiProcessStats {
+public class ProcessStats {
   static final int virtualSize;
   static final int residentSetSize;
   static final int threadCount;
   static final int kernelTime;
   static final int userTime;
+  static final int bytesRead;
+  static final int bytesWritten;
+  static final int openFiles;
+  static final int cpuLoadCumulative;
+  static final int cpuLoad;
+  static final int minorFaults;
+  static final int majorFaults;
+  static final int contextSwitches;
 
   @Immutable
   private static final StatisticsType statisticsType;
@@ -37,7 +44,7 @@ public class OshiProcessStats {
   static {
     final StatisticsTypeFactory f = StatisticsTypeFactoryImpl.singleton();
 
-    statisticsType = f.createType("OSProcessStats", "Statistics on a OS process.",
+    statisticsType = f.createType("ProcessStats", "Statistics on a process.",
         new StatisticDescriptor[]{
             f.createLongGauge("virtualSize",
                 "Gets the Virtual Memory Size (VSZ). Includes all memory that the process can access, including memory that is swapped out and memory that is from shared libraries.",
@@ -53,7 +60,31 @@ public class OshiProcessStats {
                 "milliseconds"),
             f.createLongCounter("userTime",
                 "Gets user time used by the process.",
-                "milliseconds")
+                "milliseconds"),
+            f.createLongCounter("bytesRead",
+                "The number of bytes the process has written to disk",
+                "bytes"),
+            f.createLongCounter("bytesWritten",
+                "The number of bytes the process has written to disk.",
+                "bytes"),
+            f.createLongGauge("openFiles",
+                "Gets the number of open file handles (or network connections) that belongs to the process.",
+                "files"),
+            f.createDoubleGauge("cpuLoadCumulative",
+                "Gets cumulative CPU usage of this process.",
+                "percent"),
+            f.createDoubleGauge("cpuLoad",
+                "CPU usage of this process.",
+                "percent"),
+            f.createLongCounter("minorFaults",
+                "Gets the number of minor (soft) faults the process has made which have not required loading a memory page from disk. Sometimes called reclaims.",
+                "faults"),
+            f.createLongCounter("majorFaults",
+                "Gets the number of major (hard) faults the process has made which have required loading a memory page from disk.",
+                "faults"),
+            f.createLongCounter("contextSwitches",
+                "A snapshot of the context switches the process has done. Since the context switches could be voluntary and non-voluntary, this gives the sum of both.",
+                "switches"),
         });
 
     virtualSize = statisticsType.nameToId("virtualSize");
@@ -61,9 +92,17 @@ public class OshiProcessStats {
     threadCount = statisticsType.nameToId("threadCount");
     kernelTime = statisticsType.nameToId("kernelTime");
     userTime = statisticsType.nameToId("userTime");
+    bytesRead = statisticsType.nameToId("bytesRead");
+    bytesWritten = statisticsType.nameToId("bytesWritten");
+    openFiles = statisticsType.nameToId("openFiles");
+    cpuLoadCumulative = statisticsType.nameToId("cpuLoadCumulative");
+    cpuLoad = statisticsType.nameToId("cpuLoad");
+    minorFaults = statisticsType.nameToId("minorFaults");
+    majorFaults = statisticsType.nameToId("majorFaults");
+    contextSwitches = statisticsType.nameToId("contextSwitches");
   }
 
-  private OshiProcessStats() {
+  private ProcessStats() {
     // no instances allowed
   }
 
@@ -71,12 +110,4 @@ public class OshiProcessStats {
     return statisticsType;
   }
 
-  public static ProcessStats createProcessStats(final @NotNull Statistics stats) {
-    return new ProcessStats(stats) {
-      @Override
-      public long getProcessSize() {
-        return stats.getLong(residentSetSize);
-      }
-    };
-  }
 }
diff --git a/geode-core/src/main/java/org/apache/geode/internal/statistics/oshi/ProcessorStats.java b/geode-core/src/main/java/org/apache/geode/internal/statistics/oshi/ProcessorStats.java
new file mode 100644
index 0000000..1303901
--- /dev/null
+++ b/geode-core/src/main/java/org/apache/geode/internal/statistics/oshi/ProcessorStats.java
@@ -0,0 +1,98 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
+ * agreements. See the NOTICE file distributed with this work for additional information regarding
+ * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License. You may obtain a
+ * copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License
+ * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
+ * or implied. See the License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.geode.internal.statistics.oshi;
+
+import org.jetbrains.annotations.NotNull;
+
+import org.apache.geode.StatisticDescriptor;
+import org.apache.geode.StatisticsType;
+import org.apache.geode.StatisticsTypeFactory;
+import org.apache.geode.annotations.Immutable;
+import org.apache.geode.internal.statistics.StatisticsTypeFactoryImpl;
+
+public class ProcessorStats {
+  static final int currentFreq;
+  static final int processorCpuLoadTicksUSER;
+  static final int processorCpuLoadTicksNICE;
+  static final int processorCpuLoadTicksSYSTEM;
+  static final int processorCpuLoadTicksIDLE;
+  static final int processorCpuLoadTicksIOWAIT;
+  static final int processorCpuLoadTicksIRQ;
+  static final int processorCpuLoadTicksSOFTIRQ;
+  static final int processorCpuLoadTicksSTEAL;
+  static final int processorCpuLoad;
+
+  @Immutable
+  private static final StatisticsType statisticsType;
+
+  static {
+    final StatisticsTypeFactory f = StatisticsTypeFactoryImpl.singleton();
+
+    statisticsType = f.createType("SystemProcessorStats", "Statistics for a system processor.",
+        new StatisticDescriptor[] {
+            f.createLongGauge("currentFreq",
+                "Maximum frequency (in Hz), of the logical processors on this CPU.",
+                "Hz"),
+            f.createLongCounter("processorCpuLoadTicksUSER",
+                "Time spent in User",
+                "milliseconds"),
+            f.createLongCounter("processorCpuLoadTicksNICE",
+                "Time spent in Nice",
+                "milliseconds"),
+            f.createLongCounter("processorCpuLoadTicksSYSTEM",
+                "Time spent in System",
+                "milliseconds"),
+            f.createLongCounter("processorCpuLoadTicksIDLE",
+                "Time spent in Idle",
+                "milliseconds"),
+            f.createLongCounter("processorCpuLoadTicksIOWAIT",
+                "Time spent in IOWait",
+                "milliseconds"),
+            f.createLongCounter("processorCpuLoadTicksIRQ",
+                "Time spent in IRQ",
+                "milliseconds"),
+            f.createLongCounter("processorCpuLoadTicksSOFTIRQ",
+                "Time spent in SoftIRQ",
+                "milliseconds"),
+            f.createLongCounter("processorCpuLoadTicksSTEAL",
+                "Time spent in Steal",
+                "milliseconds"),
+            f.createDoubleGauge("processorCpuLoad",
+                "CPU usage",
+                "percent"),
+    });
+
+    currentFreq = statisticsType.nameToId("currentFreq");
+    processorCpuLoadTicksUSER = statisticsType.nameToId("processorCpuLoadTicksUSER");
+    processorCpuLoadTicksNICE = statisticsType.nameToId("processorCpuLoadTicksNICE");
+    processorCpuLoadTicksSYSTEM = statisticsType.nameToId("processorCpuLoadTicksSYSTEM");
+    processorCpuLoadTicksIDLE = statisticsType.nameToId("processorCpuLoadTicksIDLE");
+    processorCpuLoadTicksIOWAIT = statisticsType.nameToId("processorCpuLoadTicksIOWAIT");
+    processorCpuLoadTicksIRQ = statisticsType.nameToId("processorCpuLoadTicksIRQ");
+    processorCpuLoadTicksSOFTIRQ = statisticsType.nameToId("processorCpuLoadTicksSOFTIRQ");
+    processorCpuLoadTicksSTEAL = statisticsType.nameToId("processorCpuLoadTicksSTEAL");
+    processorCpuLoad = statisticsType.nameToId("processorCpuLoad");
+
+  }
+
+  private ProcessorStats() {
+    // no instances allowed
+  }
+
+  public static @NotNull StatisticsType getType() {
+    return statisticsType;
+  }
+}
diff --git a/geode-core/src/main/java/org/apache/geode/internal/statistics/platform/OshiStatistics.java b/geode-core/src/main/java/org/apache/geode/internal/statistics/platform/OshiStatistics.java
deleted file mode 100644
index 2da3f55..0000000
--- a/geode-core/src/main/java/org/apache/geode/internal/statistics/platform/OshiStatistics.java
+++ /dev/null
@@ -1,109 +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.platform;
-
-import org.jetbrains.annotations.NotNull;
-import oshi.SystemInfo;
-import oshi.hardware.CentralProcessor;
-import oshi.hardware.CentralProcessor.TickType;
-import oshi.hardware.HardwareAbstractionLayer;
-import oshi.software.os.OSProcess;
-import oshi.software.os.OperatingSystem;
-
-import org.apache.geode.internal.statistics.LocalStatisticsImpl;
-import org.apache.geode.internal.statistics.SuppliableStatistics;
-
-public class OshiStatistics {
-
-  private static final OshiStatistics instance;
-
-  static {
-    final SystemInfo systemInfo = new SystemInfo();
-    instance = new OshiStatistics(systemInfo.getOperatingSystem(), systemInfo.getHardware());
-  }
-
-  private final OperatingSystem operatingSystem;
-  private final HardwareAbstractionLayer hardware;
-
-  OshiStatistics(final @NotNull OperatingSystem operatingSystem,
-                 final @NotNull HardwareAbstractionLayer hardware) {
-    this.operatingSystem = operatingSystem;
-    this.hardware = hardware;
-  }
-
-  public static int init() {
-    return 0;
-  }
-
-  public static void readyRefresh() {
-
-  }
-
-  public static void refreshProcess(final int pid, final SuppliableStatistics stats) {
-    instance.updateProcessStats(pid, stats);
-  }
-
-  public static void refreshSystem(final SuppliableStatistics stats) {
-    instance.updateSystemStats(stats);
-  }
-
-  public void updateProcessStats(final int pid, final SuppliableStatistics stats) {
-    final OSProcess process = operatingSystem.getProcess(pid);
-    stats.setLong(OshiProcessStats.virtualSize, process.getVirtualSize());
-    stats.setLong(OshiProcessStats.residentSetSize, process.getResidentSetSize());
-    stats.setLong(OshiProcessStats.threadCount, process.getThreadCount());
-    stats.setLong(OshiProcessStats.kernelTime, process.getKernelTime());
-    stats.setLong(OshiProcessStats.userTime, process.getUserTime());
-  }
-
-  public void updateSystemStats(final SuppliableStatistics stats) {
-    stats.setLong(OshiSystemStats.processCount, operatingSystem.getProcessCount());
-    stats.setLong(OshiSystemStats.threadCount, operatingSystem.getThreadCount());
-
-    final CentralProcessor processor = hardware.getProcessor();
-    stats.setLong(OshiSystemStats.contextSwitches, processor.getContextSwitches());
-    stats.setLong(OshiSystemStats.interrupts, processor.getInterrupts());
-    stats.setLong(OshiSystemStats.physicalProcessorCount, processor.getPhysicalProcessorCount());
-    stats.setLong(OshiSystemStats.logicalProcessorCount, processor.getLogicalProcessorCount());
-    stats.setLong(OshiSystemStats.maxFreq, processor.getMaxFreq());
-
-    final double[] systemLoadAverage = processor.getSystemLoadAverage(3);
-    stats.setDouble(OshiSystemStats.systemLoadAverage1, systemLoadAverage[0]);
-    stats.setDouble(OshiSystemStats.systemLoadAverage5, systemLoadAverage[1]);
-    stats.setDouble(OshiSystemStats.systemLoadAverage15, systemLoadAverage[2]);
-
-    long[] systemCpuLoadTicks = processor.getSystemCpuLoadTicks();
-    stats.setLong(OshiSystemStats.systemCpuLoadTicksUSER,
-        systemCpuLoadTicks[TickType.USER.getIndex()]);
-    stats.setLong(OshiSystemStats.systemCpuLoadTicksNICE,
-        systemCpuLoadTicks[TickType.NICE.getIndex()]);
-    stats.setLong(OshiSystemStats.systemCpuLoadTicksSYSTEM,
-        systemCpuLoadTicks[TickType.SYSTEM.getIndex()]);
-    stats.setLong(OshiSystemStats.systemCpuLoadTicksIDLE,
-        systemCpuLoadTicks[TickType.IDLE.getIndex()]);
-    stats.setLong(OshiSystemStats.systemCpuLoadTicksIOWAIT,
-        systemCpuLoadTicks[TickType.IOWAIT.getIndex()]);
-    stats.setLong(OshiSystemStats.systemCpuLoadTicksIRQ,
-        systemCpuLoadTicks[TickType.IRQ.getIndex()]);
-    stats.setLong(OshiSystemStats.systemCpuLoadTicksSOFTIRQ,
-        systemCpuLoadTicks[TickType.SOFTIRQ.getIndex()]);
-    stats.setLong(OshiSystemStats.systemCpuLoadTicksSTEAL,
-        systemCpuLoadTicks[TickType.STEAL.getIndex()]);
-
-    final long[] currentFreq = processor.getCurrentFreq();
-  }
-
-}
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 7888901..993c617 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
@@ -298,7 +298,7 @@ public class MemberMBeanBridge {
 
   private Statistics fetchSystemStats() {
     if (osStatisticsProvider.osStatsSupported()) {
-      Statistics[] systemStats = system.findStatisticsByType(OsStatisticsProvider.getSystemStatType());
+      Statistics[] systemStats = system.findStatisticsByType(LinuxSystemStats.getType());
 
       if (systemStats != null) {
         return systemStats[0];
diff --git a/geode-core/src/test/java/org/apache/geode/internal/statistics/oshi/OshiStatisticsTest.java b/geode-core/src/test/java/org/apache/geode/internal/statistics/oshi/OshiStatisticsTest.java
new file mode 100644
index 0000000..2837d1c
--- /dev/null
+++ b/geode-core/src/test/java/org/apache/geode/internal/statistics/oshi/OshiStatisticsTest.java
@@ -0,0 +1,103 @@
+/*
+ * 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.oshi;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.mockito.ArgumentMatchers.eq;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+import org.junit.Test;
+import oshi.hardware.CentralProcessor;
+import oshi.hardware.HardwareAbstractionLayer;
+import oshi.software.os.OSProcess;
+import oshi.software.os.OperatingSystem;
+
+import org.apache.geode.internal.statistics.SuppliableStatistics;
+
+public class OshiStatisticsTest {
+
+//  private final OSProcess process = mock(OSProcess.class);
+//  private final OperatingSystem operatingSystem = mock(OperatingSystem.class);
+//  private final HardwareAbstractionLayer hardwareAbstractionLayer = mock(HardwareAbstractionLayer.class);
+//  private final OshiStatistics
+//      oshiStatistics = new OshiStatistics(operatingSystem, hardwareAbstractionLayer);
+//
+//
+//  private final SuppliableStatistics stats = mock(SuppliableStatistics.class);
+//
+//  public OshiStatisticsTest() {
+//    when(operatingSystem.getProcess(eq(1))).thenReturn(process);
+//  }
+//
+//  @Test
+//  public void testInit() {
+//    assertThat(OshiStatistics.init()).isEqualTo(0);
+//  }
+//
+//  @Test
+//  public void updateProcessStats() {
+//    when(process.getVirtualSize()).thenReturn(42L);
+//    when(process.getResidentSetSize()).thenReturn(420L);
+//    when(process.getThreadCount()).thenReturn(4200);
+//    when(process.getKernelTime()).thenReturn(42000L);
+//    when(process.getUserTime()).thenReturn(420000L);
+//
+//    oshiStatistics.updateProcessStats(1, stats);
+//
+//    verify(stats).setLong(eq(ProcessStats.virtualSize), eq(42L));
+//    verify(stats).setLong(eq(ProcessStats.residentSetSize), eq(420L));
+//    verify(stats).setLong(eq(ProcessStats.threadCount), eq(4200L));
+//    verify(stats).setLong(eq(ProcessStats.kernelTime), eq(42000L));
+//    verify(stats).setLong(eq(ProcessStats.userTime), eq(420000L));
+//  }
+//
+//  @Test
+//  public void updateSystemStats() {
+//    when(operatingSystem.getProcessCount()).thenReturn(1);
+//    when(operatingSystem.getThreadCount()).thenReturn(2);
+//    final CentralProcessor centralProcessor = mock(CentralProcessor.class);
+//    when(centralProcessor.getContextSwitches()).thenReturn(3L);
+//    when(centralProcessor.getInterrupts()).thenReturn(4L);
+//    when(centralProcessor.getPhysicalProcessorCount()).thenReturn(5);
+//    when(centralProcessor.getLogicalProcessorCount()).thenReturn(6);
+//    when(centralProcessor.getSystemLoadAverage(eq(3))).thenReturn(new double[]{1.0, 2.0, 3.0});
+//    when(centralProcessor.getSystemCpuLoadTicks()).thenReturn(new long[]{1, 2, 3, 4, 5, 6, 7, 8});
+//    when(hardwareAbstractionLayer.getProcessor()).thenReturn(centralProcessor);
+//
+//    oshiStatistics.updateSystemStats(stats);
+//
+//    verify(stats).setLong(eq(OperatingSystemStats.processCount), eq(1L));
+//    verify(stats).setLong(eq(OperatingSystemStats.threadCount), eq(2L));
+//    verify(stats).setLong(eq(OperatingSystemStats.contextSwitches), eq(3L));
+//    verify(stats).setLong(eq(OperatingSystemStats.interrupts), eq(4L));
+//    verify(stats).setLong(eq(OperatingSystemStats.physicalProcessorCount), eq(5L));
+//    verify(stats).setLong(eq(OperatingSystemStats.logicalProcessorCount), eq(6L));
+//    verify(stats).setDouble(eq(OperatingSystemStats.systemLoadAverage1), eq(1.0));
+//    verify(stats).setDouble(eq(OperatingSystemStats.systemLoadAverage5), eq(2.0));
+//    verify(stats).setDouble(eq(OperatingSystemStats.systemLoadAverage15), eq(3.0));
+//    verify(stats).setLong(eq(OperatingSystemStats.systemCpuLoadTicksUSER), eq(1L));
+//    verify(stats).setLong(eq(OperatingSystemStats.systemCpuLoadTicksNICE), eq(2L));
+//    verify(stats).setLong(eq(OperatingSystemStats.systemCpuLoadTicksSYSTEM), eq(3L));
+//    verify(stats).setLong(eq(OperatingSystemStats.systemCpuLoadTicksIDLE), eq(4L));
+//    verify(stats).setLong(eq(OperatingSystemStats.systemCpuLoadTicksIOWAIT), eq(5L));
+//    verify(stats).setLong(eq(OperatingSystemStats.systemCpuLoadTicksIRQ), eq(6L));
+//    verify(stats).setLong(eq(OperatingSystemStats.systemCpuLoadTicksSOFTIRQ), eq(7L));
+//    verify(stats).setLong(eq(OperatingSystemStats.systemCpuLoadTicksSTEAL), eq(8L));
+//  }
+
+}
\ No newline at end of file
diff --git a/geode-core/src/test/java/org/apache/geode/internal/statistics/platform/OshiStatisticsTest.java b/geode-core/src/test/java/org/apache/geode/internal/statistics/platform/OshiStatisticsTest.java
deleted file mode 100644
index a8220d8..0000000
--- a/geode-core/src/test/java/org/apache/geode/internal/statistics/platform/OshiStatisticsTest.java
+++ /dev/null
@@ -1,103 +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.platform;
-
-
-import static org.assertj.core.api.Assertions.assertThat;
-import static org.mockito.ArgumentMatchers.eq;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.verify;
-import static org.mockito.Mockito.when;
-
-import org.junit.Test;
-import oshi.hardware.CentralProcessor;
-import oshi.hardware.HardwareAbstractionLayer;
-import oshi.software.os.OSProcess;
-import oshi.software.os.OperatingSystem;
-
-import org.apache.geode.internal.statistics.SuppliableStatistics;
-
-public class OshiStatisticsTest {
-
-  private final OSProcess process = mock(OSProcess.class);
-  private final OperatingSystem operatingSystem = mock(OperatingSystem.class);
-  private final HardwareAbstractionLayer hardwareAbstractionLayer = mock(HardwareAbstractionLayer.class);
-  private final OshiStatistics oshiStatistics = new OshiStatistics(operatingSystem, hardwareAbstractionLayer);
-
-
-  private final SuppliableStatistics stats = mock(SuppliableStatistics.class);
-
-  public OshiStatisticsTest() {
-    when(operatingSystem.getProcess(eq(1))).thenReturn(process);
-  }
-
-  @Test
-  public void testInit() {
-    assertThat(OshiStatistics.init()).isEqualTo(0);
-  }
-
-  @Test
-  public void updateProcessStats() {
-    when(process.getVirtualSize()).thenReturn(42L);
-    when(process.getResidentSetSize()).thenReturn(420L);
-    when(process.getThreadCount()).thenReturn(4200);
-    when(process.getKernelTime()).thenReturn(42000L);
-    when(process.getUserTime()).thenReturn(420000L);
-
-    oshiStatistics.updateProcessStats(1, stats);
-
-    verify(stats).setLong(eq(OshiProcessStats.virtualSize), eq(42L));
-    verify(stats).setLong(eq(OshiProcessStats.residentSetSize), eq(420L));
-    verify(stats).setLong(eq(OshiProcessStats.threadCount), eq(4200L));
-    verify(stats).setLong(eq(OshiProcessStats.kernelTime), eq(42000L));
-    verify(stats).setLong(eq(OshiProcessStats.userTime), eq(420000L));
-  }
-
-  @Test
-  public void updateSystemStats() {
-    when(operatingSystem.getProcessCount()).thenReturn(1);
-    when(operatingSystem.getThreadCount()).thenReturn(2);
-    final CentralProcessor centralProcessor = mock(CentralProcessor.class);
-    when(centralProcessor.getContextSwitches()).thenReturn(3L);
-    when(centralProcessor.getInterrupts()).thenReturn(4L);
-    when(centralProcessor.getPhysicalProcessorCount()).thenReturn(5);
-    when(centralProcessor.getLogicalProcessorCount()).thenReturn(6);
-    when(centralProcessor.getSystemLoadAverage(eq(3))).thenReturn(new double[]{1.0, 2.0, 3.0});
-    when(centralProcessor.getSystemCpuLoadTicks()).thenReturn(new long[]{1, 2, 3, 4, 5, 6, 7, 8});
-    when(hardwareAbstractionLayer.getProcessor()).thenReturn(centralProcessor);
-
-    oshiStatistics.updateSystemStats(stats);
-
-    verify(stats).setLong(eq(OshiSystemStats.processCount), eq(1L));
-    verify(stats).setLong(eq(OshiSystemStats.threadCount), eq(2L));
-    verify(stats).setLong(eq(OshiSystemStats.contextSwitches), eq(3L));
-    verify(stats).setLong(eq(OshiSystemStats.interrupts), eq(4L));
-    verify(stats).setLong(eq(OshiSystemStats.physicalProcessorCount), eq(5L));
-    verify(stats).setLong(eq(OshiSystemStats.logicalProcessorCount), eq(6L));
-    verify(stats).setDouble(eq(OshiSystemStats.systemLoadAverage1), eq(1.0));
-    verify(stats).setDouble(eq(OshiSystemStats.systemLoadAverage5), eq(2.0));
-    verify(stats).setDouble(eq(OshiSystemStats.systemLoadAverage15), eq(3.0));
-    verify(stats).setLong(eq(OshiSystemStats.systemCpuLoadTicksUSER), eq(1L));
-    verify(stats).setLong(eq(OshiSystemStats.systemCpuLoadTicksNICE), eq(2L));
-    verify(stats).setLong(eq(OshiSystemStats.systemCpuLoadTicksSYSTEM), eq(3L));
-    verify(stats).setLong(eq(OshiSystemStats.systemCpuLoadTicksIDLE), eq(4L));
-    verify(stats).setLong(eq(OshiSystemStats.systemCpuLoadTicksIOWAIT), eq(5L));
-    verify(stats).setLong(eq(OshiSystemStats.systemCpuLoadTicksIRQ), eq(6L));
-    verify(stats).setLong(eq(OshiSystemStats.systemCpuLoadTicksSOFTIRQ), eq(7L));
-    verify(stats).setLong(eq(OshiSystemStats.systemCpuLoadTicksSTEAL), eq(8L));
-  }
-
-}
\ No newline at end of file