You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@reef.apache.org by we...@apache.org on 2015/11/30 22:38:11 UTC

incubator-reef git commit: [REEF-1020] Improve OSUtils to support Windows and Mac

Repository: incubator-reef
Updated Branches:
  refs/heads/master 66ddb2440 -> 8ca82a950


[REEF-1020] Improve OSUtils to support Windows and Mac

JIRA:
  [REEF-1020](https://issues.apache.org/jira/browse/REEF-1020)

Pull request:
  This closes #685


Project: http://git-wip-us.apache.org/repos/asf/incubator-reef/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-reef/commit/8ca82a95
Tree: http://git-wip-us.apache.org/repos/asf/incubator-reef/tree/8ca82a95
Diff: http://git-wip-us.apache.org/repos/asf/incubator-reef/diff/8ca82a95

Branch: refs/heads/master
Commit: 8ca82a950cd84e5394d082874c8aa67d1bd882ea
Parents: 66ddb24
Author: Dongjoon Hyun <do...@apache.org>
Authored: Fri Nov 27 12:47:39 2015 +0900
Committer: Markus Weimer <we...@apache.org>
Committed: Mon Nov 30 13:37:04 2015 -0800

----------------------------------------------------------------------
 .../main/java/org/apache/reef/util/OSUtils.java | 52 +++++++++++++++++---
 .../java/org/apache/reef/util/OSUtilsTest.java  | 37 ++++++++++++++
 .../local/client/LocalRuntimeConfiguration.java |  3 ++
 .../runtime/local/process/RunnableProcess.java  |  4 +-
 .../EvaluatorFailureDuringAlarmDriver.java      |  7 +++
 .../evaluatorfailure/EvaluatorFailureTest.java  |  1 +
 6 files changed, 95 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/8ca82a95/lang/java/reef-common/src/main/java/org/apache/reef/util/OSUtils.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-common/src/main/java/org/apache/reef/util/OSUtils.java b/lang/java/reef-common/src/main/java/org/apache/reef/util/OSUtils.java
index 9926506..8890f88 100644
--- a/lang/java/reef-common/src/main/java/org/apache/reef/util/OSUtils.java
+++ b/lang/java/reef-common/src/main/java/org/apache/reef/util/OSUtils.java
@@ -42,6 +42,15 @@ public final class OSUtils {
   }
 
   /**
+   * Determines whether the current JVM is running on the Unix-based OS.
+   *
+   * @return true, if the JVM is running on Linux/Mac. false, otherwise
+   */
+  public static boolean isUnix() {
+    return isLinux() || isMac();
+  }
+
+  /**
    * Determines whether the current JVM is running on the Linux OS.
    *
    * @return true, if the JVM is running on Linux. false, otherwise
@@ -51,10 +60,19 @@ public final class OSUtils {
   }
 
   /**
-   * @return the process ID of the JVM, if running on Linux. This returns -1 for other OSs.
+   * Determines whether the current JVM is running on the Mac OS.
+   *
+   * @return true, if the JVM is running on Mac. false, otherwise
+   */
+  public static boolean isMac() {
+    return System.getProperty("os.name").toLowerCase().contains("mac");
+  }
+
+  /**
+   * @return the process ID of the JVM, if running on Linux/Windows. This returns -1 for other OSs.
    */
   public static long getPID() {
-    if (isLinux()) {
+    if (isUnix()) {
       try {
         final Process process = new ProcessBuilder()
             .command("bash", "-c", "echo $PPID")
@@ -64,31 +82,51 @@ public final class OSUtils {
         final Long result = Long.valueOf(new String(returnBytes, StandardCharsets.UTF_8).trim());
         process.destroy();
         return result;
-      } catch (final IOException e) {
+      } catch (final Exception e) {
+        LOG.log(Level.SEVERE, "Unable to determine PID", e);
+        return -1;
+      }
+    } else if (isWindows()) {
+      try {
+        final Process process = new ProcessBuilder()
+            .command("powershell.exe", "-NoProfile", "-Command",
+                "wmic process where processid=$pid get parentprocessid")
+            .start();
+        final byte[] returnBytes = new byte[128];
+        process.getInputStream().read(returnBytes);
+        final Long result = Long.valueOf(new String(returnBytes, StandardCharsets.UTF_8).split("\n")[1].trim());
+        process.destroy();
+        return result;
+      } catch (final Exception e) {
         LOG.log(Level.SEVERE, "Unable to determine PID", e);
         return -1;
       }
-
     } else {
       return -1;
     }
   }
 
   /**
-   * Applies `kill -9` to the process.
+   * Kill the process.
    *
    * @param pid Process id
    * @throws IOException
    */
   public static void kill(final long pid) throws IOException, InterruptedException {
-    if (isLinux()) {
+    if (isUnix()) {
       final Process process = new ProcessBuilder()
           .command("bash", "-c", "kill", "-9", String.valueOf(pid))
           .start();
       final int returnCode = process.waitFor();
       LOG.fine("Kill returned: " + returnCode);
+    } else if (isWindows()) {
+      final Process process = new ProcessBuilder()
+          .command("taskkill.exe", "/f", "/pid", String.valueOf(pid))
+          .start();
+      final int returnCode = process.waitFor();
+      LOG.fine("Kill returned: " + returnCode);
     } else {
-      throw new UnsupportedOperationException("Unable to execute kill on non-linux OS");
+      throw new UnsupportedOperationException("Unable to execute kill on unknown OS");
     }
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/8ca82a95/lang/java/reef-common/src/test/java/org/apache/reef/util/OSUtilsTest.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-common/src/test/java/org/apache/reef/util/OSUtilsTest.java b/lang/java/reef-common/src/test/java/org/apache/reef/util/OSUtilsTest.java
new file mode 100644
index 0000000..3431556
--- /dev/null
+++ b/lang/java/reef-common/src/test/java/org/apache/reef/util/OSUtilsTest.java
@@ -0,0 +1,37 @@
+/*
+ * 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.reef.util;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+/**
+ * Test for OSUtil.
+ */
+public final class OSUtilsTest {
+
+  /**
+   * Test if getPID() gets non-invalid process id.
+   */
+  @Test
+  public void testGetPID() {
+    Assert.assertTrue(OSUtils.isUnix() || OSUtils.isWindows());
+    Assert.assertTrue(OSUtils.getPID() != -1);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/8ca82a95/lang/java/reef-runtime-local/src/main/java/org/apache/reef/runtime/local/client/LocalRuntimeConfiguration.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-runtime-local/src/main/java/org/apache/reef/runtime/local/client/LocalRuntimeConfiguration.java b/lang/java/reef-runtime-local/src/main/java/org/apache/reef/runtime/local/client/LocalRuntimeConfiguration.java
index fbf299c..aa045f9 100644
--- a/lang/java/reef-runtime-local/src/main/java/org/apache/reef/runtime/local/client/LocalRuntimeConfiguration.java
+++ b/lang/java/reef-runtime-local/src/main/java/org/apache/reef/runtime/local/client/LocalRuntimeConfiguration.java
@@ -21,6 +21,7 @@ package org.apache.reef.runtime.local.client;
 import org.apache.reef.client.parameters.DriverConfigurationProviders;
 import org.apache.reef.runtime.common.client.CommonRuntimeConfiguration;
 import org.apache.reef.runtime.common.client.api.JobSubmissionHandler;
+import org.apache.reef.runtime.common.evaluator.PIDStoreStartHandler;
 import org.apache.reef.runtime.common.files.RuntimeClasspathProvider;
 import org.apache.reef.runtime.common.parameters.JVMHeapSlack;
 import org.apache.reef.runtime.local.LocalClasspathProvider;
@@ -32,6 +33,7 @@ import org.apache.reef.tang.formats.ConfigurationModule;
 import org.apache.reef.tang.formats.ConfigurationModuleBuilder;
 import org.apache.reef.tang.formats.OptionalImpl;
 import org.apache.reef.tang.formats.OptionalParameter;
+import org.apache.reef.wake.time.Clock;
 
 import java.util.concurrent.ExecutorService;
 
@@ -86,6 +88,7 @@ public class LocalRuntimeConfiguration extends ConfigurationModuleBuilder {
       .bindNamedParameter(RootFolder.class, RUNTIME_ROOT_FOLDER)
       .bindNamedParameter(JVMHeapSlack.class, JVM_HEAP_SLACK)
       .bindSetEntry(DriverConfigurationProviders.class, DRIVER_CONFIGURATION_PROVIDERS)
+      .bindSetEntry(Clock.StartHandler.class, PIDStoreStartHandler.class)
       .bindSetEntry(RackNames.class, RACK_NAMES)
       .build();
 

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/8ca82a95/lang/java/reef-runtime-local/src/main/java/org/apache/reef/runtime/local/process/RunnableProcess.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-runtime-local/src/main/java/org/apache/reef/runtime/local/process/RunnableProcess.java b/lang/java/reef-runtime-local/src/main/java/org/apache/reef/runtime/local/process/RunnableProcess.java
index 323dcb0..db46b14 100644
--- a/lang/java/reef-runtime-local/src/main/java/org/apache/reef/runtime/local/process/RunnableProcess.java
+++ b/lang/java/reef-runtime-local/src/main/java/org/apache/reef/runtime/local/process/RunnableProcess.java
@@ -208,12 +208,12 @@ public final class RunnableProcess implements Runnable {
 
       if (this.processIsRunning()) {
         LOG.log(Level.WARNING, "The child process survived Process.destroy()");
-        if (OSUtils.isLinux()) {
+        if (OSUtils.isUnix() || OSUtils.isWindows()) {
           LOG.log(Level.WARNING, "Attempting to kill the process via the kill command line");
           try {
             final long pid = readPID();
             OSUtils.kill(pid);
-          } catch (final IOException | InterruptedException e) {
+          } catch (final IOException | InterruptedException | NumberFormatException e) {
             LOG.log(Level.SEVERE, "Unable to kill the process.", e);
           }
         }

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/8ca82a95/lang/java/reef-tests/src/test/java/org/apache/reef/tests/evaluatorfailure/EvaluatorFailureDuringAlarmDriver.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-tests/src/test/java/org/apache/reef/tests/evaluatorfailure/EvaluatorFailureDuringAlarmDriver.java b/lang/java/reef-tests/src/test/java/org/apache/reef/tests/evaluatorfailure/EvaluatorFailureDuringAlarmDriver.java
index 11ca760..e76e2c4 100644
--- a/lang/java/reef-tests/src/test/java/org/apache/reef/tests/evaluatorfailure/EvaluatorFailureDuringAlarmDriver.java
+++ b/lang/java/reef-tests/src/test/java/org/apache/reef/tests/evaluatorfailure/EvaluatorFailureDuringAlarmDriver.java
@@ -18,6 +18,7 @@
  */
 package org.apache.reef.tests.evaluatorfailure;
 
+import org.apache.reef.driver.context.ActiveContext;
 import org.apache.reef.driver.context.ContextConfiguration;
 import org.apache.reef.driver.context.FailedContext;
 import org.apache.reef.driver.evaluator.AllocatedEvaluator;
@@ -70,6 +71,12 @@ final class EvaluatorFailureDuringAlarmDriver {
     }
   }
 
+  final class ActiveContextHandler implements EventHandler<ActiveContext> {
+    @Override
+    public void onNext(final ActiveContext context) throws DriverSideFailure {
+    }
+  }
+
   final class ContextFailureHandler implements EventHandler<FailedContext> {
     @Override
     public void onNext(final FailedContext failedContext) {

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/8ca82a95/lang/java/reef-tests/src/test/java/org/apache/reef/tests/evaluatorfailure/EvaluatorFailureTest.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-tests/src/test/java/org/apache/reef/tests/evaluatorfailure/EvaluatorFailureTest.java b/lang/java/reef-tests/src/test/java/org/apache/reef/tests/evaluatorfailure/EvaluatorFailureTest.java
index 41f6817..938f57c 100644
--- a/lang/java/reef-tests/src/test/java/org/apache/reef/tests/evaluatorfailure/EvaluatorFailureTest.java
+++ b/lang/java/reef-tests/src/test/java/org/apache/reef/tests/evaluatorfailure/EvaluatorFailureTest.java
@@ -61,6 +61,7 @@ public class EvaluatorFailureTest {
             EnvironmentUtils.getClassLocation(EvaluatorFailureDuringAlarmDriver.class))
         .set(DriverConfiguration.DRIVER_IDENTIFIER, "TEST_EvaluatorFailureTest")
         .set(DriverConfiguration.ON_DRIVER_STARTED, OnDriverStartedAllocateOne.class)
+        .set(DriverConfiguration.ON_CONTEXT_ACTIVE, EvaluatorFailureDuringAlarmDriver.ActiveContextHandler.class)
         .set(DriverConfiguration.ON_EVALUATOR_ALLOCATED,
             EvaluatorFailureDuringAlarmDriver.EvaluatorAllocatedHandler.class)
         .set(DriverConfiguration.ON_EVALUATOR_FAILED, EvaluatorFailureDuringAlarmDriver.EvaluatorFailureHandler.class)