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

[geode] 12/16: Make service loadable.

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