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

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

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());
   }