You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@geode.apache.org by kl...@apache.org on 2019/02/25 21:18:57 UTC

[geode] branch develop updated: GEODE-6442: Use PID for OS and Process stats if available

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

klund pushed a commit to branch develop
in repository https://gitbox.apache.org/repos/asf/geode.git


The following commit(s) were added to refs/heads/develop by this push:
     new 9fb803d  GEODE-6442: Use PID for OS and Process stats if available
9fb803d is described below

commit 9fb803d58321eedd4a022febb52e6186298e5139
Author: Kirk Lund <kl...@apache.org>
AuthorDate: Fri Feb 22 10:22:45 2019 -0800

    GEODE-6442: Use PID for OS and Process stats if available
    
    If the PID is unavailable for any reason, fall back to using
    DistributedSystem ID.
---
 .../geode/internal/process/ProcessUtils.java       |  17 ++-
 .../process/UncheckedPidUnavailableException.java  |  54 ++++++++
 .../geode/internal/statistics/HostStatSampler.java |  27 +++-
 .../sanctioned-geode-core-serializables.txt        |   1 +
 .../internal/statistics/HostStatSamplerTest.java   | 147 +++++++++++++++++++++
 5 files changed, 241 insertions(+), 5 deletions(-)

diff --git a/geode-core/src/main/java/org/apache/geode/internal/process/ProcessUtils.java b/geode-core/src/main/java/org/apache/geode/internal/process/ProcessUtils.java
index c1dc661..33cb9a5 100755
--- a/geode-core/src/main/java/org/apache/geode/internal/process/ProcessUtils.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/process/ProcessUtils.java
@@ -52,6 +52,22 @@ public class ProcessUtils {
   }
 
   /**
+   * Returns the pid for this process without throwing a checked exception.
+   *
+   * @throws UncheckedPidUnavailableException if parsing the pid from the name of the RuntimeMXBean
+   *         fails
+   *
+   * @see java.lang.management.RuntimeMXBean#getName()
+   */
+  public static int identifyPidAsUnchecked() throws UncheckedPidUnavailableException {
+    try {
+      return identifyPid();
+    } catch (PidUnavailableException e) {
+      throw new UncheckedPidUnavailableException(e);
+    }
+  }
+
+  /**
    * Returns the pid for this process using the specified name from RuntimeMXBean.
    *
    * @throws PidUnavailableException if parsing the pid from the RuntimeMXBean name fails
@@ -59,7 +75,6 @@ public class ProcessUtils {
   public static int identifyPid(final String name) throws PidUnavailableException {
     notEmpty(name, "Invalid name '" + name + "' specified");
 
-
     try {
       final int index = name.indexOf('@');
       if (index < 0) {
diff --git a/geode-core/src/main/java/org/apache/geode/internal/process/UncheckedPidUnavailableException.java b/geode-core/src/main/java/org/apache/geode/internal/process/UncheckedPidUnavailableException.java
new file mode 100644
index 0000000..b415762
--- /dev/null
+++ b/geode-core/src/main/java/org/apache/geode/internal/process/UncheckedPidUnavailableException.java
@@ -0,0 +1,54 @@
+/*
+ * 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.process;
+
+/**
+ * Wraps an {@link PidUnavailableException} with an unchecked exception.
+ */
+public class UncheckedPidUnavailableException extends RuntimeException {
+  private static final long serialVersionUID = -4896120572252598765L;
+
+  /**
+   * Creates a new {@code UncheckedPidUnavailableException}.
+   */
+  public UncheckedPidUnavailableException(final String message) {
+    super(message);
+  }
+
+  /**
+   * Creates a new {@code UncheckedPidUnavailableException} that was caused by a given exception
+   */
+  public UncheckedPidUnavailableException(final String message,
+      final PidUnavailableException cause) {
+    super(message, cause);
+  }
+
+  /**
+   * Creates a new {@code UncheckedPidUnavailableException} that was caused by a given exception
+   */
+  public UncheckedPidUnavailableException(final PidUnavailableException cause) {
+    super(cause.getMessage(), cause);
+  }
+
+  /**
+   * Returns the cause of this exception.
+   *
+   * @return the {@code PidUnavailableException} which is the cause of this exception.
+   */
+  @Override
+  public PidUnavailableException getCause() {
+    return (PidUnavailableException) super.getCause();
+  }
+}
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 e55accb..25fb5bc 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
@@ -19,6 +19,7 @@ import java.net.UnknownHostException;
 import java.util.List;
 import java.util.Optional;
 import java.util.concurrent.TimeUnit;
