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

[geode] branch wip/oshi-multios-stats-module created (now fb2b8cf)

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

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


      at fb2b8cf  Initial module.

This branch includes the following new commits:

     new 0a485c4  MacOSX to macOS
     new 2a8bdb4  Basic OSHI based stats.
     new 0f965eb  Refactor interface with more stats.
     new c46ec0d  adds network intraces.
     new 599dbb5  adds tcp/udp stats adds file handle stats.
     new a8dca19  spotless
     new 25b39ea  fixes max files
     new 2928716  Update to OSHI 5.8.3.
     new 64e2a8c  Updated benchmark and test.
     new 155ef60  Delete comments
     new cf79e27  Extract legacy OS stats.
     new dd3d111  Make service loadable.
     new 2a3b2af  Fix legacy process stats sample.
     new 8c8fa85  Fix tests.
     new 6239e4d  Refactor and cleanup legacy ProcessStats.
     new fb2b8cf  Initial module.

The 16 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


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

Posted by jb...@apache.org.
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;
   }
 }

[geode] 09/16: Updated benchmark and test.

Posted by jb...@apache.org.
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 64e2a8c1f310dc16c5c31158863c1d726dec65a3
Author: Jacob Barrett <jb...@pivotal.io>
AuthorDate: Mon Nov 15 22:55:16 2021 -0800

    Updated benchmark and test.
---
 .../OshiStatisticsBenchmark.java                   |  53 +++----
 .../oshi/OshiStatisticsProviderImpl.java           |  17 ++-
 .../oshi/OshiStatisticsProviderImplTest.java       | 160 +++++++++++++++++++++
 .../statistics/oshi/OshiStatisticsTest.java        |  92 ------------
 4 files changed, 202 insertions(+), 120 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/oshi/OshiStatisticsBenchmark.java
similarity index 87%
rename from geode-core/src/jmh/java/org/apache/geode/internal/statistics/platform/OshiStatisticsBenchmark.java
rename to geode-core/src/jmh/java/org/apache/geode/internal/statistics/oshi/OshiStatisticsBenchmark.java
index ddf9fcf..a2f1746 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/oshi/OshiStatisticsBenchmark.java
@@ -13,51 +13,47 @@
  * the License.
  */
 
-package org.apache.geode.internal.statistics.platform;
+package org.apache.geode.internal.statistics.oshi;
 
 import java.util.concurrent.TimeUnit;
 import java.util.function.DoubleSupplier;
 import java.util.function.IntSupplier;
 import java.util.function.LongSupplier;
 
+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.Scope;
+import org.openjdk.jmh.annotations.Setup;
 import org.openjdk.jmh.annotations.State;
-import oshi.SystemInfo;
 
 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;
 
 @State(Scope.Benchmark)
 @BenchmarkMode(Mode.SampleTime)
 @OutputTimeUnit(TimeUnit.MILLISECONDS)
 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);
-  // }
+  private final OshiStatisticsProviderImpl oshiStatisticsProvider =
+      new OshiStatisticsProviderImpl();
+
+  @Setup
+  public void setup() throws OshiStatisticsProviderException {
+    oshiStatisticsProvider.init(new NoopStatisticsProvider(), 0);
+  }
+
+//  @Benchmark
+//  public void noop() {}
+
+  @Benchmark
+  public void sampleProcess() {
+    oshiStatisticsProvider.sampleProcess();
+  }
 
   private static class NoopStatistics implements SuppliableStatistics {
     @Override
@@ -313,4 +309,13 @@ public class OshiStatisticsBenchmark {
       return null;
     }
   }
