Explorar el Código

YARN-548. Add tests for YarnUncaughtExceptionHandler (Vadim Bondarev via jeagles)

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-0.23@1485885 13f79535-47bb-0310-9956-ffa450edef68
Jonathan Turner Eagles hace 12 años
padre
commit
bfe8d947cd

+ 74 - 2
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ExitUtil.java

@@ -30,7 +30,9 @@ import org.apache.hadoop.classification.InterfaceStability;
 public final class ExitUtil {
   private final static Log LOG = LogFactory.getLog(ExitUtil.class.getName());
   private static volatile boolean systemExitDisabled = false;
+  private static volatile boolean systemHaltDisabled = false;
   private static volatile ExitException firstExitException;
+  private static volatile HaltException firstHaltException;
 
   public static class ExitException extends RuntimeException {
     private static final long serialVersionUID = 1L;
@@ -42,6 +44,16 @@ public final class ExitUtil {
     }
   }
 
+  public static class HaltException extends RuntimeException {
+    private static final long serialVersionUID = 1L;
+    public final int status;
+
+    public HaltException(int status, String msg) {
+      super(msg);
+      this.status = status;
+    }
+  }
+
   /**
    * Disable the use of System.exit for testing.
    */
@@ -49,6 +61,14 @@ public final class ExitUtil {
     systemExitDisabled = true;
   }
 
+  /**
+   * Disable the use of {@code Runtime.getRuntime().halt() }
+   * for testing.
+   */
+  public static void disableSystemHalt() {
+    systemHaltDisabled = true;
+  }
+
   /**
    * @return true if terminate has been called
    */
@@ -57,6 +77,13 @@ public final class ExitUtil {
     return firstExitException != null;
   }
 
+  /**
+   * @return true if halt has been called
+   */
+  public static boolean haltCalled() {
+    return firstHaltException != null;
+  }
+
   /**
    * @return the first ExitException thrown, null if none thrown yet
    */
@@ -64,6 +91,13 @@ public final class ExitUtil {
     return firstExitException;
   }
 
+  /**
+   * @return the first {@code HaltException} thrown, null if none thrown yet
+   */
+  public static HaltException getFirstHaltException() {
+    return firstHaltException;
+  }
+
   /**
    * Reset the tracking of process termination. This is for use
    * in unit tests where one test in the suite expects an exit
@@ -73,6 +107,9 @@ public final class ExitUtil {
     firstExitException = null;
   }
 
+  public static void resetFirstHaltException() {
+    firstHaltException = null;
+  }
   /**
    * Clear the previous exit record.
    */
@@ -84,7 +121,7 @@ public final class ExitUtil {
    * Terminate the current process. Note that terminate is the *only* method
    * that should be used to terminate the daemon processes.
    * @param status exit code
-   * @param msg message used to create the ExitException
+   * @param msg message used to create the {@code ExitException}
    * @throws ExitException if System.exit is disabled for test purposes
    */
   public static void terminate(int status, String msg) throws ExitException {
@@ -100,6 +137,25 @@ public final class ExitUtil {
     System.exit(status);
   }
 
+  /**
+   * Forcibly terminates the currently running Java virtual machine.
+   * @param status exit code
+   * @param msg message used to create the {@code HaltException}
+   * @throws HaltException if Runtime.getRuntime().halt() is disabled for test purposes
+   */
+  public static void halt(int status, String msg) throws HaltException {
+    LOG.info("Halt with status " + status + " Message: " + msg);
+    if (systemHaltDisabled) {
+      HaltException ee = new HaltException(status, msg);
+      LOG.fatal("Halt called", ee);
+      if (null == firstHaltException) {
+        firstHaltException = ee;
+      }
+      throw ee;
+    }
+    Runtime.getRuntime().halt(status);
+  }
+
   /**
    * Like {@link terminate(int, String)} but uses the given throwable to
    * initialize the ExitException.
@@ -110,7 +166,15 @@ public final class ExitUtil {
   public static void terminate(int status, Throwable t) throws ExitException {
     terminate(status, StringUtils.stringifyException(t));
   }
-
+  /**
+   * Forcibly terminates the currently running Java virtual machine.
+   * @param status
+   * @param t
+   * @throws ExitException
+   */
+  public static void halt(int status, Throwable t) throws HaltException {
+    halt(status, StringUtils.stringifyException(t));
+  }
   /**
    * Like {@link terminate(int, String)} without a message.
    * @param status
@@ -119,4 +183,12 @@ public final class ExitUtil {
   public static void terminate(int status) throws ExitException {
     terminate(status, "ExitException");
   }
+  /**
+   * Forcibly terminates the currently running Java virtual machine.
+   * @param status
+   * @throws ExitException
+   */
+  public static void halt(int status) throws HaltException {
+    halt(status, "HaltException");
+  }
 }

+ 3 - 0
hadoop-yarn-project/CHANGES.txt

@@ -7,6 +7,9 @@ Release 0.23.8 - UNRELEASED
   NEW FEATURES
   
   IMPROVEMENTS
+
+    YARN-548. Add tests for YarnUncaughtExceptionHandler (Vadim Bondarev via
+    jeagles)
   
   OPTIMIZATIONS
  

+ 3 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/YarnUncaughtExceptionHandler.java

@@ -22,6 +22,7 @@ import java.lang.Thread.UncaughtExceptionHandler;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.util.ExitUtil;
 import org.apache.hadoop.util.ShutdownHookManager;
 
 /**
@@ -55,9 +56,9 @@ public class YarnUncaughtExceptionHandler implements UncaughtExceptionHandler {
         } catch (Throwable err) {
           //Again we done want to exit because of logging issues.
         }
-        Runtime.getRuntime().halt(-1);
+        ExitUtil.halt(-1);
       } else {
-        System.exit(-1);
+        ExitUtil.terminate(-1);
       }
     } else {
       LOG.error("Thread " + t + " threw an Exception.", e);

+ 115 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/TestYarnUncaughtExceptionHandler.java

@@ -0,0 +1,115 @@
+/**
+ * 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.hadoop.yarn;
+
+import static org.junit.Assert.assertSame;
+import static org.mockito.Mockito.spy;
+import static org.mockito.Mockito.verify;
+
+import org.apache.hadoop.util.ExitUtil;
+import org.junit.Test;
+
+public class TestYarnUncaughtExceptionHandler {
+
+  private static final YarnUncaughtExceptionHandler exHandler =
+        new YarnUncaughtExceptionHandler();
+  /**
+   * Throw {@code YarnException} inside thread and
+   * check {@code YarnUncaughtExceptionHandler} instance
+   *
+   * @throws InterruptedException
+   */
+  @Test
+  public void testUncaughtExceptionHandlerWithRuntimeException()
+      throws InterruptedException {
+    final YarnUncaughtExceptionHandler spyYarnHandler = spy(exHandler);
+    final YarnException yarnException = new YarnException(
+        "test-yarn-runtime-exception");
+    final Thread yarnThread = new Thread(new Runnable() {
+      @Override
+      public void run() {
+        throw yarnException;
+      }
+    });
+
+    yarnThread.setUncaughtExceptionHandler(spyYarnHandler);
+    assertSame(spyYarnHandler, yarnThread.getUncaughtExceptionHandler());
+    yarnThread.start();
+    yarnThread.join();
+    verify(spyYarnHandler).uncaughtException(yarnThread, yarnException);
+  }
+
+  /**
+   * <p>
+   * Throw {@code Error} inside thread and
+   * check {@code YarnUncaughtExceptionHandler} instance
+   * <p>
+   * Used {@code ExitUtil} class to avoid jvm exit through
+   * {@code System.exit(-1) }
+   *
+   * @throws InterruptedException
+   */
+  @Test
+  public void testUncaughtExceptionHandlerWithError()
+      throws InterruptedException {
+    ExitUtil.disableSystemExit();
+    final YarnUncaughtExceptionHandler spyErrorHandler = spy(exHandler);
+    final java.lang.Error error = new java.lang.Error("test-error");
+    final Thread errorThread = new Thread(new Runnable() {
+      @Override
+      public void run() {
+        throw error;
+      }
+    });
+    errorThread.setUncaughtExceptionHandler(spyErrorHandler);
+    assertSame(spyErrorHandler, errorThread.getUncaughtExceptionHandler());
+    errorThread.start();
+    errorThread.join();
+    verify(spyErrorHandler).uncaughtException(errorThread, error);
+  }
+
+  /**
+   * <p>
+   * Throw {@code OutOfMemoryError} inside thread and
+   * check {@code YarnUncaughtExceptionHandler} instance
+   * <p>
+   * Used {@code ExitUtil} class to avoid jvm exit through
+   * {@code Runtime.getRuntime().halt(-1)}
+   *
+   * @throws InterruptedException
+   */
+  @Test
+  public void testUncaughtExceptionHandlerWithOutOfMemoryError()
+      throws InterruptedException {
+    ExitUtil.disableSystemHalt();
+    final YarnUncaughtExceptionHandler spyOomHandler = spy(exHandler);
+    final OutOfMemoryError oomError = new OutOfMemoryError("out-of-memory-error");
+    final Thread oomThread = new Thread(new Runnable() {
+      @Override
+      public void run() {
+        throw oomError;
+      }
+    });
+    oomThread.setUncaughtExceptionHandler(spyOomHandler);
+    assertSame(spyOomHandler, oomThread.getUncaughtExceptionHandler());
+    oomThread.start();
+    oomThread.join();
+    verify(spyOomHandler).uncaughtException(oomThread, oomError);
+  }
+}