+import java.util.function.IntSupplier;
 
 import org.apache.logging.log4j.Logger;
 
@@ -34,6 +35,8 @@ import org.apache.geode.internal.logging.LogService;
 import org.apache.geode.internal.logging.LoggingThread;
 import org.apache.geode.internal.logging.log4j.LogMarker;
 import org.apache.geode.internal.net.SocketCreator;
+import org.apache.geode.internal.process.ProcessUtils;
+import org.apache.geode.internal.process.UncheckedPidUnavailableException;
 import org.apache.geode.internal.statistics.platform.OsStatisticsFactory;
 import org.apache.geode.internal.util.concurrent.StoppableCountDownLatch;
 
@@ -88,12 +91,10 @@ public abstract class HostStatSampler
   private final StoppableCountDownLatch statSamplerInitializedLatch;
 
   private final CancelCriterion stopper;
-
   private final CallbackSampler callbackSampler;
-
   private final NanoTimer timer;
-
   private final LogFile logFile;
+  private final IntSupplier pidSupplier;
 
   protected HostStatSampler(CancelCriterion stopper, StatSamplerStats samplerStats) {
     this(stopper, samplerStats, new NanoTimer());
@@ -111,6 +112,11 @@ public abstract class HostStatSampler
 
   protected HostStatSampler(CancelCriterion stopper, StatSamplerStats samplerStats, NanoTimer timer,
       LogFile logFile) {
+    this(stopper, samplerStats, timer, logFile, ProcessUtils::identifyPidAsUnchecked);
+  }
+
+  HostStatSampler(CancelCriterion stopper, StatSamplerStats samplerStats, NanoTimer timer,
+      LogFile logFile, IntSupplier pidSupplier) {
     this.stopper = stopper;
     this.statSamplerInitializedLatch = new StoppableCountDownLatch(this.stopper, 1);
     this.samplerStats = samplerStats;
@@ -118,6 +124,7 @@ public abstract class HostStatSampler
     this.callbackSampler = new CallbackSampler(stopper, samplerStats);
     this.timer = timer;
     this.logFile = logFile;
+    this.pidSupplier = pidSupplier;
   }
 
   public StatSamplerStats getStatSamplerStats() {
@@ -448,6 +455,18 @@ public abstract class HostStatSampler
     // do nothing by default
   }
 
+  protected long getSpecialStatsId() {
+    try {
+      int pid = pidSupplier.getAsInt();
+      if (pid > 0) {
+        return pid;
+      }
+    } catch (UncheckedPidUnavailableException ignored) {
+      // ignore and fall through
+    }
+    return getSystemId();
+  }
+
   protected boolean fileSizeLimitInKB() {
     return this.fileSizeLimitInKB;
   }
@@ -469,7 +488,7 @@ public abstract class HostStatSampler
    */
   private synchronized void initSpecialStats() {
     // add a vm resource
-    long id = getSystemId();
+    long id = getSpecialStatsId();
     this.vmStats = VMStatsContractFactory.create(getStatisticsManager(), id);
     initProcessStats(id);
   }
diff --git a/geode-core/src/main/resources/org/apache/geode/internal/sanctioned-geode-core-serializables.txt b/geode-core/src/main/resources/org/apache/geode/internal/sanctioned-geode-core-serializables.txt
index 7a57f84..25aee89 100644
--- a/geode-core/src/main/resources/org/apache/geode/internal/sanctioned-geode-core-serializables.txt
+++ b/geode-core/src/main/resources/org/apache/geode/internal/sanctioned-geode-core-serializables.txt
@@ -451,6 +451,7 @@ org/apache/geode/internal/process/ConnectionFailedException,true,562263645283675
 org/apache/geode/internal/process/FileAlreadyExistsException,true,5471082555536094256
 org/apache/geode/internal/process/MBeanInvocationFailedException,true,7991096466859690801
 org/apache/geode/internal/process/PidUnavailableException,true,-1660269538268828059
+org/apache/geode/internal/process/UncheckedPidUnavailableException,true,-4896120572252598765
 org/apache/geode/internal/process/signal/Signal,false,description:java/lang/String,name:java/lang/String,number:int,type:org/apache/geode/internal/process/signal/SignalType
 org/apache/geode/internal/process/signal/SignalEvent,false,signal:org/apache/geode/internal/process/signal/Signal
 org/apache/geode/internal/process/signal/SignalType,false,description:java/lang/String
diff --git a/geode-core/src/test/java/org/apache/geode/internal/statistics/HostStatSamplerTest.java b/geode-core/src/test/java/org/apache/geode/internal/statistics/HostStatSamplerTest.java
new file mode 100644
index 0000000..2fc60b7
--- /dev/null
+++ b/geode-core/src/test/java/org/apache/geode/internal/statistics/HostStatSamplerTest.java
@@ -0,0 +1,147 @@
+/*
+ * 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 static org.mockito.Mockito.mock;
+
+import java.io.File;
+import java.util.function.IntSupplier;
+
+import org.junit.Before;
+import org.junit.Test;
+
+import org.apache.geode.CancelCriterion;
+import org.apache.geode.internal.NanoTimer;
+import org.apache.geode.internal.logging.LogFile;
+import org.apache.geode.internal.process.PidUnavailableException;
+import org.apache.geode.internal.process.UncheckedPidUnavailableException;
+
+public class HostStatSamplerTest {
+
+  private CancelCriterion cancelCriterion;
+  private StatSamplerStats statSamplerStats;
+  private NanoTimer timer;
+  private LogFile logFile;
+
+  private HostStatSampler hostStatSampler;
+
+  @Before
+  public void setUp() {
+    cancelCriterion = mock(CancelCriterion.class);
+    statSamplerStats = mock(StatSamplerStats.class);
+    timer = new NanoTimer();
+    logFile = null;
+  }
+
+  @Test
+  public void getSpecialStatsId_returnsPidFromPidSupplier_ifValueIsGreaterThanZero() {
+    int thePid = 42;
+    IntSupplier thePidSupplier = () -> thePid;
+    long anySystemId = 2;
+    hostStatSampler = new TestableHostStatSampler(cancelCriterion, statSamplerStats, timer, logFile,
+        thePidSupplier, anySystemId);
+
+    assertThat(hostStatSampler.getSpecialStatsId()).isEqualTo(thePid);
+  }
+
+  @Test
+  public void getSpecialStatsId_returnsSystemId_ifValueFromPidSupplierIsZero() {
+    IntSupplier thePidSupplier = () -> 0;
+    long theSystemId = 42;
+    hostStatSampler = new TestableHostStatSampler(cancelCriterion, statSamplerStats, timer, logFile,
+        thePidSupplier, theSystemId);
+
+    assertThat(hostStatSampler.getSpecialStatsId()).isEqualTo(theSystemId);
+  }
+
+  @Test
+  public void getSpecialStatsId_returnsSystemId_ifValueFromPidSupplierIsLessThanZero() {
+    IntSupplier thePidSupplier = () -> -1;
+    long theSystemId = 42;
+    hostStatSampler = new TestableHostStatSampler(cancelCriterion, statSamplerStats, timer, logFile,
+        thePidSupplier, theSystemId);
+
+    assertThat(hostStatSampler.getSpecialStatsId()).isEqualTo(theSystemId);
+  }
+
+  @Test
+  public void getSpecialStatsId_returnsSystemId_ifPidSupplierThrows() {
+    IntSupplier thePidSupplier = () -> {
+      throw new UncheckedPidUnavailableException(new PidUnavailableException("Pid not found"));
+    };
+    long theSystemId = 42;
+    hostStatSampler = new TestableHostStatSampler(cancelCriterion, statSamplerStats, timer, logFile,
+        thePidSupplier, theSystemId);
+
+    assertThat(hostStatSampler.getSpecialStatsId()).isEqualTo(theSystemId);
+  }
+
+  private static class TestableHostStatSampler extends HostStatSampler {
+
+    private final long systemId;
+
+    TestableHostStatSampler(CancelCriterion stopper, StatSamplerStats samplerStats, NanoTimer timer,
+        LogFile logFile, IntSupplier pidSupplier, long systemId) {
+      super(stopper, samplerStats, timer, logFile, pidSupplier);
+      this.systemId = systemId;
+    }
+
+    @Override
+    protected void checkListeners() {
+
+    }
+
+    @Override
+    protected int getSampleRate() {
+      return 0;
+    }
+
+    @Override
+    public boolean isSamplingEnabled() {
+      return false;
+    }
+
+    @Override
+    protected StatisticsManager getStatisticsManager() {
+      return null;
+    }
+
+    @Override
+    public File getArchiveFileName() {
+      return null;
+    }
+
+    @Override
+    public long getArchiveFileSizeLimit() {
+      return 0;
+    }
+
+    @Override
+    public long getArchiveDiskSpaceLimit() {
+      return 0;
+    }
+
+    @Override
+    public long getSystemId() {
+      return systemId;
+    }
+
+    @Override
+    public String getProductDescription() {
+      return null;
+    }
+  }
+}