+
+  private static class NoopStatisticsProvider implements OsStatisticsFactory {
+    @Override
+    public Statistics createOsStatistics(final StatisticsType type, final String textId,
+        final long numericId,
+        final int osStatFlags) {
+      return new NoopStatistics();
+    }
+  }
 }
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
index 8054881..71201fc 100644
--- 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
@@ -26,7 +26,7 @@ import org.apache.geode.logging.internal.log4j.api.LogService;
 public class OshiStatisticsProviderImpl implements OshiStatisticsProvider {
   private static final Logger log = LogService.getLogger();
 
-  final SystemInfo systemInfo = new SystemInfo();
+  final SystemInfo systemInfo;
 
   private int processId;
   private CentralProcessor processor;
@@ -43,6 +43,14 @@ public class OshiStatisticsProviderImpl implements OshiStatisticsProvider {
   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 {
@@ -77,10 +85,11 @@ public class OshiStatisticsProviderImpl implements OshiStatisticsProvider {
     networkInterfaceStats = new Statistics[networkIFs.size()];
     for (int i = 0, size = networkIFs.size(); i < size; i++) {
       final NetworkIF networkIF = networkIFs.get(i);
-      log.info("Creating network interfaces stats for {}", networkIF.getDisplayName());
+      final String displayName = networkIF.getDisplayName();
+      log.info("Creating network interfaces stats for {}", displayName);
       networkInterfaceStats[i] =
           osStatisticsFactory.createOsStatistics(NetworkInterfaceStats.getType(),
-              networkIF.getDisplayName(), id, 0);
+              displayName, id, 0);
     }
   }
 
@@ -95,7 +104,7 @@ public class OshiStatisticsProviderImpl implements OshiStatisticsProvider {
   @Override
   public void destroy() {}
 
-  private void sampleProcess() {
+  void sampleProcess() {
     final OSProcess process = operatingSystem.getProcess(processId);
 
     final double processCpuLoadBetweenTicks = process.getProcessCpuLoadBetweenTicks(this.process);
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/OshiStatisticsProviderImplTest.java
new file mode 100644
index 0000000..c5244f3
--- /dev/null
+++ b/geode-core/src/test/java/org/apache/geode/internal/statistics/oshi/OshiStatisticsProviderImplTest.java
@@ -0,0 +1,160 @@
+/*
+ * 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 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;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.verifyNoMoreInteractions;
+import static org.mockito.Mockito.when;
+
+import org.junit.jupiter.api.Test;
+import oshi.SystemInfo;
+import oshi.hardware.CentralProcessor;
+import oshi.hardware.HardwareAbstractionLayer;
+import oshi.hardware.NetworkIF;
+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 OshiStatisticsProviderImplTest {
+
+  private static final String SYSTEM_IDENTITY = "mock-operating-system";
+  private static final String PROCESS_IDENTITY = "mock-process";
+  private static final String LOGICAL_PROCESSOR_0_IDENTITY = "mock-processor-0";
+  private static final String LOGICAL_PROCESSOR_1_IDENTITY = "mock-processor-1";
+  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 OSProcess osProcess;
+
+  public OshiStatisticsProviderImplTest() {
+    SystemInfo systemInfo = mock(SystemInfo.class);
+    oshiStatisticsProvider = new OshiStatisticsProviderImpl(systemInfo);
+
+    final OperatingSystem operatingSystem = mock(OperatingSystem.class);
+    when(systemInfo.getOperatingSystem()).thenReturn(operatingSystem);
+
+    final int processId = 42;
+    when(operatingSystem.getProcessId()).thenReturn(processId);
+
+    final HardwareAbstractionLayer hardwareAbstractionLayer = mock(HardwareAbstractionLayer.class);
+    when(systemInfo.getHardware()).thenReturn(hardwareAbstractionLayer);
+
+    final CentralProcessor centralProcessor = mock(CentralProcessor.class);
+    when(hardwareAbstractionLayer.getProcessor()).thenReturn(centralProcessor);
+
+    osProcess = mock(OSProcess.class);
+    when(operatingSystem.getProcess(eq(processId))).thenReturn(osProcess);
+
+    when(osProcess.toString()).thenReturn(PROCESS_IDENTITY);
+
+    when(operatingSystem.toString()).thenReturn(SYSTEM_IDENTITY);
+
+    final CentralProcessor.LogicalProcessor logicalProcessor0 =
+        mock(CentralProcessor.LogicalProcessor.class);
+    when(logicalProcessor0.toString()).thenReturn(LOGICAL_PROCESSOR_0_IDENTITY);
+    final CentralProcessor.LogicalProcessor logicalProcessor1 =
+        mock(CentralProcessor.LogicalProcessor.class);
+    when(logicalProcessor1.toString()).thenReturn(LOGICAL_PROCESSOR_1_IDENTITY);
+    when(centralProcessor.getLogicalProcessors())
+        .thenReturn(asList(logicalProcessor0, logicalProcessor1));
+
+    final NetworkIF networkIf0 = mock(NetworkIF.class);
+    when(networkIf0.getDisplayName()).thenReturn(NETWORK_IF_0_DISPLAY_NAME);
+    final NetworkIF networkIf1 = mock(NetworkIF.class);
+    when(networkIf1.getDisplayName()).thenReturn(NETWORK_IF_1_DISPLAY_NAME);
+    when(hardwareAbstractionLayer.getNetworkIFs()).thenReturn(asList(networkIf0, networkIf1));
+
+  }
+
+  @Test
+  public void initCreatesOsStatistics() throws OshiStatisticsProviderException {
+    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));
+    verify(osStatisticsFactory).createOsStatistics(eq(OperatingSystemStats.getType()),
+        eq(SYSTEM_IDENTITY),
+        eq(id), eq(0));
+    verify(osStatisticsFactory)
+        .createOsStatistics(eq(ProcessorStats.getType()), eq(LOGICAL_PROCESSOR_0_IDENTITY), eq(id),
+            eq(0));
+    verify(osStatisticsFactory)
+        .createOsStatistics(eq(ProcessorStats.getType()), eq(LOGICAL_PROCESSOR_1_IDENTITY), eq(id),
+            eq(0));
+    verify(osStatisticsFactory)
+        .createOsStatistics(eq(NetworkInterfaceStats.getType()), eq(NETWORK_IF_0_DISPLAY_NAME),
+            eq(id), eq(0));
+    verify(osStatisticsFactory)
+        .createOsStatistics(eq(NetworkInterfaceStats.getType()), eq(NETWORK_IF_1_DISPLAY_NAME),
+            eq(id), eq(0));
+
+    verifyNoMoreInteractions(osStatisticsFactory);
+  }
+
+  @Test
+  public void sampleProcessUpdatesStats() throws OshiStatisticsProviderException {
+    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);
+
+    when(osProcess.getProcessCpuLoadBetweenTicks(any())).thenReturn(0.123D);
+    when(osProcess.getVirtualSize()).thenReturn(456L);
+    when(osProcess.getResidentSetSize()).thenReturn(789L);
+    when(osProcess.getThreadCount()).thenReturn(321);
+    when(osProcess.getKernelTime()).thenReturn(654L);
+    when(osProcess.getUserTime()).thenReturn(987L);
+    when(osProcess.getBytesRead()).thenReturn(1234L);
+    when(osProcess.getBytesWritten()).thenReturn(5678L);
+    when(osProcess.getOpenFiles()).thenReturn(9L);
+    when(osProcess.getProcessCpuLoadCumulative()).thenReturn(123.456D);
+    when(osProcess.getMinorFaults()).thenReturn(2L);
+    when(osProcess.getMajorFaults()).thenReturn(3L);
+    when(osProcess.getContextSwitches()).thenReturn(42L);
+
+    oshiStatisticsProvider.init(osStatisticsFactory, 0);
+    oshiStatisticsProvider.sampleProcess();
+
+    verify(statistics).setDouble(eq(ProcessStats.cpuLoad), eq(0.123D));
+    verify(statistics).setLong(eq(ProcessStats.virtualSize), eq(456L));
+    verify(statistics).setLong(eq(ProcessStats.residentSetSize), eq(789L));
+    verify(statistics).setLong(eq(ProcessStats.threadCount), eq(321L));
+    verify(statistics).setLong(eq(ProcessStats.kernelTime), eq(654L));
+    verify(statistics).setLong(eq(ProcessStats.userTime), eq(987L));
+    verify(statistics).setLong(eq(ProcessStats.bytesRead), eq(1234L));
+    verify(statistics).setLong(eq(ProcessStats.bytesWritten), eq(5678L));
+    verify(statistics).setLong(eq(ProcessStats.openFiles), eq(9L));
+    verify(statistics).setDouble(eq(ProcessStats.cpuLoadCumulative), eq(123.456D));
+    verify(statistics).setLong(eq(ProcessStats.minorFaults), eq(2L));
+    verify(statistics).setLong(eq(ProcessStats.majorFaults), eq(3L));
+    verify(statistics).setLong(eq(ProcessStats.contextSwitches), eq(42L));
+
+    verifyNoMoreInteractions(statistics);
+  }
+}
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
deleted file mode 100644
index e87ae72..0000000
--- a/geode-core/src/test/java/org/apache/geode/internal/statistics/oshi/OshiStatisticsTest.java
+++ /dev/null
@@ -1,92 +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.oshi;
-
-
-
-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));
-  // }
-
-}

[geode] 05/16: adds tcp/udp stats adds file handle stats.

Posted by jb...@apache.org.
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 599dbb54f918a54c3aa7a5f6c18810eb1e45eb97
Author: Jacob Barrett <jb...@pivotal.io>
AuthorDate: Sat Jun 12 20:12:46 2021 -0700

    adds tcp/udp stats
    adds file handle stats.
---
 .../statistics/oshi/OperatingSystemStats.java      | 150 +++++++++++++++++++++
 .../oshi/OshiStatisticsProviderImpl.java           |  46 +++++++
 2 files changed, 196 insertions(+)

diff --git a/geode-core/src/main/java/org/apache/geode/internal/statistics/oshi/OperatingSystemStats.java b/geode-core/src/main/java/org/apache/geode/internal/statistics/oshi/OperatingSystemStats.java
index 49f819d..1d4bba0 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/statistics/oshi/OperatingSystemStats.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/statistics/oshi/OperatingSystemStats.java
@@ -52,6 +52,36 @@ public class OperatingSystemStats {
   static final int virtualInUse;
   static final int swapPagesIn;
   static final int swapPagesOut;
+  static final int tcpv4ConnectionsEstablished;
+  static final int tcpv4ConnectionsActive;
+  static final int tcpv4ConnectionsPassive;
+  static final int tcpv4ConnectionFailures;
+  static final int tcpv4ConnectionsReset;
+  static final int tcpv4SegmentsSent;
+  static final int tcpv4SegmentsReceived;
+  static final int tcpv4SegmentsRetransmitted;
+  static final int tcpv4InErrors;
+  static final int tcpv4OutResets;
+  static final int udpv4DatagramsSent;
+  static final int udpv4DatagramsReceived;
+  static final int udpv4DatagramsNoPort;
+  static final int udpv4DatagramsReceivedErrors;
+  static final int tcpv6ConnectionsEstablished;
+  static final int tcpv6ConnectionsActive;
+  static final int tcpv6ConnectionsPassive;
+  static final int tcpv6ConnectionFailures;
+  static final int tcpv6ConnectionsReset;
+  static final int tcpv6SegmentsSent;
+  static final int tcpv6SegmentsReceived;
+  static final int tcpv6SegmentsRetransmitted;
+  static final int tcpv6InErrors;
+  static final int tcpv6OutResets;
+  static final int udpv6DatagramsSent;
+  static final int udpv6DatagramsReceived;
+  static final int udpv6DatagramsNoPort;
+  static final int udpv6DatagramsReceivedErrors;
+  static final int openFileDescriptors;
+  static final int maxFileDescriptors;
 
   @Immutable
   private static final StatisticsType statisticsType;
@@ -144,6 +174,96 @@ public class OperatingSystemStats {
             f.createLongCounter("swapPagesOut",
                 "Number of pages read from paging/swap file(s) to resolve hard page faults.",
                 "pages"),
+            f.createLongGauge("tcpv4ConnectionsEstablished",
+                "Connection Failures is the number of times TCP connections have made a direct transition to the CLOSED state from the SYN-SENT state or the SYN-RCVD state, plus the number of times TCP connections have made a direct transition to the LISTEN state from the SYN-RCVD state.",
+                "connections"),
+            f.createLongCounter("tcpv4ConnectionsActive",
+                "Connections Active is the number of times TCP connections have made a direct transition to the SYN-SENT state from the CLOSED state. In other words, it shows a number of connections which are initiated by the local computer. The value is a cumulative total.",
+                "connections"),
+            f.createLongCounter("tcpv4ConnectionsPassive",
+                "Connections Passive is the number of times TCP connections have made a direct transition to the SYN-RCVD state from the LISTEN state. In other words, it shows a number of connections to the local computer, which are initiated by remote computers. The value is a cumulative total.",
+                "connections"),
+            f.createLongCounter("tcpv4ConnectionFailures",
+                "Connections Established is the number of TCP connections for which the current state is either ESTABLISHED or CLOSE-WAIT.",
+                "connections"),
+            f.createLongCounter("tcpv4ConnectionsReset",
+                "Connections Reset is the number of times TCP connections have made a direct transition to the CLOSED state from either the ESTABLISHED state or the CLOSE-WAIT state.",
+                "connections"),
+            f.createLongCounter("tcpv4SegmentsSent",
+                "Segments Sent is the number of segments sent, including those on current connections, but excluding those containing only retransmitted bytes.",
+                "segments"),
+            f.createLongCounter("tcpv4SegmentsReceived",
+                "Segments Received is the number of segments received, including those received in error. This count includes segments received on currently established connections.",
+                "segments"),
+            f.createLongCounter("tcpv4SegmentsRetransmitted",
+                "Segments Retransmitted is the number of segments retransmitted, that is, segments transmitted containing one or more previously transmitted bytes.",
+                "segments"),
+            f.createLongCounter("tcpv4InErrors",
+                "The number of errors received.",
+                "errors"),
+            f.createLongCounter("tcpv4OutResets",
+                "The number of segments transmitted with the reset flag set.",
+                "segments"),
+            f.createLongCounter("udpv4DatagramsSent",
+                "Datagrams Sent is the number of UDP datagrams sent from the entity.",
+                "datagrams"),
+            f.createLongCounter("udpv4DatagramsReceived",
+                "Datagrams Received is the number of UDP datagrams delivered to UDP users",
+                "datagrams"),
+            f.createLongCounter("udpv4DatagramsNoPort",
+                "Datagrams No Port is the number of received UDP datagrams for which there was no application at the destination port.",
+                "datagrams"),
+            f.createLongCounter("udpv4DatagramsReceivedErrors",
+                "Datagrams Received Errors is the number of received UDP datagrams that could not be delivered for reasons other than the lack of an application at the destination port.",
+                "datagrams"),
+            f.createLongGauge("tcpv6ConnectionsEstablished",
+                "Connection Failures is the number of times TCP connections have made a direct transition to the CLOSED state from the SYN-SENT state or the SYN-RCVD state, plus the number of times TCP connections have made a direct transition to the LISTEN state from the SYN-RCVD state.",
+                "connections"),
+            f.createLongCounter("tcpv6ConnectionsActive",
+                "Connections Active is the number of times TCP connections have made a direct transition to the SYN-SENT state from the CLOSED state. In other words, it shows a number of connections which are initiated by the local computer. The value is a cumulative total.",
+                "connections"),
+            f.createLongCounter("tcpv6ConnectionsPassive",
+                "Connections Passive is the number of times TCP connections have made a direct transition to the SYN-RCVD state from the LISTEN state. In other words, it shows a number of connections to the local computer, which are initiated by remote computers. The value is a cumulative total.",
+                "connections"),
+            f.createLongCounter("tcpv6ConnectionFailures",
+                "Connections Established is the number of TCP connections for which the current state is either ESTABLISHED or CLOSE-WAIT.",
+                "connections"),
+            f.createLongCounter("tcpv6ConnectionsReset",
+                "Connections Reset is the number of times TCP connections have made a direct transition to the CLOSED state from either the ESTABLISHED state or the CLOSE-WAIT state.",
+                "connections"),
+            f.createLongCounter("tcpv6SegmentsSent",
+                "Segments Sent is the number of segments sent, including those on current connections, but excluding those containing only retransmitted bytes.",
+                "segments"),
+            f.createLongCounter("tcpv6SegmentsReceived",
+                "Segments Received is the number of segments received, including those received in error. This count includes segments received on currently established connections.",
+                "segments"),
+            f.createLongCounter("tcpv6SegmentsRetransmitted",
+                "Segments Retransmitted is the number of segments retransmitted, that is, segments transmitted containing one or more previously transmitted bytes.",
+                "segments"),
+            f.createLongCounter("tcpv6InErrors",
+                "The number of errors received.",
+                "errors"),
+            f.createLongCounter("tcpv6OutResets",
+                "The number of segments transmitted with the reset flag set.",
+                "segments"),
+            f.createLongCounter("udpv6DatagramsSent",
+                "Datagrams Sent is the number of UDP datagrams sent from the entity.",
+                "datagrams"),
+            f.createLongCounter("udpv6DatagramsReceived",
+                "Datagrams Received is the number of UDP datagrams delivered to UDP users",
+                "datagrams"),
+            f.createLongCounter("udpv6DatagramsNoPort",
+                "Datagrams No Port is the number of received UDP datagrams for which there was no application at the destination port.",
+                "datagrams"),
+            f.createLongCounter("udpv6DatagramsReceivedErrors",
+                "Datagrams Received Errors is the number of received UDP datagrams that could not be delivered for reasons other than the lack of an application at the destination port.",
+                "datagrams"),
+            f.createLongGauge("openFileDescriptors",
+                "The current number of open file descriptors",
+                "files"),
+            f.createLongGauge("maxFileDescriptors",
+                "The maximum number of open file descriptors.",
+                "files"),
     });
 
     processCount = statisticsType.nameToId("processCount");
@@ -174,6 +294,36 @@ public class OperatingSystemStats {
     virtualInUse = statisticsType.nameToId("virtualInUse");
     swapPagesIn = statisticsType.nameToId("swapPagesIn");
     swapPagesOut = statisticsType.nameToId("swapPagesOut");
+    tcpv4ConnectionsEstablished = statisticsType.nameToId("tcpv4ConnectionsEstablished");
+    tcpv4ConnectionsActive = statisticsType.nameToId("tcpv4ConnectionsActive");
+    tcpv4ConnectionsPassive = statisticsType.nameToId("tcpv4ConnectionsPassive");
+    tcpv4ConnectionFailures = statisticsType.nameToId("tcpv4ConnectionFailures");
+    tcpv4ConnectionsReset = statisticsType.nameToId("tcpv4ConnectionsReset");
+    tcpv4SegmentsSent = statisticsType.nameToId("tcpv4SegmentsSent");
+    tcpv4SegmentsReceived = statisticsType.nameToId("tcpv4SegmentsReceived");
+    tcpv4SegmentsRetransmitted = statisticsType.nameToId("tcpv4SegmentsRetransmitted");
+    tcpv4InErrors = statisticsType.nameToId("tcpv4InErrors");
+    tcpv4OutResets = statisticsType.nameToId("tcpv4OutResets");
+    udpv4DatagramsSent = statisticsType.nameToId("udpv4DatagramsSent");
+    udpv4DatagramsReceived = statisticsType.nameToId("udpv4DatagramsReceived");
+    udpv4DatagramsNoPort = statisticsType.nameToId("udpv4DatagramsNoPort");
+    udpv4DatagramsReceivedErrors = statisticsType.nameToId("udpv4DatagramsReceivedErrors");
+    tcpv6ConnectionsEstablished = statisticsType.nameToId("tcpv6ConnectionsEstablished");
+    tcpv6ConnectionsActive = statisticsType.nameToId("tcpv6ConnectionsActive");
+    tcpv6ConnectionsPassive = statisticsType.nameToId("tcpv6ConnectionsPassive");
+    tcpv6ConnectionFailures = statisticsType.nameToId("tcpv6ConnectionFailures");
+    tcpv6ConnectionsReset = statisticsType.nameToId("tcpv6ConnectionsReset");
+    tcpv6SegmentsSent = statisticsType.nameToId("tcpv6SegmentsSent");
+    tcpv6SegmentsReceived = statisticsType.nameToId("tcpv6SegmentsReceived");
+    tcpv6SegmentsRetransmitted = statisticsType.nameToId("tcpv6SegmentsRetransmitted");
+    tcpv6InErrors = statisticsType.nameToId("tcpv6InErrors");
+    tcpv6OutResets = statisticsType.nameToId("tcpv6OutResets");
+    udpv6DatagramsSent = statisticsType.nameToId("udpv6DatagramsSent");
+    udpv6DatagramsReceived = statisticsType.nameToId("udpv6DatagramsReceived");
+    udpv6DatagramsNoPort = statisticsType.nameToId("udpv6DatagramsNoPort");
+    udpv6DatagramsReceivedErrors = statisticsType.nameToId("udpv6DatagramsReceivedErrors");
+    openFileDescriptors = statisticsType.nameToId("openFileDescriptors");
+    maxFileDescriptors = statisticsType.nameToId("maxFileDescriptors");
 
   }
 
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
index ce02b38..44fc546 100644
--- 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
@@ -12,6 +12,10 @@ 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;
 
@@ -161,6 +165,48 @@ public class OshiStatisticsProviderImpl implements OshiStatisticsProvider {
     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.openFileDescriptors, fileSystem.getMaxFileDescriptors());
+
   }
 
   private void sampleProcessors() {

[geode] 07/16: fixes max files

Posted by jb...@apache.org.
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 25b39ea10d006f73cfdd3ceb308e37ef0a279650
Author: Jacob Barrett <jb...@pivotal.io>
AuthorDate: Sat Jun 12 20:24:02 2021 -0700

    fixes max files
---
 .../oshi/OshiStatisticsProviderImpl.java           | 32 ++++++++++++++--------
 1 file changed, 21 insertions(+), 11 deletions(-)

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
index b79a038..8054881 100644
--- 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
@@ -181,19 +181,24 @@ public class OshiStatisticsProviderImpl implements OshiStatisticsProvider {
         tcPv4Stats.getConnectionFailures());
     systemStats.setLong(OperatingSystemStats.tcpv4ConnectionsReset,
         tcPv4Stats.getConnectionsReset());
-    systemStats.setLong(OperatingSystemStats.tcpv4SegmentsSent, tcPv4Stats.getSegmentsSent());
+    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());
+    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.udpv4DatagramsSent,
+        udPv4Stats.getDatagramsSent());
     systemStats.setLong(OperatingSystemStats.udpv4DatagramsReceived,
         udPv4Stats.getDatagramsReceived());
-    systemStats.setLong(OperatingSystemStats.udpv4DatagramsNoPort, udPv4Stats.getDatagramsNoPort());
+    systemStats.setLong(OperatingSystemStats.udpv4DatagramsNoPort,
+        udPv4Stats.getDatagramsNoPort());
     systemStats.setLong(OperatingSystemStats.udpv4DatagramsReceivedErrors,
         udPv4Stats.getDatagramsReceivedErrors());
 
@@ -208,26 +213,31 @@ public class OshiStatisticsProviderImpl implements OshiStatisticsProvider {
         tcPv6Stats.getConnectionFailures());
     systemStats.setLong(OperatingSystemStats.tcpv6ConnectionsReset,
         tcPv6Stats.getConnectionsReset());
-    systemStats.setLong(OperatingSystemStats.tcpv6SegmentsSent, tcPv6Stats.getSegmentsSent());
+    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());
+    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.udpv6DatagramsSent,
+        udPv6Stats.getDatagramsSent());
     systemStats.setLong(OperatingSystemStats.udpv6DatagramsReceived,
         udPv6Stats.getDatagramsReceived());
-    systemStats.setLong(OperatingSystemStats.udpv6DatagramsNoPort, udPv6Stats.getDatagramsNoPort());
+    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.openFileDescriptors,
+    systemStats.setLong(OperatingSystemStats.maxFileDescriptors,
         fileSystem.getMaxFileDescriptors());
 
   }

[geode] 15/16: Refactor and cleanup legacy ProcessStats.

Posted by jb...@apache.org.
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 6239e4d653870e85b334137c72fb786968948da0
Author: Jacob Barrett <jb...@pivotal.io>
AuthorDate: Thu Nov 18 13:11:45 2021 -0800

    Refactor and cleanup legacy ProcessStats.
---
 .../internal/MemberHealthEvaluatorJUnitTest.java   | 16 +++--
 .../GemFireStatSamplerIntegrationTest.java         |  4 +-
 .../internal/statistics/LinuxSystemStatsTest.java  | 12 ++--
 .../internal/statistics/OsStatisticsBenchmark.java |  4 +-
 .../admin/internal/MemberHealthEvaluator.java      | 13 ++--
 .../internal/InternalDistributedSystem.java        |  2 +-
 .../internal/statistics/GemFireStatSampler.java    | 24 ++++++-
 ...ovider.java => LegacyOsStatisticsProvider.java} | 15 +++--
 .../internal/statistics/OsStatisticsProvider.java  |  5 ++
 ...isticsProvider.java => ProcessSizeSuppler.java} | 10 +--
 ...rovider.java => LinuxOsStatisticsProvider.java} | 76 ++++++++++++----------
 .../LinuxProcFsStatistics.java                     |  2 +-
 .../{platform => legacy}/LinuxProcessStats.java    | 15 ++---
 .../{platform => legacy}/LinuxSystemStats.java     |  2 +-
 .../statistics/oshi/OshiStatisticsProvider.java    |  7 ++
 .../internal/statistics/platform/ProcessStats.java | 22 ++-----
 .../internal/beans/MemberMBeanBridge.java          | 12 ++--
 ....geode.internal.statistics.OsStatisticsProvider |  2 +-
 .../legacy/LegacyOsStatisticsProviderTest.java     |  2 +-
 19 files changed, 136 insertions(+), 109 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 4809b40..9e84c07 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
@@ -21,13 +21,14 @@ import static org.junit.Assert.assertTrue;
 import java.util.ArrayList;
 import java.util.List;
 
+import org.jetbrains.annotations.Nullable;
 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.legacy.LegacyOsStatisticsProvider;
-import org.apache.geode.internal.statistics.platform.ProcessStats;
+import org.apache.geode.internal.statistics.ProcessSizeSuppler;
+import org.apache.geode.internal.statistics.legacy.LinuxOsStatisticsProvider;
 
 /**
  * Contains simple tests for the {@link MemberHealthEvaluator}.
@@ -46,17 +47,18 @@ public class MemberHealthEvaluatorJUnitTest extends HealthEvaluatorTestCase {
    */
   @Test
   public void testCheckVMProcessSize() throws InterruptedException {
-    if (LegacyOsStatisticsProvider.build().osStatsSupported()) {
+    if (LinuxOsStatisticsProvider.build().osStatsSupported()) {
       GemFireStatSampler sampler = system.getStatSampler();
       assertNotNull(sampler);
 
       sampler.waitForInitialization(10000); // fix: remove infinite wait
 
-      ProcessStats stats = sampler.getProcessStats();
-      assertNotNull(stats);
+      @Nullable
+      ProcessSizeSuppler processSizeSuppler = sampler.getProcessSizeSuppler();
+      assertNotNull(processSizeSuppler);
 
       List status = new ArrayList();
-      long threshold = stats.getProcessSize() * 2;
+      long threshold = processSizeSuppler.getAsLong() * 2;
 
       if (threshold <= 0) {
         // The process size is zero on some Linux versions
@@ -72,7 +74,7 @@ public class MemberHealthEvaluatorJUnitTest extends HealthEvaluatorTestCase {
       assertTrue(status.isEmpty());
 
       status = new ArrayList();
-      long processSize = stats.getProcessSize();
+      long processSize = processSizeSuppler.getAsLong();
       threshold = processSize / 2;
       assertTrue("Threshold (" + threshold + ") is > 0.  " + "Process size is " + processSize,
           threshold > 0);
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 08b20e1..b67febe 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,7 +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.legacy.LinuxOsStatisticsProvider;
 import org.apache.geode.internal.statistics.platform.ProcessStats;
 import org.apache.geode.internal.stats50.VMStats50;
 import org.apache.geode.logging.internal.log4j.api.LogService;
@@ -172,7 +172,7 @@ public class GemFireStatSamplerIntegrationTest extends StatSamplerTestCase {
       assertThat(processStats)
           .withFailMessage("ProcessStats were not created on" + osName)
           .isNotNull();
-      assertThat(LegacyOsStatisticsProvider.build().osStatsSupported())
+      assertThat(LinuxOsStatisticsProvider.build().osStatsSupported())
           .as("os stats are available on Linux")
           .isTrue();
       assertThat(allStats.containsStatisticsType("LinuxProcessStats"))
diff --git a/geode-core/src/integrationTest/java/org/apache/geode/internal/statistics/LinuxSystemStatsTest.java b/geode-core/src/integrationTest/java/org/apache/geode/internal/statistics/LinuxSystemStatsTest.java
index b53c6ca..50221e2 100644
--- a/geode-core/src/integrationTest/java/org/apache/geode/internal/statistics/LinuxSystemStatsTest.java
+++ b/geode-core/src/integrationTest/java/org/apache/geode/internal/statistics/LinuxSystemStatsTest.java
@@ -14,10 +14,10 @@
  */
 package org.apache.geode.internal.statistics;
 
-import static org.apache.geode.internal.statistics.platform.LinuxSystemStats.TCP_EXT_LISTEN_DROPS;
-import static org.apache.geode.internal.statistics.platform.LinuxSystemStats.TCP_EXT_LISTEN_OVERFLOWS;
-import static org.apache.geode.internal.statistics.platform.LinuxSystemStats.TCP_EXT_SYN_COOKIES_RECV;
-import static org.apache.geode.internal.statistics.platform.LinuxSystemStats.TCP_EXT_SYN_COOKIES_SENT;
+import static org.apache.geode.internal.statistics.legacy.LinuxSystemStats.TCP_EXT_LISTEN_DROPS;
+import static org.apache.geode.internal.statistics.legacy.LinuxSystemStats.TCP_EXT_LISTEN_OVERFLOWS;
+import static org.apache.geode.internal.statistics.legacy.LinuxSystemStats.TCP_EXT_SYN_COOKIES_RECV;
+import static org.apache.geode.internal.statistics.legacy.LinuxSystemStats.TCP_EXT_SYN_COOKIES_SENT;
 import static org.assertj.core.api.Assertions.assertThat;
 
 import java.io.File;
@@ -32,8 +32,8 @@ import org.junit.rules.TemporaryFolder;
 
 import org.apache.geode.CancelCriterion;
 import org.apache.geode.Statistics;
-import org.apache.geode.internal.statistics.platform.LinuxProcFsStatistics;
-import org.apache.geode.internal.statistics.platform.LinuxSystemStats;
+import org.apache.geode.internal.statistics.legacy.LinuxProcFsStatistics;
+import org.apache.geode.internal.statistics.legacy.LinuxSystemStats;
 import org.apache.geode.test.junit.categories.StatisticsTest;
 
 
diff --git a/geode-core/src/jmh/java/org/apache/geode/internal/statistics/OsStatisticsBenchmark.java b/geode-core/src/jmh/java/org/apache/geode/internal/statistics/OsStatisticsBenchmark.java
index cf91437..12abc01 100644
--- a/geode-core/src/jmh/java/org/apache/geode/internal/statistics/OsStatisticsBenchmark.java
+++ b/geode-core/src/jmh/java/org/apache/geode/internal/statistics/OsStatisticsBenchmark.java
@@ -33,7 +33,7 @@ import org.apache.geode.StatisticDescriptor;
 import org.apache.geode.Statistics;
 import org.apache.geode.StatisticsType;
 import org.apache.geode.internal.process.ProcessUtils;
-import org.apache.geode.internal.statistics.legacy.LegacyOsStatisticsProvider;
+import org.apache.geode.internal.statistics.legacy.LinuxOsStatisticsProvider;
 import org.apache.geode.internal.statistics.oshi.OshiStatisticsProvider;
 
 @State(Scope.Benchmark)
@@ -54,7 +54,7 @@ public class OsStatisticsBenchmark {
   public void setup() throws OsStatisticsProviderException {
     switch (impl) {
       case Legacy:
-        osStatisticsProvider = new LegacyOsStatisticsProvider();
+        osStatisticsProvider = new LinuxOsStatisticsProvider();
         break;
       case Oshi:
         osStatisticsProvider = new OshiStatisticsProvider();
diff --git a/geode-core/src/main/java/org/apache/geode/admin/internal/MemberHealthEvaluator.java b/geode-core/src/main/java/org/apache/geode/admin/internal/MemberHealthEvaluator.java
index 1cce491..9accb9f 100644
--- a/geode-core/src/main/java/org/apache/geode/admin/internal/MemberHealthEvaluator.java
+++ b/geode-core/src/main/java/org/apache/geode/admin/internal/MemberHealthEvaluator.java
@@ -26,7 +26,7 @@ import org.apache.geode.distributed.internal.InternalDistributedSystem;
 import org.apache.geode.internal.cache.CachePerfStats;
 import org.apache.geode.internal.cache.InternalCache;
 import org.apache.geode.internal.statistics.GemFireStatSampler;
-import org.apache.geode.internal.statistics.platform.ProcessStats;
+import org.apache.geode.internal.statistics.ProcessSizeSuppler;
 import org.apache.geode.logging.internal.OSProcess;
 
 /**
@@ -44,7 +44,7 @@ class MemberHealthEvaluator extends AbstractHealthEvaluator {
   private final String description;
 
   /** Statistics about this process (may be null) */
-  private ProcessStats processStats;
+  private ProcessSizeSuppler processSizeSuppler;
 
   /** Statistics about the distribution manager */
   private final DMStats dmStats;
@@ -64,7 +64,7 @@ class MemberHealthEvaluator extends AbstractHealthEvaluator {
     GemFireStatSampler sampler = system.getStatSampler();
     if (sampler != null) {
       // Sampling is enabled
-      this.processStats = sampler.getProcessStats();
+      this.processSizeSuppler = sampler.getProcessSizeSuppler();
     }
 
     this.dmStats = dm.getStats();
@@ -86,17 +86,18 @@ class MemberHealthEvaluator extends AbstractHealthEvaluator {
   }
 
   /**
-   * Checks to make sure that the {@linkplain ProcessStats#getProcessSize VM's process size} is less
+   * Checks to make sure that the {@linkplain GemFireStatSampler#getProcessSizeSuppler() VM's
+   * process size} is less
    * than the {@linkplain MemberHealthConfig#getMaxVMProcessSize threshold}. If not, the status is
    * "okay" health.
    */
   void checkVMProcessSize(List<HealthStatus> status) {
     // There is no need to check isFirstEvaluation()
-    if (this.processStats == null) {
+    if (this.processSizeSuppler == null) {
       return;
     }
 
-    long vmSize = this.processStats.getProcessSize();
+    long vmSize = this.processSizeSuppler.getAsLong();
     long threshold = this.config.getMaxVMProcessSize();
     if (vmSize > threshold) {
       String s =
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/InternalDistributedSystem.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/InternalDistributedSystem.java
index e8cecc1..d051dc0 100644
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/InternalDistributedSystem.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/InternalDistributedSystem.java
@@ -105,7 +105,7 @@ import org.apache.geode.internal.statistics.StatisticsConfig;
 import org.apache.geode.internal.statistics.StatisticsManager;
 import org.apache.geode.internal.statistics.StatisticsManagerFactory;
 import org.apache.geode.internal.statistics.StatisticsRegistry;
-import org.apache.geode.internal.statistics.platform.LinuxProcFsStatistics;
+import org.apache.geode.internal.statistics.legacy.LinuxProcFsStatistics;
 import org.apache.geode.internal.tcp.ConnectionTable;
 import org.apache.geode.logging.internal.LoggingSession;
 import org.apache.geode.logging.internal.NullLoggingSession;
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 2a1c79d..9189333 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
@@ -26,6 +26,7 @@ import java.util.concurrent.ConcurrentHashMap;
 
 import org.apache.logging.log4j.Logger;
 import org.jetbrains.annotations.NotNull;
+import org.jetbrains.annotations.Nullable;
 
 import org.apache.geode.CancelCriterion;
 import org.apache.geode.Statistics;
@@ -65,9 +66,12 @@ public class GemFireStatSampler extends HostStatSampler {
   private final DistributionManager distributionManager;
 
   private int nextListenerId = 1;
+
+  @Deprecated
   private ProcessStats processStats;
 
   private final OsStatisticsProvider[] osStatisticsProviders;
+  private ProcessSizeSuppler processSizeSuppler;
 
   public GemFireStatSampler(InternalDistributedSystem internalDistributedSystem) {
     this(internalDistributedSystem, null);
@@ -128,11 +132,17 @@ public class GemFireStatSampler extends HostStatSampler {
    * returned if operating statistics are disabled.
    *
    * @since GemFire 3.5
+   * @deprecated no replacement
    */
-  public ProcessStats getProcessStats() {
+  @Deprecated
+  public @Nullable ProcessStats getProcessStats() {
     return processStats;
   }
 
+  public @Nullable ProcessSizeSuppler getProcessSizeSuppler() {
+    return processSizeSuppler;
+  }
+
   @Override
   public String getProductDescription() {
     return "GemFire " + GemFireVersion.getGemFireVersion() + " #" + GemFireVersion.getBuildId()
@@ -307,12 +317,22 @@ public class GemFireStatSampler extends HostStatSampler {
     try {
       for (final OsStatisticsProvider osStatisticsProvider : osStatisticsProviders) {
         osStatisticsProvider.init(osStatisticsFactory, pid);
+        if (null == processSizeSuppler) {
+          processSizeSuppler = osStatisticsProvider.createProcessSizeSuppler();
+        }
+        registerLegacyOsStatisticsProvider(osStatisticsProvider);
       }
     } catch (OsStatisticsProviderException e) {
       logger.error(LogMarker.STATISTICS_MARKER, "Failed to initialize OS statistics.", e);
     }
+  }
 
-    processStats = null; // TODO jbarrett osStatisticsProvider.newProcessStats(stats);
+  @SuppressWarnings("deprecation")
+  private void registerLegacyOsStatisticsProvider(
+      final @NotNull OsStatisticsProvider osStatisticsProvider) {
+    if (null == processStats && osStatisticsProvider instanceof LegacyOsStatisticsProvider) {
+      processStats = ((LegacyOsStatisticsProvider) osStatisticsProvider).getProcessStats();
+    }
   }
 
   @Override
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/LegacyOsStatisticsProvider.java
similarity index 77%
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/LegacyOsStatisticsProvider.java
index 79a96b8..c9e09e1 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/LegacyOsStatisticsProvider.java
@@ -15,13 +15,14 @@
 
 package org.apache.geode.internal.statistics;
 
-import org.jetbrains.annotations.NotNull;
+import org.apache.geode.internal.statistics.platform.ProcessStats;
 
-public interface OsStatisticsProvider {
-  void init(final @NotNull OsStatisticsFactory osStatisticsFactory,
-      final long pid) throws OsStatisticsProviderException;
-
-  void sample();
+/**
+ * @deprecated no replacement
+ */
+@Deprecated
+public interface LegacyOsStatisticsProvider {
 
-  void destroy();
+  @Deprecated
+  ProcessStats getProcessStats();
 }
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 79a96b8..b2adcc3 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
@@ -16,6 +16,7 @@
 package org.apache.geode.internal.statistics;
 
 import org.jetbrains.annotations.NotNull;
+import org.jetbrains.annotations.Nullable;
 
 public interface OsStatisticsProvider {
   void init(final @NotNull OsStatisticsFactory osStatisticsFactory,
@@ -24,4 +25,8 @@ public interface OsStatisticsProvider {
   void sample();
 
   void destroy();
+
+  default @Nullable ProcessSizeSuppler createProcessSizeSuppler() {
+    return null;
+  }
 }
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/ProcessSizeSuppler.java
similarity index 77%
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/ProcessSizeSuppler.java
index 79a96b8..d7b1a20 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/ProcessSizeSuppler.java
@@ -15,13 +15,9 @@
 
 package org.apache.geode.internal.statistics;
 
-import org.jetbrains.annotations.NotNull;
+import java.util.function.LongSupplier;
 
-public interface OsStatisticsProvider {
-  void init(final @NotNull OsStatisticsFactory osStatisticsFactory,
-      final long pid) throws OsStatisticsProviderException;
+@FunctionalInterface
+public interface ProcessSizeSuppler extends LongSupplier {
 
-  void sample();
-
-  void destroy();
 }
diff --git a/geode-core/src/main/java/org/apache/geode/internal/statistics/legacy/LegacyOsStatisticsProvider.java b/geode-core/src/main/java/org/apache/geode/internal/statistics/legacy/LinuxOsStatisticsProvider.java
similarity index 68%
rename from geode-core/src/main/java/org/apache/geode/internal/statistics/legacy/LegacyOsStatisticsProvider.java
rename to geode-core/src/main/java/org/apache/geode/internal/statistics/legacy/LinuxOsStatisticsProvider.java
index 349b7ee..9101410 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/statistics/legacy/LegacyOsStatisticsProvider.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/statistics/legacy/LinuxOsStatisticsProvider.java
@@ -17,22 +17,22 @@ package org.apache.geode.internal.statistics.legacy;
 import java.net.UnknownHostException;
 
 import org.jetbrains.annotations.NotNull;
+import org.jetbrains.annotations.Nullable;
 
 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.LegacyOsStatisticsProvider;
 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.ProcessSizeSuppler;
 import org.apache.geode.internal.statistics.platform.ProcessStats;
 
 /**
  * Provides methods which fetch operating system statistics.
  * Only Linux OS is currently allowed.
  */
-public class LegacyOsStatisticsProvider implements OsStatisticsProvider {
+public class LinuxOsStatisticsProvider implements OsStatisticsProvider, LegacyOsStatisticsProvider {
   private final boolean osStatsSupported;
   private Statistics systemStatistics;
   private Statistics processStatistics;
@@ -42,27 +42,19 @@ public class LegacyOsStatisticsProvider implements OsStatisticsProvider {
     return osStatsSupported;
   }
 
-  public LegacyOsStatisticsProvider() {
+  public LinuxOsStatisticsProvider() {
     osStatsSupported = SystemUtils.isLinux();
   }
 
-  public static LegacyOsStatisticsProvider build() {
-    return new LegacyOsStatisticsProvider();
-  }
-
-  private void initOSStats() {
-    LinuxProcFsStatistics.init();
-  }
-
-  private void closeOSStats() {
-    LinuxProcFsStatistics.close();
+  public static LinuxOsStatisticsProvider build() {
+    return new LinuxOsStatisticsProvider();
   }
 
   /**
    * 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(@NotNull final Statistics statistics) {
+  private void refreshProcess(final @NotNull Statistics statistics) {
     int pid = (int) statistics.getNumericId();
     LinuxProcFsStatistics.refreshProcess(pid, statistics);
   }
@@ -71,7 +63,7 @@ public class LegacyOsStatisticsProvider implements 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(@NotNull final Statistics statistics) {
+  private void refreshSystem(final @NotNull Statistics statistics) {
     LinuxProcFsStatistics.refreshSystem(statistics);
   }
 
@@ -79,7 +71,8 @@ public class LegacyOsStatisticsProvider implements OsStatisticsProvider {
    * 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.
    */
-  private Statistics newProcess(OsStatisticsFactory osStatisticsFactory, long pid, String name) {
+  private Statistics newProcess(final @NotNull OsStatisticsFactory osStatisticsFactory, long pid,
+      String name) {
     return osStatisticsFactory.createOsStatistics(LinuxProcessStats.getType(), name, pid);
   }
 
@@ -89,7 +82,7 @@ public class LegacyOsStatisticsProvider implements OsStatisticsProvider {
    * @see #newProcess
    * @since GemFire 3.5
    */
-  private @NotNull ProcessStats newProcessStats(@NotNull Statistics statistics) {
+  private @NotNull ProcessStats newProcessStats(final @NotNull Statistics statistics) {
     refreshProcess(statistics);
     return LinuxProcessStats.createProcessStats(statistics);
   }
@@ -98,7 +91,7 @@ public class LegacyOsStatisticsProvider implements 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.
    */
-  private Statistics newSystem(@NotNull OsStatisticsFactory osStatisticsFactory, long id) {
+  private Statistics newSystem(final @NotNull OsStatisticsFactory osStatisticsFactory, long id) {
     final Statistics statistics = osStatisticsFactory.createOsStatistics(LinuxSystemStats.getType(),
         getHostSystemName(), id);
     refreshSystem(statistics);
@@ -126,28 +119,43 @@ public class LegacyOsStatisticsProvider implements OsStatisticsProvider {
 
   @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);
+    if (osStatsSupported) {
+      LinuxProcFsStatistics.init();
+
+      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();
+    if (osStatsSupported) {
+      sampleSystem();
+      sampleProcess();
+    }
   }
 
   @Override
   public void destroy() {
-    processStats.close();
-    closeOSStats();
+    if (osStatsSupported) {
+      LinuxProcFsStatistics.close();
+    }
+  }
+
+  @Override
+  public @Nullable ProcessSizeSuppler createProcessSizeSuppler() {
+    return osStatsSupported ? () -> processStatistics.getLong(LinuxProcessStats.rssSizeLONG) : null;
   }
 
+  @Override
+  public ProcessStats getProcessStats() {
+    return processStats;
+  }
 }
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/legacy/LinuxProcFsStatistics.java
similarity index 99%
rename from geode-core/src/main/java/org/apache/geode/internal/statistics/platform/LinuxProcFsStatistics.java
rename to geode-core/src/main/java/org/apache/geode/internal/statistics/legacy/LinuxProcFsStatistics.java
index e8f3720..1ad3ebb 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/legacy/LinuxProcFsStatistics.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.legacy;
 
 import java.io.BufferedReader;
 import java.io.File;
diff --git a/geode-core/src/main/java/org/apache/geode/internal/statistics/platform/LinuxProcessStats.java b/geode-core/src/main/java/org/apache/geode/internal/statistics/legacy/LinuxProcessStats.java
similarity index 87%
rename from geode-core/src/main/java/org/apache/geode/internal/statistics/platform/LinuxProcessStats.java
rename to geode-core/src/main/java/org/apache/geode/internal/statistics/legacy/LinuxProcessStats.java
index ccecba5..2086624 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/statistics/platform/LinuxProcessStats.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/statistics/legacy/LinuxProcessStats.java
@@ -13,7 +13,7 @@
  * the License.
  */
 
-package org.apache.geode.internal.statistics.platform;
+package org.apache.geode.internal.statistics.legacy;
 
 import org.apache.geode.StatisticDescriptor;
 import org.apache.geode.Statistics;
@@ -21,6 +21,7 @@ import org.apache.geode.StatisticsType;
 import org.apache.geode.StatisticsTypeFactory;
 import org.apache.geode.annotations.Immutable;
 import org.apache.geode.internal.statistics.StatisticsTypeFactoryImpl;
+import org.apache.geode.internal.statistics.platform.ProcessStats;
 
 /**
  * <P>
@@ -56,17 +57,11 @@ public class LinuxProcessStats {
   }
 
   /**
-   * Returns a <code>ProcessStats</code> that wraps Linux process <code>Statistics</code>.
-   *
-   * @since GemFire 3.5
+   * @deprecated no replacement
    */
+  @Deprecated
   public static ProcessStats createProcessStats(final Statistics stats) {
-    return new ProcessStats(stats) {
-      @Override
-      public long getProcessSize() {
-        return stats.getLong(rssSizeLONG);
-      }
-    };
+    return new ProcessStats(stats);
   }
 
 }
diff --git a/geode-core/src/main/java/org/apache/geode/internal/statistics/platform/LinuxSystemStats.java b/geode-core/src/main/java/org/apache/geode/internal/statistics/legacy/LinuxSystemStats.java
similarity index 99%
rename from geode-core/src/main/java/org/apache/geode/internal/statistics/platform/LinuxSystemStats.java
rename to geode-core/src/main/java/org/apache/geode/internal/statistics/legacy/LinuxSystemStats.java
index b241c97..f7c01f6 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/statistics/platform/LinuxSystemStats.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/statistics/legacy/LinuxSystemStats.java
@@ -13,7 +13,7 @@
  * the License.
  */
 
-package org.apache.geode.internal.statistics.platform;
+package org.apache.geode.internal.statistics.legacy;
 
 import org.apache.geode.StatisticDescriptor;
 import org.apache.geode.StatisticsType;
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 35f67bb..9adf74e 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
@@ -4,6 +4,7 @@ import java.util.List;
 
 import org.apache.logging.log4j.Logger;
 import org.jetbrains.annotations.NotNull;
+import org.jetbrains.annotations.Nullable;
 import oshi.SystemInfo;
 import oshi.hardware.CentralProcessor;
 import oshi.hardware.CentralProcessor.LogicalProcessor;
@@ -23,6 +24,7 @@ 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.internal.statistics.ProcessSizeSuppler;
 import org.apache.geode.logging.internal.log4j.api.LogService;
 
 public class OshiStatisticsProvider implements OsStatisticsProvider {
@@ -306,4 +308,9 @@ public class OshiStatisticsProvider implements OsStatisticsProvider {
     }
 
   }
+
+  @Override
+  public @Nullable ProcessSizeSuppler createProcessSizeSuppler() {
+    return () -> processStats.getLong(ProcessStats.residentSetSize);
+  }
 }
diff --git a/geode-core/src/main/java/org/apache/geode/internal/statistics/platform/ProcessStats.java b/geode-core/src/main/java/org/apache/geode/internal/statistics/platform/ProcessStats.java
index 690b514..e838c59 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/statistics/platform/ProcessStats.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/statistics/platform/ProcessStats.java
@@ -22,8 +22,10 @@ import org.apache.geode.Statistics;
  *
  *
  * @since GemFire 3.5
+ * @deprecated no replacement
  */
-public abstract class ProcessStats {
+@Deprecated
+public class ProcessStats {
 
   /** The underlying statistics */
   private final Statistics stats;
@@ -31,26 +33,12 @@ public abstract class ProcessStats {
   /**
    * Creates a new <code>ProcessStats</code> that wraps the given <code>Statistics</code>.
    */
-  ProcessStats(Statistics stats) {
+  public ProcessStats(Statistics stats) {
     this.stats = stats;
   }
 
-  /**
-   * Closes these process stats
-   *
-   * @see Statistics#close
-   */
-  public void close() {
-    this.stats.close();
-  }
-
   public Statistics getStatistics() {
-    return this.stats;
+    return stats;
   }
 
-  /**
-   * Returns the size of this process (resident set on UNIX or working set on Windows) in megabytes
-   */
-  public abstract long getProcessSize();
-
 }
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 e2e8aa6..8bbe536 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
@@ -76,8 +76,8 @@ import org.apache.geode.internal.serialization.KnownVersion;
 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.legacy.LinuxOsStatisticsProvider;
+import org.apache.geode.internal.statistics.legacy.LinuxSystemStats;
 import org.apache.geode.internal.statistics.platform.ProcessStats;
 import org.apache.geode.internal.stats50.VMStats50;
 import org.apache.geode.internal.tcp.ConnectionTable;
@@ -110,8 +110,8 @@ public class MemberMBeanBridge {
   private static final String MEMBER_LEVEL_REGION_MONITOR = "MemberLevelRegionMonitor";
   private static final long MBFactor = 1024 * 1024;
 
-  private final LegacyOsStatisticsProvider osStatisticsProvider =
-      LegacyOsStatisticsProvider.build();
+  private final LinuxOsStatisticsProvider osStatisticsProvider =
+      LinuxOsStatisticsProvider.build();
 
   private InternalCache cache;
   private DistributionConfig config;
@@ -424,6 +424,10 @@ public class MemberMBeanBridge {
     monitor.addStatisticsToMonitor(stats.getStats());
   }
 
+  /**
+   * @deprecated no replacement
+   */
+  @Deprecated
   private ProcessStats fetchProcessStats() {
     return system.getStatSampler().getProcessStats();
   }
diff --git a/geode-core/src/main/resources/META-INF/services/org.apache.geode.internal.statistics.OsStatisticsProvider b/geode-core/src/main/resources/META-INF/services/org.apache.geode.internal.statistics.OsStatisticsProvider
index 64fc550..fa4cef5 100644
--- a/geode-core/src/main/resources/META-INF/services/org.apache.geode.internal.statistics.OsStatisticsProvider
+++ b/geode-core/src/main/resources/META-INF/services/org.apache.geode.internal.statistics.OsStatisticsProvider
@@ -13,5 +13,5 @@
 # the License.
 #
 
-org.apache.geode.internal.statistics.legacy.LegacyOsStatisticsProvider
+org.apache.geode.internal.statistics.legacy.LinuxOsStatisticsProvider
 org.apache.geode.internal.statistics.oshi.OshiStatisticsProvider
diff --git a/geode-core/src/test/java/org/apache/geode/internal/statistics/legacy/LegacyOsStatisticsProviderTest.java b/geode-core/src/test/java/org/apache/geode/internal/statistics/legacy/LegacyOsStatisticsProviderTest.java
index 90e2037..523fd40 100644
--- a/geode-core/src/test/java/org/apache/geode/internal/statistics/legacy/LegacyOsStatisticsProviderTest.java
+++ b/geode-core/src/test/java/org/apache/geode/internal/statistics/legacy/LegacyOsStatisticsProviderTest.java
@@ -24,7 +24,7 @@ public class LegacyOsStatisticsProviderTest {
 
   @Test
   public void osSystemSupportedOnLinuxOnly() {
-    Assertions.assertThat(LegacyOsStatisticsProvider.build().osStatsSupported()).isEqualTo(
+    Assertions.assertThat(LinuxOsStatisticsProvider.build().osStatsSupported()).isEqualTo(
         SystemUtils.isLinux());
   }
 

[geode] 06/16: spotless

Posted by jb...@apache.org.
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 a8dca191b4f3eed02714a39efe976914c8ffb079
Author: Jacob Barrett <jb...@pivotal.io>
AuthorDate: Sat Jun 12 20:14:39 2021 -0700

    spotless
---
 .../platform/OshiStatisticsBenchmark.java          |  46 +++----
 .../internal/statistics/GemFireStatSampler.java    |  56 ++++----
 .../statistics/oshi/NetworkInterfaceStats.java     |   2 +-
 .../statistics/oshi/OperatingSystemStats.java      |   2 +-
 .../statistics/oshi/OshiStatisticsProvider.java    |   2 +-
 .../oshi/OshiStatisticsProviderImpl.java           |  89 +++++++-----
 .../internal/statistics/oshi/ProcessStats.java     |   2 +-
 .../internal/statistics/oshi/ProcessorStats.java   |   2 +-
 .../statistics/oshi/OshiStatisticsTest.java        | 151 ++++++++++-----------
 9 files changed, 182 insertions(+), 170 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 6bfb079..ddf9fcf 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
@@ -20,12 +20,10 @@ import java.util.function.DoubleSupplier;
 import java.util.function.IntSupplier;
 import java.util.function.LongSupplier;
 
-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.Scope;
-import org.openjdk.jmh.annotations.Setup;
 import org.openjdk.jmh.annotations.State;
 import oshi.SystemInfo;
 
@@ -41,25 +39,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
@@ -279,7 +277,7 @@ public class OshiStatisticsBenchmark {
 
     @Override
     public IntSupplier setIntSupplier(final StatisticDescriptor descriptor,
-                                      final IntSupplier supplier) {
+        final IntSupplier supplier) {
       return null;
     }
 
@@ -295,7 +293,7 @@ public class OshiStatisticsBenchmark {
 
     @Override
     public LongSupplier setLongSupplier(final StatisticDescriptor descriptor,
-                                        final LongSupplier supplier) {
+        final LongSupplier supplier) {
       return null;
     }
 
@@ -311,7 +309,7 @@ public class OshiStatisticsBenchmark {
 
     @Override
     public DoubleSupplier setDoubleSupplier(final StatisticDescriptor descriptor,
-                                            final DoubleSupplier supplier) {
+        final DoubleSupplier supplier) {
       return null;
     }
   }
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 a6d8e97..2c3c922 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
@@ -69,7 +69,7 @@ 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) {
@@ -282,17 +282,17 @@ public class GemFireStatSampler extends HostStatSampler {
     try {
       oshiStatisticsProvider.init(getOsStatisticsFactory(), pid);
     } catch (OshiStatisticsProviderException e) {
-      logger.error(LogMarker.STATISTICS_MARKER,"Failed to initialize OS statistics.", 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);
+    // 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);
 
   }
 
@@ -302,29 +302,29 @@ public class GemFireStatSampler extends HostStatSampler {
       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);
-//      }
-//    }
+    // 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() {
     oshiStatisticsProvider.destroy();
-//    if (osStatisticsProvider.osStatsSupported()) {
-//      if (!osStatsDisabled()) {
-//        if (processStats != null) {
-//          processStats.close();
-//        }
-//        osStatisticsProvider.closeOSStats();
-//      }
-//    }
+    // 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/oshi/NetworkInterfaceStats.java b/geode-core/src/main/java/org/apache/geode/internal/statistics/oshi/NetworkInterfaceStats.java
index 95d4528..dd1bd75 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/statistics/oshi/NetworkInterfaceStats.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/statistics/oshi/NetworkInterfaceStats.java
@@ -42,7 +42,7 @@ public class NetworkInterfaceStats {
     final StatisticsTypeFactory f = StatisticsTypeFactoryImpl.singleton();
 
     statisticsType = f.createType("NetworkInterfaceStats", "Stats for a network interface.",
-        new StatisticDescriptor[]{
+        new StatisticDescriptor[] {
             f.createLongGauge("mtu",
                 "The interface Maximum Transmission Unit (MTU).",
                 "bytes"),
diff --git a/geode-core/src/main/java/org/apache/geode/internal/statistics/oshi/OperatingSystemStats.java b/geode-core/src/main/java/org/apache/geode/internal/statistics/oshi/OperatingSystemStats.java
index 1d4bba0..85b386f 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/statistics/oshi/OperatingSystemStats.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/statistics/oshi/OperatingSystemStats.java
@@ -264,7 +264,7 @@ public class OperatingSystemStats {
             f.createLongGauge("maxFileDescriptors",
                 "The maximum number of open file descriptors.",
                 "files"),
-    });
+        });
 
     processCount = statisticsType.nameToId("processCount");
     threadCount = statisticsType.nameToId("threadCount");
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 d60be5c..b1ec885 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
@@ -6,7 +6,7 @@ import org.apache.geode.internal.statistics.platform.OsStatisticsFactory;
 
 public interface OshiStatisticsProvider {
   void init(final @NotNull OsStatisticsFactory osStatisticsFactory,
-            final long pid) throws OshiStatisticsProviderException;
+      final long pid) throws OshiStatisticsProviderException;
 
   void sample();
 
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
index 44fc546..b79a038 100644
--- 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
@@ -27,7 +27,7 @@ public class OshiStatisticsProviderImpl implements OshiStatisticsProvider {
   private static final Logger log = LogService.getLogger();
 
   final SystemInfo systemInfo = new SystemInfo();
-  
+
   private int processId;
   private CentralProcessor processor;
   private OperatingSystem operatingSystem;
@@ -45,7 +45,7 @@ public class OshiStatisticsProviderImpl implements OshiStatisticsProvider {
 
   @Override
   public void init(final @NotNull OsStatisticsFactory osStatisticsFactory,
-                   final long id) throws OshiStatisticsProviderException {
+      final long id) throws OshiStatisticsProviderException {
 
     operatingSystem = systemInfo.getOperatingSystem();
     processId = operatingSystem.getProcessId();
@@ -60,7 +60,7 @@ public class OshiStatisticsProviderImpl implements OshiStatisticsProvider {
     final String systemIdentity = operatingSystem.toString();
     systemCpuLoadTicks = new long[TickType.values().length];
     systemStats = osStatisticsFactory.createOsStatistics(OperatingSystemStats.getType(),
-         systemIdentity, id, 0);
+        systemIdentity, id, 0);
 
 
     final List<LogicalProcessor> logicalProcessors = processor.getLogicalProcessors();
@@ -78,8 +78,9 @@ public class OshiStatisticsProviderImpl implements OshiStatisticsProvider {
     for (int i = 0, size = networkIFs.size(); i < size; i++) {
       final NetworkIF networkIF = networkIFs.get(i);
       log.info("Creating network interfaces stats for {}", networkIF.getDisplayName());
-      networkInterfaceStats[i] = osStatisticsFactory.createOsStatistics(NetworkInterfaceStats.getType(),
-          networkIF.getDisplayName(), id, 0);
+      networkInterfaceStats[i] =
+          osStatisticsFactory.createOsStatistics(NetworkInterfaceStats.getType(),
+              networkIF.getDisplayName(), id, 0);
     }
   }
 
@@ -92,8 +93,7 @@ public class OshiStatisticsProviderImpl implements OshiStatisticsProvider {
   }
 
   @Override
-  public void destroy() {
-  }
+  public void destroy() {}
 
   private void sampleProcess() {
     final OSProcess process = operatingSystem.getProcess(processId);
@@ -123,8 +123,10 @@ public class OshiStatisticsProviderImpl implements OshiStatisticsProvider {
     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.physicalProcessorCount,
+        processor.getPhysicalProcessorCount());
+    systemStats.setLong(OperatingSystemStats.logicalProcessorCount,
+        processor.getLogicalProcessorCount());
     systemStats.setLong(OperatingSystemStats.maxFreq, processor.getMaxFreq());
 
     final double[] systemLoadAverage = processor.getSystemLoadAverage(3);
@@ -132,7 +134,8 @@ public class OshiStatisticsProviderImpl implements OshiStatisticsProvider {
     systemStats.setDouble(OperatingSystemStats.systemLoadAverage5, systemLoadAverage[1]);
     systemStats.setDouble(OperatingSystemStats.systemLoadAverage15, systemLoadAverage[2]);
 
-    final double systemCpuLoadBetweenTicks = processor.getSystemCpuLoadBetweenTicks(systemCpuLoadTicks);
+    final double systemCpuLoadBetweenTicks =
+        processor.getSystemCpuLoadBetweenTicks(systemCpuLoadTicks);
     systemStats.setDouble(OperatingSystemStats.systemCpuLoad, systemCpuLoadBetweenTicks);
 
     systemCpuLoadTicks = processor.getSystemCpuLoadTicks();
@@ -168,50 +171,71 @@ public class OshiStatisticsProviderImpl implements OshiStatisticsProvider {
 
     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.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.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.udpv4DatagramsReceived,
+        udPv4Stats.getDatagramsReceived());
     systemStats.setLong(OperatingSystemStats.udpv4DatagramsNoPort, udPv4Stats.getDatagramsNoPort());
-    systemStats.setLong(OperatingSystemStats.udpv4DatagramsReceivedErrors, udPv4Stats.getDatagramsReceivedErrors());
+    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.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.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.udpv6DatagramsReceived,
+        udPv6Stats.getDatagramsReceived());
     systemStats.setLong(OperatingSystemStats.udpv6DatagramsNoPort, udPv6Stats.getDatagramsNoPort());
-    systemStats.setLong(OperatingSystemStats.udpv6DatagramsReceivedErrors, udPv6Stats.getDatagramsReceivedErrors());
+    systemStats.setLong(OperatingSystemStats.udpv6DatagramsReceivedErrors,
+        udPv6Stats.getDatagramsReceivedErrors());
 
     final FileSystem fileSystem = operatingSystem.getFileSystem();
-    systemStats.setLong(OperatingSystemStats.openFileDescriptors, fileSystem.getOpenFileDescriptors());
-    systemStats.setLong(OperatingSystemStats.openFileDescriptors, fileSystem.getMaxFileDescriptors());
+    systemStats.setLong(OperatingSystemStats.openFileDescriptors,
+        fileSystem.getOpenFileDescriptors());
+    systemStats.setLong(OperatingSystemStats.openFileDescriptors,
+        fileSystem.getMaxFileDescriptors());
 
   }
 
   private void sampleProcessors() {
     final long[] currentFreq = processor.getCurrentFreq();
-    final double[] processorCpuLoad = processor.getProcessorCpuLoadBetweenTicks(processorCpuLoadTicks);
+    final double[] processorCpuLoad =
+        processor.getProcessorCpuLoadBetweenTicks(processorCpuLoadTicks);
     processorCpuLoadTicks = processor.getProcessorCpuLoadTicks();
 
     for (int i = 0; i < processorStats.length; i++) {
@@ -250,7 +274,8 @@ public class OshiStatisticsProviderImpl implements OshiStatisticsProvider {
       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.packetsReceived,
+          networkIF.getPacketsRecv());
       networkInterfaceStat.setLong(NetworkInterfaceStats.packetsSent, networkIF.getPacketsSent());
       networkInterfaceStat.setLong(NetworkInterfaceStats.inErrors, networkIF.getInErrors());
       networkInterfaceStat.setLong(NetworkInterfaceStats.outErrors, networkIF.getOutErrors());
diff --git a/geode-core/src/main/java/org/apache/geode/internal/statistics/oshi/ProcessStats.java b/geode-core/src/main/java/org/apache/geode/internal/statistics/oshi/ProcessStats.java
index 1e6199e..2fddd0c 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/statistics/oshi/ProcessStats.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/statistics/oshi/ProcessStats.java
@@ -45,7 +45,7 @@ public class ProcessStats {
     final StatisticsTypeFactory f = StatisticsTypeFactoryImpl.singleton();
 
     statisticsType = f.createType("ProcessStats", "Statistics on a process.",
-        new StatisticDescriptor[]{
+        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.",
                 "bytes"),
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
index 1303901..be69378 100644
--- 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
@@ -73,7 +73,7 @@ public class ProcessorStats {
             f.createDoubleGauge("processorCpuLoad",
                 "CPU usage",
                 "percent"),
-    });
+        });
 
     currentFreq = statisticsType.nameToId("currentFreq");
     processorCpuLoadTicksUSER = statisticsType.nameToId("processorCpuLoadTicksUSER");
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
index 2837d1c..e87ae72 100644
--- 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
@@ -15,89 +15,78 @@
 
 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));
-//  }
+  // 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
+}

[geode] 12/16: Make service loadable.

Posted by jb...@apache.org.
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 dd3d111f9ed72d0dc68ee97ac65595f19eb7dde5
Author: Jacob Barrett <jb...@pivotal.io>
AuthorDate: Wed Nov 17 11:19:45 2021 -0800

    Make service loadable.
---
 .../internal/statistics/OsStatisticsBenchmark.java |  3 +-
 .../internal/statistics/GemFireStatSampler.java    | 62 +++++++++++++++++-----
 .../internal/statistics/StatisticsRegistry.java    |  3 +-
 .../legacy/LegacyOsStatisticsProvider.java         |  2 -
 ....geode.internal.statistics.OsStatisticsProvider | 17 ++++++
 .../statistics/StatisticsRegistryTest.java         |  3 +-
 6 files changed, 72 insertions(+), 18 deletions(-)

diff --git a/geode-core/src/jmh/java/org/apache/geode/internal/statistics/OsStatisticsBenchmark.java b/geode-core/src/jmh/java/org/apache/geode/internal/statistics/OsStatisticsBenchmark.java
index 9ec47f1..cf91437 100644
--- a/geode-core/src/jmh/java/org/apache/geode/internal/statistics/OsStatisticsBenchmark.java
+++ b/geode-core/src/jmh/java/org/apache/geode/internal/statistics/OsStatisticsBenchmark.java
@@ -330,7 +330,8 @@ public class OsStatisticsBenchmark {
 
   public static class NoopStatisticsProvider implements OsStatisticsFactory {
     @Override
-    public Statistics createOsStatistics(final StatisticsType type, final String textId, final long numericId) {
+    public Statistics createOsStatistics(final StatisticsType type, final String textId,
+        final long numericId) {
       return new NoopStatistics();
     }
   }
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 6ceb510..2a1c79d 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
@@ -20,10 +20,12 @@ import java.util.HashMap;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
+import java.util.ServiceLoader;
 import java.util.Set;
 import java.util.concurrent.ConcurrentHashMap;
 
 import org.apache.logging.log4j.Logger;
+import org.jetbrains.annotations.NotNull;
 
 import org.apache.geode.CancelCriterion;
 import org.apache.geode.Statistics;
@@ -35,7 +37,6 @@ 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.platform.ProcessStats;
 import org.apache.geode.logging.internal.log4j.api.LogService;
 import org.apache.geode.logging.internal.spi.LogFile;
@@ -66,13 +67,14 @@ public class GemFireStatSampler extends HostStatSampler {
   private int nextListenerId = 1;
   private ProcessStats processStats;
 
-  private OsStatisticsProvider osStatisticsProvider = new OshiStatisticsProvider();
+  private final OsStatisticsProvider[] osStatisticsProviders;
 
   public GemFireStatSampler(InternalDistributedSystem internalDistributedSystem) {
     this(internalDistributedSystem, null);
   }
 
-  public GemFireStatSampler(InternalDistributedSystem internalDistributedSystem, LogFile logFile) {
+  public GemFireStatSampler(final @NotNull InternalDistributedSystem internalDistributedSystem,
+      final @NotNull LogFile logFile) {
     this(internalDistributedSystem.getCancelCriterion(),
         new StatSamplerStats(internalDistributedSystem,
             internalDistributedSystem.getStatisticsManager().getPid()),
@@ -84,18 +86,41 @@ public class GemFireStatSampler extends HostStatSampler {
   }
 
   @VisibleForTesting
-  public GemFireStatSampler(CancelCriterion cancelCriterion,
-      StatSamplerStats statSamplerStats,
-      LogFile logFile,
-      StatisticsConfig statisticsConfig,
-      StatisticsManager statisticsManager,
-      DistributionManager distributionManager,
+  public GemFireStatSampler(final @NotNull CancelCriterion cancelCriterion,
+      final @NotNull StatSamplerStats statSamplerStats,
+      final @NotNull LogFile logFile,
+      final @NotNull StatisticsConfig statisticsConfig,
+      final @NotNull StatisticsManager statisticsManager,
+      final @NotNull DistributionManager distributionManager,
       long systemId) {
+    this(cancelCriterion, statSamplerStats, logFile, statisticsConfig, statisticsManager,
+        distributionManager, systemId, loadOsStatisticsProviders());
+  }
+
+  private static OsStatisticsProvider[] loadOsStatisticsProviders() {
+    final ServiceLoader<OsStatisticsProvider> loader =
+        ServiceLoader.load(OsStatisticsProvider.class);
+    final List<OsStatisticsProvider> osStatisticsProviders = new ArrayList<>();
+    for (OsStatisticsProvider osStatisticsProvider : loader) {
+      osStatisticsProviders.add(osStatisticsProvider);
+    }
+    return osStatisticsProviders.toArray(new OsStatisticsProvider[0]);
+  }
+
+  private GemFireStatSampler(final @NotNull CancelCriterion cancelCriterion,
+      final @NotNull StatSamplerStats statSamplerStats,
+      final @NotNull LogFile logFile,
+      final @NotNull StatisticsConfig statisticsConfig,
+      final @NotNull StatisticsManager statisticsManager,
+      final @NotNull DistributionManager distributionManager,
+      final long systemId,
+      final @NotNull OsStatisticsProvider[] osStatisticsProviders) {
     super(cancelCriterion, statSamplerStats, logFile);
     this.systemId = systemId;
     this.statisticsConfig = statisticsConfig;
     this.statisticsManager = statisticsManager;
     this.distributionManager = distributionManager;
+    this.osStatisticsProviders = osStatisticsProviders;
   }
 
   /**
@@ -274,13 +299,20 @@ public class GemFireStatSampler extends HostStatSampler {
       return;
     }
 
+    if (osStatisticsProviders.length == 0) {
+      logger.warn(LogMarker.STATISTICS_MARKER, "No OS statistics providers available.");
+    }
+
+    final OsStatisticsFactory osStatisticsFactory = getOsStatisticsFactory();
     try {
-      osStatisticsProvider.init(getOsStatisticsFactory(), pid);
+      for (final OsStatisticsProvider osStatisticsProvider : osStatisticsProviders) {
+        osStatisticsProvider.init(osStatisticsFactory, pid);
+      }
     } catch (OsStatisticsProviderException e) {
       logger.error(LogMarker.STATISTICS_MARKER, "Failed to initialize OS statistics.", e);
     }
 
-    processStats = null; // osStatisticsProvider.newProcessStats(stats);
+    processStats = null; // TODO jbarrett osStatisticsProvider.newProcessStats(stats);
   }
 
   @Override
@@ -288,12 +320,16 @@ public class GemFireStatSampler extends HostStatSampler {
     if (prepareOnly || osStatsDisabled() || stopRequested()) {
       return;
     }
-    osStatisticsProvider.sample();
+    for (final OsStatisticsProvider osStatisticsProvider : osStatisticsProviders) {
+      osStatisticsProvider.sample();
+    }
   }
 
   @Override
   protected void closeProcessStats() {
-    osStatisticsProvider.destroy();
+    for (final OsStatisticsProvider osStatisticsProvider : osStatisticsProviders) {
+      osStatisticsProvider.destroy();
+    }
   }
 
   private void checkLocalListeners() {
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 04db506..53f9c26 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,7 +51,8 @@ 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, StatisticsManager manager);
+    Statistics create(StatisticsType type, String textId, long numericId, long uniqueId,
+        StatisticsManager manager);
   }
 
   /**
diff --git a/geode-core/src/main/java/org/apache/geode/internal/statistics/legacy/LegacyOsStatisticsProvider.java b/geode-core/src/main/java/org/apache/geode/internal/statistics/legacy/LegacyOsStatisticsProvider.java
index 1578178..a40fadf 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/statistics/legacy/LegacyOsStatisticsProvider.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/statistics/legacy/LegacyOsStatisticsProvider.java
@@ -33,8 +33,6 @@ import org.apache.geode.internal.statistics.platform.ProcessStats;
  * Only Linux OS is currently allowed.
  */
 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;
diff --git a/geode-core/src/main/resources/META-INF/services/org.apache.geode.internal.statistics.OsStatisticsProvider b/geode-core/src/main/resources/META-INF/services/org.apache.geode.internal.statistics.OsStatisticsProvider
new file mode 100644
index 0000000..64fc550
--- /dev/null
+++ b/geode-core/src/main/resources/META-INF/services/org.apache.geode.internal.statistics.OsStatisticsProvider
@@ -0,0 +1,17 @@
+#
+# 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.
+#
+
+org.apache.geode.internal.statistics.legacy.LegacyOsStatisticsProvider
+org.apache.geode.internal.statistics.oshi.OshiStatisticsProvider
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 0f8932a..9370966 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
@@ -314,7 +314,8 @@ public class StatisticsRegistryTest {
     when(osStatisticsFactory.create(any(), any(), anyLong(), anyLong(), any()))
         .thenReturn(statisticsCreatedByFactory);
 
-    Statistics result = registry.createOsStatistics(type, STATISTICS_TEXT_ID, STATISTICS_NUMERIC_ID);
+    Statistics result =
+        registry.createOsStatistics(type, STATISTICS_TEXT_ID, STATISTICS_NUMERIC_ID);
 
     assertThat(result)
         .isSameAs(statisticsCreatedByFactory);

[geode] 04/16: adds network intraces.

Posted by jb...@apache.org.
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 c46ec0d862c984508c72eee6f9e3c16c51f175ea
Author: Jacob Barrett <jb...@pivotal.io>
AuthorDate: Sat Jun 12 09:32:04 2021 -0700

    adds network intraces.
---
 .../statistics/oshi/NetworkInterfaceStats.java     | 98 ++++++++++++++++++++++
 .../oshi/OshiStatisticsProviderImpl.java           | 48 +++++++++--
 2 files changed, 141 insertions(+), 5 deletions(-)

diff --git a/geode-core/src/main/java/org/apache/geode/internal/statistics/oshi/NetworkInterfaceStats.java b/geode-core/src/main/java/org/apache/geode/internal/statistics/oshi/NetworkInterfaceStats.java
new file mode 100644
index 0000000..95d4528
--- /dev/null
+++ b/geode-core/src/main/java/org/apache/geode/internal/statistics/oshi/NetworkInterfaceStats.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 NetworkInterfaceStats {
+  static final int mtu;
+  static final int bytesReceived;
+  static final int bytesSent;
+  static final int packetsReceived;
+  static final int packetsSent;
+  static final int inErrors;
+  static final int outErrors;
+  static final int inDrops;
+  static final int collisions;
+  static final int speed;
+
+  @Immutable
+  private static final StatisticsType statisticsType;
+
+  static {
+    final StatisticsTypeFactory f = StatisticsTypeFactoryImpl.singleton();
+
+    statisticsType = f.createType("NetworkInterfaceStats", "Stats for a network interface.",
+        new StatisticDescriptor[]{
+            f.createLongGauge("mtu",
+                "The interface Maximum Transmission Unit (MTU).",
+                "bytes"),
+            f.createLongCounter("bytesReceived",
+                "The bytes received.",
+                "bytes"),
+            f.createLongCounter("bytesSent",
+                "The bytes sent.",
+                "bytes"),
+            f.createLongCounter("packetsReceived",
+                "The packets received",
+                "packets"),
+            f.createLongCounter("packetsSent",
+                "The packets sent.",
+                "packets"),
+            f.createLongCounter("inErrors",
+                "Input errors",
+                "packets"),
+            f.createLongCounter("outErrors",
+                "Output errors",
+                "packets"),
+            f.createLongCounter("inDrops",
+                "Incoming/Received dropped packets.",
+                "packets"),
+            f.createLongCounter("collisions",
+                "Packet collisions.",
+                "packets"),
+            f.createLongGauge("speed",
+                "The speed of the network interface in bits per second.",
+                "bits/s"),
+        });
+
+    mtu = statisticsType.nameToId("mtu");
+    bytesReceived = statisticsType.nameToId("bytesReceived");
+    bytesSent = statisticsType.nameToId("bytesSent");
+    packetsReceived = statisticsType.nameToId("packetsReceived");
+    packetsSent = statisticsType.nameToId("packetsSent");
+    inErrors = statisticsType.nameToId("inErrors");
+    outErrors = statisticsType.nameToId("outErrors");
+    inDrops = statisticsType.nameToId("inDrops");
+    collisions = statisticsType.nameToId("collisions");
+    speed = statisticsType.nameToId("speed");
+  }
+
+  private NetworkInterfaceStats() {
+    // no instances allowed
+  }
+
+  public static @NotNull StatisticsType getType() {
+    return statisticsType;
+  }
+
+}
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
index dcaf5b4..ce02b38 100644
--- 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
@@ -2,6 +2,7 @@ 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;
@@ -9,14 +10,17 @@ 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.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 = new SystemInfo();
   
@@ -24,13 +28,16 @@ public class OshiStatisticsProviderImpl implements OshiStatisticsProvider {
   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 OSProcess process;
+  private Statistics[] networkInterfaceStats;
 
   @Override
   public void init(final @NotNull OsStatisticsFactory osStatisticsFactory,
@@ -55,12 +62,21 @@ public class OshiStatisticsProviderImpl implements OshiStatisticsProvider {
     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++) {
+    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);
+      log.info("Creating network interfaces stats for {}", networkIF.getDisplayName());
+      networkInterfaceStats[i] = osStatisticsFactory.createOsStatistics(NetworkInterfaceStats.getType(),
+          networkIF.getDisplayName(), id, 0);
+    }
   }
 
   @Override
@@ -68,13 +84,14 @@ public class OshiStatisticsProviderImpl implements OshiStatisticsProvider {
     sampleProcess();
     sampleSystem();
     sampleProcessors();
+    sampleNetworkInterfaces();
   }
 
   @Override
   public void destroy() {
   }
 
-  public void sampleProcess() {
+  private void sampleProcess() {
     final OSProcess process = operatingSystem.getProcess(processId);
 
     final double processCpuLoadBetweenTicks = process.getProcessCpuLoadBetweenTicks(this.process);
@@ -95,7 +112,7 @@ public class OshiStatisticsProviderImpl implements OshiStatisticsProvider {
     processStats.setLong(ProcessStats.contextSwitches, process.getContextSwitches());
   }
 
-  public void sampleSystem() {
+  private void sampleSystem() {
     systemStats.setLong(OperatingSystemStats.processCount, operatingSystem.getProcessCount());
     systemStats.setLong(OperatingSystemStats.threadCount, operatingSystem.getThreadCount());
 
@@ -175,5 +192,26 @@ public class OshiStatisticsProviderImpl implements OshiStatisticsProvider {
           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());
+    }
+
+  }
 }

[geode] 03/16: Refactor interface with more stats.

Posted by jb...@apache.org.
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

[geode] 13/16: Fix legacy process stats sample.

Posted by jb...@apache.org.
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 2a3b2af8a148fe289679f6e36e91a7701acb227e
Author: Jacob Barrett <jb...@pivotal.io>
AuthorDate: Wed Nov 17 11:23:02 2021 -0800

    Fix legacy process stats sample.
---
 .../geode/internal/statistics/legacy/LegacyOsStatisticsProvider.java  | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)

diff --git a/geode-core/src/main/java/org/apache/geode/internal/statistics/legacy/LegacyOsStatisticsProvider.java b/geode-core/src/main/java/org/apache/geode/internal/statistics/legacy/LegacyOsStatisticsProvider.java
index a40fadf..349b7ee 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/statistics/legacy/LegacyOsStatisticsProvider.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/statistics/legacy/LegacyOsStatisticsProvider.java
@@ -120,8 +120,8 @@ public class LegacyOsStatisticsProvider implements OsStatisticsProvider {
     refreshSystem(systemStatistics);
   }
 
-  void sampleProcess() {
-    refreshSystem(processStatistics);
+  private void sampleProcess() {
+    refreshProcess(processStatistics);
   }
 
   @Override

[geode] 10/16: Delete comments

Posted by jb...@apache.org.
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 155ef609c831d10e908835b4f5f24204c14439b4
Author: Jacob Barrett <jb...@pivotal.io>
AuthorDate: Tue Nov 16 12:40:19 2021 -0800

    Delete comments
---
 .../java/org/apache/geode/internal/statistics/GemFireStatSampler.java    | 1 -
 1 file changed, 1 deletion(-)

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 2c3c922..b39ff2f 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
@@ -241,7 +241,6 @@ public class GemFireStatSampler extends HostStatSampler {
               statListener.checkForChange(statListenerMessage);
             } else {
               // its stale; indicate this with a negative listener id
-              // fix for bug 29405
               statListenerMessage.addChange(-statListener.getListenerId(), 0);
             }
           }

[geode] 16/16: Initial module.

Posted by jb...@apache.org.
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 fb2b8cf261ed9509cbba81ee70f5195fe8f0a8d9
Author: Jacob Barrett <jb...@pivotal.io>
AuthorDate: Fri Nov 19 11:42:30 2021 -0800

    Initial module.
---
 .../internal/InternalDistributedSystem.java        |  3 +--
 ....geode.internal.statistics.OsStatisticsProvider |  1 -
 geode-stats-legacy/build.gradle                    | 31 ++++++++++++++++++++++
 .../legacy/LinuxOsStatisticsProvider.java          |  0
 .../statistics/legacy/LinuxProcFsStatistics.java   |  0
 .../statistics/legacy/LinuxProcessStats.java       |  0
 .../statistics/legacy/LinuxSystemStats.java        |  0
 ....geode.internal.statistics.OsStatisticsProvider |  1 -
 settings.gradle                                    |  1 +
 9 files changed, 33 insertions(+), 4 deletions(-)

diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/InternalDistributedSystem.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/InternalDistributedSystem.java
index d051dc0..84e2225 100644
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/InternalDistributedSystem.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/InternalDistributedSystem.java
@@ -105,7 +105,6 @@ import org.apache.geode.internal.statistics.StatisticsConfig;
 import org.apache.geode.internal.statistics.StatisticsManager;
 import org.apache.geode.internal.statistics.StatisticsManagerFactory;
 import org.apache.geode.internal.statistics.StatisticsRegistry;
-import org.apache.geode.internal.statistics.legacy.LinuxProcFsStatistics;
 import org.apache.geode.internal.tcp.ConnectionTable;
 import org.apache.geode.logging.internal.LoggingSession;
 import org.apache.geode.logging.internal.NullLoggingSession;
@@ -737,7 +736,7 @@ public class InternalDistributedSystem extends DistributedSystem
         // This calculation is not exact, but seems fairly close. So far we have
         // not loaded much into the heap and the current RSS usage is already
         // included the available memory calculation.
-        long avail = LinuxProcFsStatistics.getAvailableMemory(logger);
+        long avail = 0; //LinuxProcFsStatistics.getAvailableMemory(logger);
         long size = offHeapMemorySize + Runtime.getRuntime().totalMemory();
         lockMemory(avail, size);
       }
diff --git a/geode-core/src/main/resources/META-INF/services/org.apache.geode.internal.statistics.OsStatisticsProvider b/geode-core/src/main/resources/META-INF/services/org.apache.geode.internal.statistics.OsStatisticsProvider
index fa4cef5..4b59cde 100644
--- a/geode-core/src/main/resources/META-INF/services/org.apache.geode.internal.statistics.OsStatisticsProvider
+++ b/geode-core/src/main/resources/META-INF/services/org.apache.geode.internal.statistics.OsStatisticsProvider
@@ -13,5 +13,4 @@
 # the License.
 #
 
-org.apache.geode.internal.statistics.legacy.LinuxOsStatisticsProvider
 org.apache.geode.internal.statistics.oshi.OshiStatisticsProvider
diff --git a/geode-stats-legacy/build.gradle b/geode-stats-legacy/build.gradle
new file mode 100644
index 0000000..f8979ad
--- /dev/null
+++ b/geode-stats-legacy/build.gradle
@@ -0,0 +1,31 @@
+/*
+ * 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.
+ */
+
+
+apply from: "${rootDir}/${scriptDir}/standard-subproject-configuration.gradle"
+
+
+apply from: "${project.projectDir}/../gradle/publish-java.gradle"
+apply from: "${project.projectDir}/../gradle/warnings.gradle"
+
+dependencies {
+  api(platform(project(':boms:geode-all-bom')))
+
+  implementation(project(':geode-core'))
+  implementation(project(':geode-logging'))
+
+  compileOnly('org.jetbrains:annotations')
+
+}
diff --git a/geode-core/src/main/java/org/apache/geode/internal/statistics/legacy/LinuxOsStatisticsProvider.java b/geode-stats-legacy/src/main/java/org/apache/geode/internal/statistics/legacy/LinuxOsStatisticsProvider.java
similarity index 100%
rename from geode-core/src/main/java/org/apache/geode/internal/statistics/legacy/LinuxOsStatisticsProvider.java
rename to geode-stats-legacy/src/main/java/org/apache/geode/internal/statistics/legacy/LinuxOsStatisticsProvider.java
diff --git a/geode-core/src/main/java/org/apache/geode/internal/statistics/legacy/LinuxProcFsStatistics.java b/geode-stats-legacy/src/main/java/org/apache/geode/internal/statistics/legacy/LinuxProcFsStatistics.java
similarity index 100%
rename from geode-core/src/main/java/org/apache/geode/internal/statistics/legacy/LinuxProcFsStatistics.java
rename to geode-stats-legacy/src/main/java/org/apache/geode/internal/statistics/legacy/LinuxProcFsStatistics.java
diff --git a/geode-core/src/main/java/org/apache/geode/internal/statistics/legacy/LinuxProcessStats.java b/geode-stats-legacy/src/main/java/org/apache/geode/internal/statistics/legacy/LinuxProcessStats.java
similarity index 100%
rename from geode-core/src/main/java/org/apache/geode/internal/statistics/legacy/LinuxProcessStats.java
rename to geode-stats-legacy/src/main/java/org/apache/geode/internal/statistics/legacy/LinuxProcessStats.java
diff --git a/geode-core/src/main/java/org/apache/geode/internal/statistics/legacy/LinuxSystemStats.java b/geode-stats-legacy/src/main/java/org/apache/geode/internal/statistics/legacy/LinuxSystemStats.java
similarity index 100%
rename from geode-core/src/main/java/org/apache/geode/internal/statistics/legacy/LinuxSystemStats.java
rename to geode-stats-legacy/src/main/java/org/apache/geode/internal/statistics/legacy/LinuxSystemStats.java
diff --git a/geode-core/src/main/resources/META-INF/services/org.apache.geode.internal.statistics.OsStatisticsProvider b/geode-stats-legacy/src/main/resources/META-INF/services/org.apache.geode.internal.statistics.OsStatisticsProvider
similarity index 92%
copy from geode-core/src/main/resources/META-INF/services/org.apache.geode.internal.statistics.OsStatisticsProvider
copy to geode-stats-legacy/src/main/resources/META-INF/services/org.apache.geode.internal.statistics.OsStatisticsProvider
index fa4cef5..a162efb 100644
--- a/geode-core/src/main/resources/META-INF/services/org.apache.geode.internal.statistics.OsStatisticsProvider
+++ b/geode-stats-legacy/src/main/resources/META-INF/services/org.apache.geode.internal.statistics.OsStatisticsProvider
@@ -14,4 +14,3 @@
 #
 
 org.apache.geode.internal.statistics.legacy.LinuxOsStatisticsProvider
-org.apache.geode.internal.statistics.oshi.OshiStatisticsProvider
diff --git a/settings.gradle b/settings.gradle
index 817d8a9..4a11bd0 100644
--- a/settings.gradle
+++ b/settings.gradle
@@ -52,6 +52,7 @@ include 'geode-for-redis'
 include 'geode-memcached'
 include 'geode-connectors'
 include 'geode-http-service'
+include 'geode-stats-legacy'
 include 'extensions:geode-modules'
 include 'extensions:geode-modules-test'
 include 'extensions:geode-modules-tomcat7'

[geode] 02/16: Basic OSHI based stats.

Posted by jb...@apache.org.
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 2a8bdb42d7a7713b22fd4550ff2915eee16139ab
Author: Jacob Barrett <jb...@pivotal.io>
AuthorDate: Fri Jun 11 07:28:08 2021 -0700

    Basic OSHI based stats.
---
 geode-core/build.gradle                            |   4 +-
 .../platform/OshiStatisticsBenchmark.java          | 321 +++++++++++++++++++++
 .../platform/OshiStatisticsBenchmarkTest.java      |  25 ++
 .../internal/statistics/OsStatisticsProvider.java  |  62 +++-
 .../statistics/platform/OshiProcessStats.java      |  82 ++++++
 .../statistics/platform/OshiStatistics.java        | 109 +++++++
 .../statistics/platform/OshiSystemStats.java       | 136 +++++++++
 .../internal/beans/MemberMBeanBridge.java          |   2 +-
 .../statistics/platform/OshiStatisticsTest.java    | 103 +++++++
 9 files changed, 832 insertions(+), 12 deletions(-)

diff --git a/geode-core/build.gradle b/geode-core/build.gradle
index c9c9b7b..eaa2f54 100755
--- a/geode-core/build.gradle
+++ b/geode-core/build.gradle
@@ -312,6 +312,9 @@ dependencies {
   //RMIIO is used for uploading jar files and copying them between locator an servers
   implementation('com.healthmarketscience.rmiio:rmiio')
 
+  // OSHI OS system stats
+  implementation('com.github.oshi:oshi-core:5.7.4')
+
   //Geode-common has annotations and other pieces used geode-core
   api(project(':geode-common'))
   implementation(project(':geode-logging'))
@@ -324,7 +327,6 @@ dependencies {
   //copied into it, so it is an API dependency
   api(project(':geode-management'))
 
-
   jcaImplementation(sourceSets.main.output)
 
   testImplementation(project(':geode-junit')) {
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
new file mode 100644
index 0000000..d6fbb07
--- /dev/null
+++ b/geode-core/src/jmh/java/org/apache/geode/internal/statistics/platform/OshiStatisticsBenchmark.java
@@ -0,0 +1,321 @@
+/*
+ * 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 java.util.concurrent.TimeUnit;
+import java.util.function.DoubleSupplier;
+import java.util.function.IntSupplier;
+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;
+import org.apache.geode.StatisticsType;
+import org.apache.geode.internal.statistics.SuppliableStatistics;
+
+@State(Scope.Benchmark)
+@BenchmarkMode(Mode.SampleTime)
+@OutputTimeUnit(TimeUnit.MILLISECONDS)
+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);
+  }
+
+  private static class NoopStatistics implements SuppliableStatistics {
+    @Override
+    public int updateSuppliedValues() {
+      return 0;
+    }
+
+    @Override
+    public void close() {
+
+    }
+
+    @Override
+    public int nameToId(final String name) {
+      return 0;
+    }
+
+    @Override
+    public StatisticDescriptor nameToDescriptor(final String name) {
+      return null;
+    }
+
+    @Override
+    public long getUniqueId() {
+      return 0;
+    }
+
+    @Override
+    public StatisticsType getType() {
+      return null;
+    }
+
+    @Override
+    public String getTextId() {
+      return null;
+    }
+
+    @Override
+    public long getNumericId() {
+      return 0;
+    }
+
+    @Override
+    public boolean isAtomic() {
+      return false;
+    }
+
+    @Override
+    public boolean isClosed() {
+      return false;
+    }
+
+    @Override
+    public void setInt(final int id, final int value) {
+
+    }
+
+    @Override
+    public void setInt(final String name, final int value) {
+
+    }
+
+    @Override
+    public void setInt(final StatisticDescriptor descriptor, final int value) {
+
+    }
+
+    @Override
+    public void setLong(final int id, final long value) {
+
+    }
+
+    @Override
+    public void setLong(final StatisticDescriptor descriptor, final long value) {
+
+    }
+
+    @Override
+    public void setLong(final String name, final long value) {
+
+    }
+
+    @Override
+    public void setDouble(final int id, final double value) {
+
+    }
+
+    @Override
+    public void setDouble(final StatisticDescriptor descriptor, final double value) {
+
+    }
+
+    @Override
+    public void setDouble(final String name, final double value) {
+
+    }
+
+    @Override
+    public int getInt(final int id) {
+      return 0;
+    }
+
+    @Override
+    public int getInt(final StatisticDescriptor descriptor) {
+      return 0;
+    }
+
+    @Override
+    public int getInt(final String name) {
+      return 0;
+    }
+
+    @Override
+    public long getLong(final int id) {
+      return 0;
+    }
+
+    @Override
+    public long getLong(final StatisticDescriptor descriptor) {
+      return 0;
+    }
+
+    @Override
+    public long getLong(final String name) {
+      return 0;
+    }
+
+    @Override
+    public double getDouble(final int id) {
+      return 0;
+    }
+
+    @Override
+    public double getDouble(final StatisticDescriptor descriptor) {
+      return 0;
+    }
+
+    @Override
+    public double getDouble(final String name) {
+      return 0;
+    }
+
+    @Override
+    public Number get(final StatisticDescriptor descriptor) {
+      return null;
+    }
+
+    @Override
+    public Number get(final String name) {
+      return null;
+    }
+
+    @Override
+    public long getRawBits(final StatisticDescriptor descriptor) {
+      return 0;
+    }
+
+    @Override
+    public long getRawBits(final String name) {
+      return 0;
+    }
+
+    @Override
+    public void incInt(final int id, final int delta) {
+
+    }
+
+    @Override
+    public void incInt(final StatisticDescriptor descriptor, final int delta) {
+
+    }
+
+    @Override
+    public void incInt(final String name, final int delta) {
+
+    }
+
+    @Override
+    public void incLong(final int id, final long delta) {
+
+    }
+
+    @Override
+    public void incLong(final StatisticDescriptor descriptor, final long delta) {
+
+    }
+
+    @Override
+    public void incLong(final String name, final long delta) {
+
+    }
+
+    @Override
+    public void incDouble(final int id, final double delta) {
+
+    }
+
+    @Override
+    public void incDouble(final StatisticDescriptor descriptor, final double delta) {
+
+    }
+
+    @Override
+    public void incDouble(final String name, final double delta) {
+
+    }
+
+    @Override
+    public IntSupplier setIntSupplier(final int id, final IntSupplier supplier) {
+      return null;
+    }
+
+    @Override
+    public IntSupplier setIntSupplier(final String name, final IntSupplier supplier) {
+      return null;
+    }
+
+    @Override
+    public IntSupplier setIntSupplier(final StatisticDescriptor descriptor,
+                                      final IntSupplier supplier) {
+      return null;
+    }
+
+    @Override
+    public LongSupplier setLongSupplier(final int id, final LongSupplier supplier) {
+      return null;
+    }
+
+    @Override
+    public LongSupplier setLongSupplier(final String name, final LongSupplier supplier) {
+      return null;
+    }
+
+    @Override
+    public LongSupplier setLongSupplier(final StatisticDescriptor descriptor,
+                                        final LongSupplier supplier) {
+      return null;
+    }
+
+    @Override
+    public DoubleSupplier setDoubleSupplier(final int id, final DoubleSupplier supplier) {
+      return null;
+    }
+
+    @Override
+    public DoubleSupplier setDoubleSupplier(final String name, final DoubleSupplier supplier) {
+      return null;
+    }
+
+    @Override
+    public DoubleSupplier setDoubleSupplier(final StatisticDescriptor descriptor,
+                                            final DoubleSupplier supplier) {
+      return null;
+    }
+  }
+}
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/OshiStatisticsBenchmarkTest.java
new file mode 100644
index 0000000..db0d965
--- /dev/null
+++ b/geode-core/src/jmhTest/java/org/apache/geode/internal/statistics/platform/OshiStatisticsBenchmarkTest.java
@@ -0,0 +1,25 @@
+/*
+ * 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.junit.Test;
+
+public class OshiStatisticsBenchmarkTest {
+  @Test
+  public void getsValidValues() {
+
+  }
+}
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..f2c0011 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,15 +14,24 @@
  */
 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;
 
 /**
@@ -39,7 +48,7 @@ public class OsStatisticsProvider {
   }
 
   private OsStatisticsProvider() {
-    osStatsSupported = SystemUtils.isLinux();
+    osStatsSupported = isLinux() || SystemUtils.isMacOS();
   }
 
   public static OsStatisticsProvider build() {
@@ -47,7 +56,12 @@ public class OsStatisticsProvider {
   }
 
   int initOSStats() {
-    return LinuxProcFsStatistics.init();
+    if (isLinux()) {
+      return LinuxProcFsStatistics.init();
+    } else if (isMacOS()) {
+      return OshiStatistics.init();
+    }
+    return 1;
   }
 
   void closeOSStats() {
@@ -55,7 +69,11 @@ public class OsStatisticsProvider {
   }
 
   void readyRefreshOSStats() {
-    LinuxProcFsStatistics.readyRefresh();
+    if (isLinux()) {
+      LinuxProcFsStatistics.readyRefresh();
+    } else if (isMacOS()) {
+      OshiStatistics.readyRefresh();
+    }
   }
 
   /**
@@ -64,7 +82,11 @@ public class OsStatisticsProvider {
    */
   private void refreshProcess(LocalStatisticsImpl statistics) {
     int pid = (int) statistics.getNumericId();
-    LinuxProcFsStatistics.refreshProcess(pid, statistics);
+    if (isLinux()) {
+      LinuxProcFsStatistics.refreshProcess(pid, statistics);
+    } else if (isMacOS()) {
+      OshiStatistics.refreshProcess(pid, statistics);
+    }
   }
 
   /**
@@ -72,7 +94,11 @@ public class OsStatisticsProvider {
    * machine and storing them in the instance.
    */
   private void refreshSystem(LocalStatisticsImpl statistics) {
-    LinuxProcFsStatistics.refreshSystem(statistics);
+    if (isLinux()) {
+      LinuxProcFsStatistics.refreshSystem(statistics);
+    } else if (isMacOS()) {
+      OshiStatistics.refreshSystem(statistics);
+    }
   }
 
   /**
@@ -95,12 +121,19 @@ public class OsStatisticsProvider {
    */
   Statistics newProcess(OsStatisticsFactory osStatisticsFactory, long pid, String name) {
     Statistics statistics;
-    statistics = osStatisticsFactory.createOsStatistics(LinuxProcessStats.getType(), name, pid,
+    statistics = osStatisticsFactory.createOsStatistics(getProcessStatType(), 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>.
    *
@@ -111,22 +144,31 @@ public class OsStatisticsProvider {
     if (statistics instanceof LocalStatisticsImpl) {
       refresh((LocalStatisticsImpl) statistics);
     } // otherwise its a Dummy implementation so do nothing
-    return LinuxProcessStats.createProcessStats(statistics);
+    if (isLinux()) {
+      return LinuxProcessStats.createProcessStats(statistics);
+    }
+    return OshiProcessStats.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(OsStatisticsFactory osStatisticsFactory, long id) {
-    Statistics statistics;
-    statistics = osStatisticsFactory.createOsStatistics(LinuxSystemStats.getType(),
+  void newSystem(final @NotNull OsStatisticsFactory osStatisticsFactory, long id) {
+    final Statistics statistics = osStatisticsFactory.createOsStatistics(getSystemStatType(),
         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/OshiProcessStats.java b/geode-core/src/main/java/org/apache/geode/internal/statistics/platform/OshiProcessStats.java
new file mode 100644
index 0000000..23e6541
--- /dev/null
+++ b/geode-core/src/main/java/org/apache/geode/internal/statistics/platform/OshiProcessStats.java
@@ -0,0 +1,82 @@
+/*
+ * 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 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 {
+  static final int virtualSize;
+  static final int residentSetSize;
+  static final int threadCount;
+  static final int kernelTime;
+  static final int userTime;
+
+  @Immutable
+  private static final StatisticsType statisticsType;
+
+  static {
+    final StatisticsTypeFactory f = StatisticsTypeFactoryImpl.singleton();
+
+    statisticsType = f.createType("OSProcessStats", "Statistics on a OS 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.",
+                "bytes"),
+            f.createLongGauge("residentSetSize",
+                "Gets the Resident Set Size (RSS). Used to show how much memory is allocated to that process and is in RAM. It does not include memory that is swapped out. It does include memory from shared libraries as long as the pages from those libraries are actually in memory. It does include all stack and heap memory.",
+                "bytes"),
+            f.createLongGauge("threadCount",
+                "Gets the number of threads being executed by this process.",
+                "threads"),
+            f.createLongCounter("kernelTime",
+                "Gets kernel time used by the process.",
+                "milliseconds"),
+            f.createLongCounter("userTime",
+                "Gets user time used by the process.",
+                "milliseconds")
+        });
+
+    virtualSize = statisticsType.nameToId("virtualSize");
+    residentSetSize = statisticsType.nameToId("residentSetSize");
+    threadCount = statisticsType.nameToId("threadCount");
+    kernelTime = statisticsType.nameToId("kernelTime");
+    userTime = statisticsType.nameToId("userTime");
+  }
+
+  private OshiProcessStats() {
+    // no instances allowed
+  }
+
+  public static @NotNull StatisticsType getType() {
+    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/platform/OshiStatistics.java b/geode-core/src/main/java/org/apache/geode/internal/statistics/platform/OshiStatistics.java
new file mode 100644
index 0000000..2da3f55
--- /dev/null
+++ b/geode-core/src/main/java/org/apache/geode/internal/statistics/platform/OshiStatistics.java
@@ -0,0 +1,109 @@
+/*
+ * 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/internal/statistics/platform/OshiSystemStats.java b/geode-core/src/main/java/org/apache/geode/internal/statistics/platform/OshiSystemStats.java
new file mode 100644
index 0000000..7693cd0
--- /dev/null
+++ b/geode-core/src/main/java/org/apache/geode/internal/statistics/platform/OshiSystemStats.java
@@ -0,0 +1,136 @@
+/*
+ * 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 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 OshiSystemStats {
+  static final int processCount;
+  static final int threadCount;
+  static final int contextSwitches;
+  static final int interrupts;
+  static final int physicalProcessorCount;
+  static final int logicalProcessorCount;
+  static final int maxFreq;
+  static final int systemLoadAverage1;
+  static final int systemLoadAverage5;
+  static final int systemLoadAverage15;
+  static final int systemCpuLoadTicksUSER;
+  static final int systemCpuLoadTicksNICE;
+  static final int systemCpuLoadTicksSYSTEM;
+  static final int systemCpuLoadTicksIDLE;
+  static final int systemCpuLoadTicksIOWAIT;
+  static final int systemCpuLoadTicksIRQ;
+  static final int systemCpuLoadTicksSOFTIRQ;
+  static final int systemCpuLoadTicksSTEAL;
+
+  @Immutable
+  private static final StatisticsType statisticsType;
+
+  static {
+    final StatisticsTypeFactory f = StatisticsTypeFactoryImpl.singleton();
+
+    statisticsType = f.createType("OperatingSystemStats", "Statistics for an Operating System.",
+        new StatisticDescriptor[] {
+            f.createLongGauge("processCount", "Get the number of processes currently running.",
+                "processes"),
+            f.createLongGauge("threadCount",
+                "Get the number of threads currently running",
+                "threads"),
+            f.createLongCounter("contextSwitches",
+                "Get the number of system-wide context switches which have occurred.",
+                "operations", false),
+            f.createLongCounter("interrupts",
+                "Get the number of system-wide interrupts which have occurred.",
+                "interrupts"),
+            f.createLongGauge("physicalProcessorCount",
+                "Get the number of physical CPUs/cores available for processing.",
+                "processors"),
+            f.createLongGauge("logicalProcessorCount",
+                "Get the number of logical CPUs available for processing. This value may be higher than physical CPUs if hyperthreading is enabled.",
+                "processors"),
+            f.createLongGauge("maxFreq",
+                "Maximum frequency (in Hz), of the logical processors on this CPU.",
+                "Hz"),
+            f.createLongCounter("systemCpuLoadTicksUSER",
+                "Time spent in User",
+                "milliseconds"),
+            f.createLongCounter("systemCpuLoadTicksNICE",
+                "Time spent in Nice",
+                "milliseconds"),
+            f.createLongCounter("systemCpuLoadTicksSYSTEM",
+                "Time spent in System",
+                "milliseconds"),
+            f.createLongCounter("systemCpuLoadTicksIDLE",
+                "Time spent in Idle",
+                "milliseconds"),
+            f.createLongCounter("systemCpuLoadTicksIOWAIT",
+                "Time spent in IOWait",
+                "milliseconds"),
+            f.createLongCounter("systemCpuLoadTicksIRQ",
+                "Time spent in IRQ",
+                "milliseconds"),
+            f.createLongCounter("systemCpuLoadTicksSOFTIRQ",
+                "Time spent in SoftIRQ",
+                "milliseconds"),
+            f.createLongCounter("systemCpuLoadTicksSTEAL",
+                "Time spent in Steal",
+                "milliseconds"),
+            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"),
+            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"),
+            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"),
+    });
+
+    processCount = statisticsType.nameToId("processCount");
+    threadCount = statisticsType.nameToId("threadCount");
+    contextSwitches = statisticsType.nameToId("contextSwitches");
+    interrupts = statisticsType.nameToId("interrupts");
+    physicalProcessorCount = statisticsType.nameToId("physicalProcessorCount");
+    logicalProcessorCount = statisticsType.nameToId("logicalProcessorCount");
+    maxFreq = statisticsType.nameToId("maxFreq");
+    systemLoadAverage1 = statisticsType.nameToId("systemLoadAverage1");
+    systemLoadAverage5 = statisticsType.nameToId("systemLoadAverage5");
+    systemLoadAverage15 = statisticsType.nameToId("systemLoadAverage15");
+    systemCpuLoadTicksUSER = statisticsType.nameToId("systemCpuLoadTicksUSER");
+    systemCpuLoadTicksNICE = statisticsType.nameToId("systemCpuLoadTicksNICE");
+    systemCpuLoadTicksSYSTEM = statisticsType.nameToId("systemCpuLoadTicksSYSTEM");
+    systemCpuLoadTicksIDLE = statisticsType.nameToId("systemCpuLoadTicksIDLE");
+    systemCpuLoadTicksIOWAIT = statisticsType.nameToId("systemCpuLoadTicksIOWAIT");
+    systemCpuLoadTicksIRQ = statisticsType.nameToId("systemCpuLoadTicksIRQ");
+    systemCpuLoadTicksSOFTIRQ = statisticsType.nameToId("systemCpuLoadTicksSOFTIRQ");
+    systemCpuLoadTicksSTEAL = statisticsType.nameToId("systemCpuLoadTicksSTEAL");
+  }
+
+  private OshiSystemStats() {
+    // no instances allowed
+  }
+
+  public static @NotNull StatisticsType getType() {
+    return statisticsType;
+  }
+}
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..7888901 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(LinuxSystemStats.getType());
+      Statistics[] systemStats = system.findStatisticsByType(OsStatisticsProvider.getSystemStatType());
 
       if (systemStats != null) {
         return systemStats[0];
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
new file mode 100644
index 0000000..a8220d8
--- /dev/null
+++ b/geode-core/src/test/java/org/apache/geode/internal/statistics/platform/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.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

[geode] 01/16: MacOSX to macOS

Posted by jb...@apache.org.
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 0a485c43b1ea3a260fb4f0dfa75bc0229caf52b7
Author: Jacob Barrett <jb...@pivotal.io>
AuthorDate: Thu Jun 10 16:29:19 2021 -0700

    MacOSX to macOS
---
 .../src/main/java/org/apache/geode/internal/lang/SystemUtils.java | 6 +++---
 .../java/org/apache/geode/internal/lang/SystemUtilsJUnitTest.java | 8 ++++----
 .../test/java/org/apache/geode/internal/util/HostNameTest.java    | 8 ++++----
 .../geode/management/internal/cli/functions/NetstatFunction.java  | 4 ++--
 .../org/apache/geode/management/internal/cli/util/JdkTool.java    | 2 +-
 5 files changed, 14 insertions(+), 14 deletions(-)

diff --git a/geode-core/src/main/java/org/apache/geode/internal/lang/SystemUtils.java b/geode-core/src/main/java/org/apache/geode/internal/lang/SystemUtils.java
index 53f8660..64e079b 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/lang/SystemUtils.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/lang/SystemUtils.java
@@ -44,7 +44,7 @@ public class SystemUtils {
 
   // Operating System Names
   public static final String LINUX_OS_NAME = "Linux";
-  public static final String MAC_OSX_NAME = "Mac";
+  public static final String MAC_OS_NAME = "Mac";
   public static final String WINDOWS_OS_NAME = "Windows";
   public static final String SOLARIS_OS_NAME = "SunOS";
 
@@ -152,8 +152,8 @@ public class SystemUtils {
    *         OSX.
    * @see #isOS(String)
    */
-  public static boolean isMacOSX() {
-    return isOS(MAC_OSX_NAME);
+  public static boolean isMacOS() {
+    return isOS(MAC_OS_NAME);
   }
 
   /**
diff --git a/geode-core/src/test/java/org/apache/geode/internal/lang/SystemUtilsJUnitTest.java b/geode-core/src/test/java/org/apache/geode/internal/lang/SystemUtilsJUnitTest.java
index 5676252..0f5b8eb 100644
--- a/geode-core/src/test/java/org/apache/geode/internal/lang/SystemUtilsJUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/internal/lang/SystemUtilsJUnitTest.java
@@ -18,7 +18,7 @@ import static org.apache.geode.internal.lang.SystemUtils.APPLE_JVM_VENDOR_NAME;
 import static org.apache.geode.internal.lang.SystemUtils.IBM_J9_JVM_NAME;
 import static org.apache.geode.internal.lang.SystemUtils.JAVA_HOTSPOT_JVM_NAME;
 import static org.apache.geode.internal.lang.SystemUtils.LINUX_OS_NAME;
-import static org.apache.geode.internal.lang.SystemUtils.MAC_OSX_NAME;
+import static org.apache.geode.internal.lang.SystemUtils.MAC_OS_NAME;
 import static org.apache.geode.internal.lang.SystemUtils.ORACLE_JROCKIT_JVM_NAME;
 import static org.apache.geode.internal.lang.SystemUtils.ORACLE_JVM_VENDOR_NAME;
 import static org.apache.geode.internal.lang.SystemUtils.WINDOWS_OS_NAME;
@@ -32,7 +32,7 @@ import static org.apache.geode.internal.lang.SystemUtils.isHotSpotVM;
 import static org.apache.geode.internal.lang.SystemUtils.isJ9VM;
 import static org.apache.geode.internal.lang.SystemUtils.isJRockitVM;
 import static org.apache.geode.internal.lang.SystemUtils.isLinux;
-import static org.apache.geode.internal.lang.SystemUtils.isMacOSX;
+import static org.apache.geode.internal.lang.SystemUtils.isMacOS;
 import static org.apache.geode.internal.lang.SystemUtils.isOracleJVM;
 import static org.apache.geode.internal.lang.SystemUtils.isWindows;
 import static org.assertj.core.api.Assertions.assertThat;
@@ -99,8 +99,8 @@ public class SystemUtilsJUnitTest {
   @Test
   public void testIsMacOSX() {
     final boolean expected =
-        ManagementFactory.getOperatingSystemMXBean().getName().contains(MAC_OSX_NAME);
-    assertEquals(expected, isMacOSX());
+        ManagementFactory.getOperatingSystemMXBean().getName().contains(MAC_OS_NAME);
+    assertEquals(expected, isMacOS());
   }
 
   @Test
diff --git a/geode-core/src/test/java/org/apache/geode/internal/util/HostNameTest.java b/geode-core/src/test/java/org/apache/geode/internal/util/HostNameTest.java
index fa502db..fd0450c 100644
--- a/geode-core/src/test/java/org/apache/geode/internal/util/HostNameTest.java
+++ b/geode-core/src/test/java/org/apache/geode/internal/util/HostNameTest.java
@@ -15,7 +15,7 @@
 package org.apache.geode.internal.util;
 
 import static org.apache.geode.internal.lang.SystemUtils.LINUX_OS_NAME;
-import static org.apache.geode.internal.lang.SystemUtils.MAC_OSX_NAME;
+import static org.apache.geode.internal.lang.SystemUtils.MAC_OS_NAME;
 import static org.apache.geode.internal.lang.SystemUtils.SOLARIS_OS_NAME;
 import static org.apache.geode.internal.lang.SystemUtils.WINDOWS_OS_NAME;
 import static org.apache.geode.internal.lang.SystemUtils.isWindows;
@@ -54,7 +54,7 @@ public class HostNameTest {
   }
 
   @Test
-  @Parameters({MAC_OSX_NAME, LINUX_OS_NAME, SOLARIS_OS_NAME, WINDOWS_OS_NAME})
+  @Parameters({MAC_OS_NAME, LINUX_OS_NAME, SOLARIS_OS_NAME, WINDOWS_OS_NAME})
   public void shouldExecHostNameIfEnvValueNotAvailableOnOS(String osName) throws IOException {
     setHostNamePropertiesNull(osName);
     String result = new HostName().determineHostName();
@@ -62,7 +62,7 @@ public class HostNameTest {
   }
 
   @Test
-  @Parameters({MAC_OSX_NAME, LINUX_OS_NAME, SOLARIS_OS_NAME, WINDOWS_OS_NAME})
+  @Parameters({MAC_OS_NAME, LINUX_OS_NAME, SOLARIS_OS_NAME, WINDOWS_OS_NAME})
   public void shouldUseComputerNameIfAvailableOnOS(String osName) throws IOException {
     setHostNameProperties(osName);
     String result = new HostName().determineHostName();
@@ -70,7 +70,7 @@ public class HostNameTest {
   }
 
   @Test
-  @Parameters({MAC_OSX_NAME, LINUX_OS_NAME, SOLARIS_OS_NAME, WINDOWS_OS_NAME})
+  @Parameters({MAC_OS_NAME, LINUX_OS_NAME, SOLARIS_OS_NAME, WINDOWS_OS_NAME})
   public void shouldBeNullIfEnvValueNotAvailableOnOS(String osName) throws IOException {
     setHostNamePropertiesNull(osName);
     String result = new HostName().getHostNameFromEnv();
diff --git a/geode-gfsh/src/main/java/org/apache/geode/management/internal/cli/functions/NetstatFunction.java b/geode-gfsh/src/main/java/org/apache/geode/management/internal/cli/functions/NetstatFunction.java
index 7375c0b..e9f3a02 100644
--- a/geode-gfsh/src/main/java/org/apache/geode/management/internal/cli/functions/NetstatFunction.java
+++ b/geode-gfsh/src/main/java/org/apache/geode/management/internal/cli/functions/NetstatFunction.java
@@ -18,7 +18,7 @@ import static org.apache.geode.internal.lang.SystemUtils.getOsArchitecture;
 import static org.apache.geode.internal.lang.SystemUtils.getOsName;
 import static org.apache.geode.internal.lang.SystemUtils.getOsVersion;
 import static org.apache.geode.internal.lang.SystemUtils.isLinux;
-import static org.apache.geode.internal.lang.SystemUtils.isMacOSX;
+import static org.apache.geode.internal.lang.SystemUtils.isMacOS;
 import static org.apache.geode.internal.lang.SystemUtils.isSolaris;
 
 import java.io.BufferedReader;
@@ -163,7 +163,7 @@ public class NetstatFunction implements InternalFunction<NetstatFunction.Netstat
     existingNetstatInfo.append("################ ").append(LSOF_COMMAND)
         .append(" output ###################").append(lineSeparator);
 
-    if (isLinux() || isMacOSX() || isSolaris()) {
+    if (isLinux() || isMacOS() || isSolaris()) {
       List<String> cmdOptionsList = new ArrayList<>();
       cmdOptionsList.add(LSOF_COMMAND);
       cmdOptionsList.add("-n");
diff --git a/geode-gfsh/src/main/java/org/apache/geode/management/internal/cli/util/JdkTool.java b/geode-gfsh/src/main/java/org/apache/geode/management/internal/cli/util/JdkTool.java
index e0ebd9b..99e79a1 100644
--- a/geode-gfsh/src/main/java/org/apache/geode/management/internal/cli/util/JdkTool.java
+++ b/geode-gfsh/src/main/java/org/apache/geode/management/internal/cli/util/JdkTool.java
@@ -31,7 +31,7 @@ public class JdkTool {
   protected static final String JAVA_HOME = System.getProperty("java.home");
 
   public static String getJVisualVMPathname() {
-    if (SystemUtils.isMacOSX()) {
+    if (SystemUtils.isMacOS()) {
       try {
         return IOUtils.verifyPathnameExists(
             "/System/Library/Java/Support/VisualVM.bundle/Contents/Home/bin/jvisualvm");

[geode] 08/16: Update to OSHI 5.8.3.

Posted by jb...@apache.org.
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 29287169f5fee315ac8f90c9bcbb8f75adc5e101
Author: Jacob Barrett <jb...@pivotal.io>
AuthorDate: Mon Nov 15 21:06:22 2021 -0800

    Update to OSHI 5.8.3.
---
 .../groovy/org/apache/geode/gradle/plugins/DependencyConstraints.groovy | 1 +
 geode-core/build.gradle                                                 | 2 +-
 2 files changed, 2 insertions(+), 1 deletion(-)

diff --git a/buildSrc/src/main/groovy/org/apache/geode/gradle/plugins/DependencyConstraints.groovy b/buildSrc/src/main/groovy/org/apache/geode/gradle/plugins/DependencyConstraints.groovy
index a12b318..815e5d2 100644
--- a/buildSrc/src/main/groovy/org/apache/geode/gradle/plugins/DependencyConstraints.groovy
+++ b/buildSrc/src/main/groovy/org/apache/geode/gradle/plugins/DependencyConstraints.groovy
@@ -174,6 +174,7 @@ class DependencyConstraints implements Plugin<Project> {
         api(group: 'redis.clients', name: 'jedis', version: '3.6.3')
         api(group: 'xerces', name: 'xercesImpl', version: '2.12.0')
         api(group: 'xml-apis', name: 'xml-apis', version: '1.4.01')
+        api(group: 'com.github.oshi', name: 'oshi-core', version: '5.8.3')
       }
     }
 
diff --git a/geode-core/build.gradle b/geode-core/build.gradle
index eaa2f54..e6389f5 100755
--- a/geode-core/build.gradle
+++ b/geode-core/build.gradle
@@ -313,7 +313,7 @@ dependencies {
   implementation('com.healthmarketscience.rmiio:rmiio')
 
   // OSHI OS system stats
-  implementation('com.github.oshi:oshi-core:5.7.4')
+  implementation('com.github.oshi:oshi-core')
 
   //Geode-common has annotations and other pieces used geode-core
   api(project(':geode-common'))

[geode] 14/16: Fix tests.

Posted by jb...@apache.org.
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 8c8fa8518cab3ff872563399fe188e4ca6400171
Author: Jacob Barrett <jb...@pivotal.io>
AuthorDate: Thu Nov 18 11:34:14 2021 -0800

    Fix tests.
---
 .../geode/internal/statistics/GemFireStatSamplerIntegrationTest.java    | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

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 d799b32..08b20e1 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
@@ -461,7 +461,7 @@ public class GemFireStatSamplerIntegrationTest extends StatSamplerTestCase {
     Properties props = createGemFireProperties();
     props.setProperty(STATISTIC_ARCHIVE_FILE, archiveFileName);
     props.setProperty(ARCHIVE_FILE_SIZE_LIMIT, "2");
-    props.setProperty(ARCHIVE_DISK_SPACE_LIMIT, "14");
+    props.setProperty(ARCHIVE_DISK_SPACE_LIMIT, "140");
     props.setProperty(STATISTIC_SAMPLE_RATE, String.valueOf(sampleRate));
 
     connect(props);