You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by in...@apache.org on 2017/05/02 22:05:34 UTC

[17/50] [abbrv] hadoop git commit: YARN-679. Add an entry point that can start any Yarn service. Contributed by Steve Loughran.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/373bb493/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/service/launcher/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/service/launcher/package-info.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/service/launcher/package-info.java
new file mode 100644
index 0000000..8516357
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/service/launcher/package-info.java
@@ -0,0 +1,462 @@
+/*
+ * 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.
+ */
+
+/**
+
+ This package contains classes, interfaces and exceptions to launch
+ YARN services from the command line.
+
+ <h2>Key Features</h2>
+
+ <p>
+ <b>General purpose YARN service launcher</b>:<p>
+ The {@link org.apache.hadoop.service.launcher.ServiceLauncher} class parses
+ a command line, then instantiates and launches the specified YARN service. It
+ then waits for the service to finish, converting any exceptions raised or
+ exit codes returned into an exit code for the (then exited) process. 
+ <p>
+ This class is designed be invokable from the static 
+ {@link org.apache.hadoop.service.launcher.ServiceLauncher#main(String[])}
+ method, or from {@code main(String[])} methods implemented by
+ other classes which provide their own entry points.
+  
+
+ <p>
+ <b>Extended YARN Service Interface</b>:<p>
+ The {@link org.apache.hadoop.service.launcher.LaunchableService} interface
+ extends {@link org.apache.hadoop.service.Service} with methods to pass
+ down the CLI arguments and to execute an operation without having to
+ spawn a thread in the  {@link org.apache.hadoop.service.Service#start()} phase.
+  
+
+ <p>
+ <b>Standard Exit codes</b>:<p>
+ {@link org.apache.hadoop.service.launcher.LauncherExitCodes}
+ defines a set of exit codes that can be used by services to standardize
+ exit causes.
+
+ <p>
+ <b>Escalated shutdown</b>:<p>
+ The {@link org.apache.hadoop.service.launcher.ServiceShutdownHook}
+ shuts down any service via the hadoop shutdown mechanism.
+ The {@link org.apache.hadoop.service.launcher.InterruptEscalator} can be
+ registered to catch interrupts, triggering the shutdown -and forcing a JVM
+ exit if it times out or a second interrupt is received.
+
+ <p><b>Tests:</b><p> test cases include interrupt handling and
+ lifecycle failures.
+
+ <h2>Launching a YARN Service</h2>
+
+ The Service Launcher can launch <i>any YARN service</i>.
+ It will instantiate the service classname provided, using the no-args
+ constructor, or if no such constructor is available, it will fall back
+ to a constructor with a single {@code String} parameter,
+ passing the service name as the parameter value.
+ <p>
+
+ The launcher will initialize the service via
+ {@link org.apache.hadoop.service.Service#init(Configuration)},
+ then start it via its {@link org.apache.hadoop.service.Service#start()} method.
+ It then waits indefinitely for the service to stop.
+ <p> 
+ After the service has stopped, a non-null value  of
+ {@link org.apache.hadoop.service.Service#getFailureCause()} is interpreted
+ as a failure, and, if it didn't happen during the stop phase (i.e. when
+ {@link org.apache.hadoop.service.Service#getFailureState()} is not
+ {@code STATE.STOPPED}, escalated into a non-zero return code).
+ <p>
+ 
+ To view the workflow in sequence, it is:
+ <ol>
+ <li>(prepare configuration files &mdash;covered later)</li>
+ <li>instantiate service via its empty or string constructor</li>
+ <li>call {@link org.apache.hadoop.service.Service#init(Configuration)}</li>
+ <li>call {@link org.apache.hadoop.service.Service#start()}</li>
+ <li>call
+   {@link org.apache.hadoop.service.Service#waitForServiceToStop(long)}</li>
+ <li>If an exception was raised: propagate it</li>
+ <li>If an exception was recorded in
+ {@link org.apache.hadoop.service.Service#getFailureCause()}
+ while the service was running: propagate it.</li>
+ </ol>
+
+ For a service to be fully compatible with this launch model, it must
+ <ul>
+ <li>Start worker threads, processes and executors in its
+ {@link org.apache.hadoop.service.Service#start()} method</li>
+ <li>Terminate itself via a call to
+ {@link org.apache.hadoop.service.Service#stop()}
+ in one of these asynchronous methods.</li>
+ </ul>
+
+ If a service does not stop itself, <i>ever</i>, then it can be launched
+ as a long-lived daemon.
+ The service launcher will never terminate, but neither will the service.
+ The service launcher does register signal handlers to catch {@code kill}
+ and control-C signals &mdash;calling {@code stop()} on the service when
+ signaled.
+ This means that a daemon service <i>may</i> get a warning and time to shut
+ down.
+
+ <p>
+ To summarize: provided a service launches its long-lived threads in its Service
+ {@code start()} method, the service launcher can create it, configure it
+ and start it, triggering shutdown when signaled.
+
+ What these services can not do is get at the command line parameters or easily
+ propagate exit codes (there is a way covered later). These features require
+ some extensions to the base {@code Service} interface: <i>the Launchable
+ Service</i>.
+
+ <h2>Launching a Launchable YARN Service</h2>
+
+ A Launchable YARN Service is a YARN service which implements the interface
+ {@link org.apache.hadoop.service.launcher.LaunchableService}. 
+ <p>
+ It adds two methods to the service interface &mdash;and hence two new features:
+
+ <ol>
+ <li>Access to the command line passed to the service launcher </li>
+ <li>A blocking {@code int execute()} method which can return the exit
+ code for the application.</li>
+ </ol>
+
+ This design is ideal for implementing services which parse the command line,
+ and which execute short-lived applications. For example, end user 
+ commands can be implemented as such services, thus integrating with YARN's
+ workflow and {@code YarnClient} client-side code.  
+
+ <p>
+ It can just as easily be used for implementing long-lived services that
+ parse the command line -it just becomes the responsibility of the
+ service to decide when to return from the {@code execute()} method.
+ It doesn't even need to {@code stop()} itself; the launcher will handle
+ that if necessary.
+ <p>
+ The {@link org.apache.hadoop.service.launcher.LaunchableService} interface
+ extends {@link org.apache.hadoop.service.Service} with two new methods.
+
+ <p>
+ {@link org.apache.hadoop.service.launcher.LaunchableService#bindArgs(Configuration, List)}
+ provides the {@code main(String args[])} arguments as a list, after any
+ processing by the Service Launcher to extract configuration file references.
+ This method <i>is called before
+ {@link org.apache.hadoop.service.Service#init(Configuration)}.</i>
+ This is by design: it allows the arguments to be parsed before the service is
+ initialized, thus allowing services to tune their configuration data before
+ passing it to any superclass in that {@code init()} method.
+ To make this operation even simpler, the
+ {@link org.apache.hadoop.conf.Configuration} that is to be passed in
+ is provided as an argument.
+ This reference passed in is the initial configuration for this service;
+ the one that will be passed to the init operation.
+
+ In
+ {@link org.apache.hadoop.service.launcher.LaunchableService#bindArgs(Configuration, List)},
+ a Launchable Service may manipulate this configuration by setting or removing
+ properties. It may also create a new {@code Configuration} instance
+ which may be needed to trigger the injection of HDFS or YARN resources
+ into the default resources of all Configurations.
+ If the return value of the method call is a configuration
+ reference (as opposed to a null value), the returned value becomes that
+ passed in to the {@code init()} method.
+ <p>
+ After the {@code bindArgs()} processing, the service's {@code init()}
+ and {@code start()} methods are called, as usual.
+ <p>
+ At this point, rather than block waiting for the service to terminate (as
+ is done for a basic service), the method
+ {@link org.apache.hadoop.service.launcher.LaunchableService#execute()}
+ is called.
+ This is a method expected to block until completed, returning the intended 
+ application exit code of the process when it does so. 
+ <p> 
+ After this {@code execute()} operation completes, the
+ service is stopped and exit codes generated. Any exception raised
+ during the {@code execute()} method takes priority over any exit codes
+ returned by the method. This allows services to signal failures simply
+ by raising exceptions with exit codes.
+ <p>
+
+ <p>
+ To view the workflow in sequence, it is:
+ <ol>
+ <li>(prepare configuration files &mdash;covered later)</li>
+ <li>instantiate service via its empty or string constructor</li>
+ <li>call {@link org.apache.hadoop.service.launcher.LaunchableService#bindArgs(Configuration, List)}</li>
+ <li>call {@link org.apache.hadoop.service.Service#init(Configuration)} with the existing config,
+  or any new one returned by
+  {@link org.apache.hadoop.service.launcher.LaunchableService#bindArgs(Configuration, List)}</li>
+ <li>call {@link org.apache.hadoop.service.Service#start()}</li>
+ <li>call {@link org.apache.hadoop.service.launcher.LaunchableService#execute()}</li>
+ <li>call {@link org.apache.hadoop.service.Service#stop()}</li>
+ <li>The return code from
+  {@link org.apache.hadoop.service.launcher.LaunchableService#execute()}
+  becomes the exit code of the process, unless overridden by an exception.</li>
+ <li>If an exception was raised in this workflow: propagate it</li>
+ <li>If an exception was recorded in
+  {@link org.apache.hadoop.service.Service#getFailureCause()}
+  while the service was running: propagate it.</li>
+ </ol>
+
+
+ <h2>Exit Codes and Exceptions</h2>
+
+ <p>
+ For a basic service, the return code is 0 unless an exception
+ was raised. 
+ <p>
+ For a {@link org.apache.hadoop.service.launcher.LaunchableService}, the return
+ code is the number returned from the
+ {@link org.apache.hadoop.service.launcher.LaunchableService#execute()}
+ operation, again, unless overridden an exception was raised.
+
+ <p>
+ Exceptions are converted into exit codes -but rather than simply
+ have a "something went wrong" exit code, exceptions <i>may</i>
+ provide exit codes which will be extracted and used as the return code.
+ This enables Launchable Services to use exceptions as a way
+ of returning error codes to signal failures and for
+ normal Services to return any error code at all.
+
+ <p>
+ Any exception which implements the
+ {@link org.apache.hadoop.util.ExitCodeProvider}
+ interface is considered be a provider of the exit code: the method
+ {@link org.apache.hadoop.util.ExitCodeProvider#getExitCode()}
+ will be called on the caught exception to generate the return code.
+ This return code and the message in the exception will be used to
+ generate an instance of
+ {@link org.apache.hadoop.util.ExitUtil.ExitException}
+ which can be passed down to
+ {@link org.apache.hadoop.util.ExitUtil#terminate(ExitUtil.ExitException)}
+ to trigger a JVM exit. The initial exception will be used as the cause
+ of the {@link org.apache.hadoop.util.ExitUtil.ExitException}.
+
+ <p>
+ If the exception is already an instance or subclass of 
+ {@link org.apache.hadoop.util.ExitUtil.ExitException}, it is passed
+ directly to
+ {@link org.apache.hadoop.util.ExitUtil#terminate(ExitUtil.ExitException)}
+ without any conversion.
+ One such subclass,
+ {@link org.apache.hadoop.service.launcher.ServiceLaunchException}
+ may be useful: it includes formatted exception message generation. 
+ It also declares that it extends the
+ {@link org.apache.hadoop.service.launcher.LauncherExitCodes}
+ interface listing common exception codes. These are exception codes
+ that can be raised by the {@link org.apache.hadoop.service.launcher.ServiceLauncher}
+ itself to indicate problems during parsing the command line, creating
+ the service instance and the like. There are also some common exit codes
+ for Hadoop/YARN service failures, such as
+ {@link org.apache.hadoop.service.launcher.LauncherExitCodes#EXIT_UNAUTHORIZED}.
+ Note that {@link org.apache.hadoop.util.ExitUtil.ExitException} itself
+ implements {@link org.apache.hadoop.util.ExitCodeProvider#getExitCode()}
+
+ <p>
+ If an exception does not implement
+ {@link org.apache.hadoop.util.ExitCodeProvider#getExitCode()},
+ it will be wrapped in an {@link org.apache.hadoop.util.ExitUtil.ExitException}
+ with the exit code
+ {@link org.apache.hadoop.service.launcher.LauncherExitCodes#EXIT_EXCEPTION_THROWN}.
+
+ <p>
+ To view the exit code extraction in sequence, it is:
+ <ol>
+ <li>If no exception was triggered by a basic service, a
+ {@link org.apache.hadoop.service.launcher.ServiceLaunchException} with an
+ exit code of 0 is created.</li>
+
+ <li>For a LaunchableService, the exit code is the result of {@code execute()}
+ Again, a {@link org.apache.hadoop.service.launcher.ServiceLaunchException}
+ with a return code of 0 is created.
+ </li>
+
+ <li>Otherwise, if the exception is an instance of {@code ExitException},
+ it is returned as the service terminating exception.</li>
+
+ <li>If the exception implements {@link org.apache.hadoop.util.ExitCodeProvider},
+ its exit code and {@code getMessage()} value become the exit exception.</li>
+
+ <li>Otherwise, it is wrapped as a
+ {@link org.apache.hadoop.service.launcher.ServiceLaunchException}
+ with the exit code
+ {@link org.apache.hadoop.service.launcher.LauncherExitCodes#EXIT_EXCEPTION_THROWN}
+ to indicate that an exception was thrown.</li>
+
+ <li>This is finally passed to
+ {@link org.apache.hadoop.util.ExitUtil#terminate(ExitUtil.ExitException)},
+ by way of
+ {@link org.apache.hadoop.service.launcher.ServiceLauncher#exit(ExitUtil.ExitException)};
+ a method designed to allow subclasses to override for testing.</li>
+
+ <li>The {@link org.apache.hadoop.util.ExitUtil} class then terminates the JVM
+ with the specified exit code, printing the {@code toString()} value
+ of the exception if the return code is non-zero.</li>
+ </ol>
+
+ This process may seem convoluted, but it is designed to allow any exception
+ in the Hadoop exception hierarchy to generate exit codes,
+ and to minimize the amount of exception wrapping which takes place.
+
+ <h2>Interrupt Handling</h2>
+
+ The Service Launcher has a helper class,
+ {@link org.apache.hadoop.service.launcher.InterruptEscalator}
+ to handle the standard SIGKILL signal and control-C signals.
+ This class registers for signal callbacks from these signals, and,
+ when received, attempts to stop the service in a limited period of time.
+ It then triggers a JVM shutdown by way of
+ {@link org.apache.hadoop.util.ExitUtil#terminate(int, String)}
+ <p>
+ If a second signal is received, the
+ {@link org.apache.hadoop.service.launcher.InterruptEscalator}
+ reacts by triggering an immediate JVM halt, invoking 
+ {@link org.apache.hadoop.util.ExitUtil#halt(int, String)}. 
+ This escalation process is designed to address the situation in which
+ a shutdown-hook can block, yet the caller (such as an init.d daemon)
+ wishes to kill the process.
+ The shutdown script should repeat the kill signal after a chosen time period,
+ to trigger the more aggressive process halt. The exit code will always be
+ {@link org.apache.hadoop.service.launcher.LauncherExitCodes#EXIT_INTERRUPTED}.
+ <p>
+ The {@link org.apache.hadoop.service.launcher.ServiceLauncher} also registers
+ a {@link org.apache.hadoop.service.launcher.ServiceShutdownHook} with the
+ Hadoop shutdown hook manager, unregistering it afterwards. This hook will
+ stop the service if a shutdown request is received, so ensuring that
+ if the JVM is exited by any thread, an attempt to shut down the service
+ will be made.
+ 
+
+ <h2>Configuration class creation</h2>
+
+ The Configuration class used to initialize a service is a basic
+ {@link org.apache.hadoop.conf.Configuration} instance. As the launcher is
+ the entry point for an application, this implies that the HDFS, YARN or other
+ default configurations will not have been forced in through the constructors
+ of {@code HdfsConfiguration} or {@code YarnConfiguration}.
+ <p>
+ What the launcher does do is use reflection to try and create instances of
+ these classes simply to force in the common resources. If the classes are
+ not on the classpath this fact will be logged.
+ <p>
+ Applications may consider it essential to either force load in the relevant
+ configuration, or pass it down to the service being created. In which
+ case further measures may be needed.
+ 
+ <p><b>1: Creation in an extended {@code ServiceLauncher}</b>
+ 
+ <p>
+ Subclass the Service launcher and override its
+ {@link org.apache.hadoop.service.launcher.ServiceLauncher#createConfiguration()}
+ method with one that creates the right configuration.
+ This is good if a single
+ launcher can be created for all services launched by a module, such as
+ HDFS or YARN. It does imply a dedicated script to invoke the custom
+ {@code main()} method.
+
+ <p><b>2: Creation in {@code bindArgs()}</b>
+
+ <p>
+ In
+ {@link org.apache.hadoop.service.launcher.LaunchableService#bindArgs(Configuration, List)},
+ a new configuration is created:
+
+ <pre>
+ public Configuration bindArgs(Configuration config, List<String> args)
+    throws Exception {
+   Configuration newConf = new YarnConfiguration(config);
+   return newConf;
+ }
+ </pre>
+
+ This guarantees a configuration of the right type is generated for all
+ instances created via the service launcher. It does imply that this is
+ expected to be only way that services will be launched.
+
+ <p><b>3: Creation in {@code serviceInit()}</b>
+
+ <pre>
+ protected void serviceInit(Configuration conf) throws Exception {
+   super.serviceInit(new YarnConfiguration(conf));
+ }
+ </pre>
+
+ <p>
+ This is a strategy used by many existing YARN services, and is ideal for
+ services which do not implement the LaunchableService interface. Its one
+ weakness is that the configuration is now private to that instance. Some
+ YARN services use a single shared configuration instance as a way of
+ propagating information between peer services in a
+ {@link org.apache.hadoop.service.CompositeService}.
+ While a dangerous practice, it does happen.
+
+
+ <b>Summary</b>: the ServiceLauncher makes a best-effort attempt to load the
+ standard Configuration subclasses, but does not fail if they are not present.
+ Services which require a specific subclasses should follow one of the
+ strategies listed;
+ creation in {@code serviceInit()} is the recommended policy.
+ 
+ <h2>Configuration file loading</h2>
+
+ Before the service is bound to the CLI, the ServiceLauncher scans through
+ all the arguments after the first one, looking for instances of the argument
+ {@link org.apache.hadoop.service.launcher.ServiceLauncher#ARG_CONF}
+ argument pair: {@code --conf &lt;file&gt;}. This must refer to a file
+ in the local filesystem which exists.
+ <p>
+ It will be loaded into the Hadoop configuration
+ class (the one created by the
+ {@link org.apache.hadoop.service.launcher.ServiceLauncher#createConfiguration()}
+ method.
+ If this argument is repeated multiple times, all configuration
+ files are merged with the latest file on the command line being the
+ last one to be applied.
+ <p>
+ All the {@code --conf &lt;file&gt;} argument pairs are stripped off
+ the argument list provided to the instantiated service; they get the
+ merged configuration, but not the commands used to create it.
+
+ <h2>Utility Classes</h2>
+
+ <ul>
+
+ <li>
+ {@link org.apache.hadoop.service.launcher.IrqHandler}: registers interrupt
+ handlers using {@code sun.misc} APIs.
+ </li>
+
+ <li>
+ {@link org.apache.hadoop.service.launcher.ServiceLaunchException}: a
+ subclass of {@link org.apache.hadoop.util.ExitUtil.ExitException} which
+ takes a String-formatted format string and a list of arguments to create
+ the exception text.
+ </li>
+
+ </ul>
+ */
+
+
+package org.apache.hadoop.service.launcher;
+
+import java.util.List;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.util.ExitUtil;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/373bb493/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ExitCodeProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ExitCodeProvider.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ExitCodeProvider.java
new file mode 100644
index 0000000..0424ba0
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ExitCodeProvider.java
@@ -0,0 +1,35 @@
+/*
+ * 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.util;
+
+/**
+ * Get the exit code of an exception.
+ * Making it an interface makes
+ * it possible to retrofit exit codes onto existing classes,
+ * and add exit code providers under all parts of the Exception tree.
+ */
+
+public interface ExitCodeProvider {
+
+  /**
+   * Method to get the exit code.
+   * @return the exit code
+   */
+  int getExitCode();
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/373bb493/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ExitUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ExitUtil.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ExitUtil.java
index 5208927..5642a23 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ExitUtil.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ExitUtil.java
@@ -17,41 +17,123 @@
  */
 package org.apache.hadoop.util;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
- * Facilitates hooking process termination for tests and debugging.
+ * Facilitates hooking process termination for tests, debugging
+ * and embedding.
+ * 
+ * Hadoop code that attempts to call {@link System#exit(int)} 
+ * or {@link Runtime#halt(int)} MUST invoke it via these methods.
  */
-@InterfaceAudience.LimitedPrivate({"HDFS", "MapReduce"})
+@InterfaceAudience.LimitedPrivate({"HDFS", "MapReduce", "YARN"})
 @InterfaceStability.Unstable
 public final class ExitUtil {
-  private final static Log LOG = LogFactory.getLog(ExitUtil.class.getName());
+  private static final Logger
+      LOG = LoggerFactory.getLogger(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 ExitUtil() {
+  }
+
+  /**
+   * An exception raised when a call to {@link #terminate(int)} was
+   * called and system exits were blocked.
+   */
+  public static class ExitException extends RuntimeException
+      implements ExitCodeProvider {
     private static final long serialVersionUID = 1L;
+    /**
+     * The status code.
+     */
     public final int status;
 
     public ExitException(int status, String msg) {
       super(msg);
       this.status = status;
     }
+
+    public ExitException(int status,
+        String message,
+        Throwable cause) {
+      super(message, cause);
+      this.status = status;
+    }
+
+    public ExitException(int status, Throwable cause) {
+      super(cause);
+      this.status = status;
+    }
+
+    @Override
+    public int getExitCode() {
+      return status;
+    }
+
+    /**
+     * String value does not include exception type, just exit code and message.
+     * @return the exit code and any message
+     */
+    @Override
+    public String toString() {
+      String message = getMessage();
+      if (message == null) {
+        message = super.toString();
+      }
+      return Integer.toString(status) + ": " + message;
+    }
   }
 
-  public static class HaltException extends RuntimeException {
+  /**
+   * An exception raised when a call to {@link #terminate(int)} was
+   * called and system halts were blocked.
+   */
+  public static class HaltException extends RuntimeException
+      implements ExitCodeProvider {
     private static final long serialVersionUID = 1L;
     public final int status;
 
+    public HaltException(int status, Throwable cause) {
+      super(cause);
+      this.status = status;
+    }
+
     public HaltException(int status, String msg) {
       super(msg);
       this.status = status;
     }
+
+    public HaltException(int status,
+        String message,
+        Throwable cause) {
+      super(message, cause);
+      this.status = status;
+    }
+
+    @Override
+    public int getExitCode() {
+      return status;
+    }
+
+    /**
+     * String value does not include exception type, just exit code and message.
+     * @return the exit code and any message
+     */
+    @Override
+    public String toString() {
+      String message = getMessage();
+      if (message == null) {
+        message = super.toString();
+      }
+      return Integer.toString(status) + ": " + message;
+    }
+
   }
 
   /**
@@ -69,7 +151,7 @@ public final class ExitUtil {
   }
 
   /**
-   * @return true if terminate has been called
+   * @return true if terminate has been called.
    */
   public static boolean terminateCalled() {
     // Either we set this member or we actually called System#exit
@@ -77,21 +159,21 @@ public final class ExitUtil {
   }
 
   /**
-   * @return true if halt has been called
+   * @return true if halt has been called.
    */
   public static boolean haltCalled() {
     return firstHaltException != null;
   }
 
   /**
-   * @return the first ExitException thrown, null if none thrown yet
+   * @return the first ExitException thrown, null if none thrown yet.
    */
   public static ExitException getFirstExitException() {
     return firstExitException;
   }
 
   /**
-   * @return the first {@code HaltException} thrown, null if none thrown yet
+   * @return the first {@code HaltException} thrown, null if none thrown yet.
    */
   public static HaltException getFirstHaltException() {
     return firstHaltException;
@@ -110,22 +192,22 @@ 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 {@code ExitException}
-   * @throws ExitException
-   *           if System.exit is disabled for test purposes
+   * Inner termination: either exit with the exception's exit code,
+   * or, if system exits are disabled, rethrow the exception.
+   * @param ee exit exception
    */
-  public static void terminate(int status, String msg) throws ExitException {
-    LOG.info("Exiting with status " + status);
+  public static synchronized void terminate(ExitException ee)
+      throws ExitException {
+    int status = ee.getExitCode();
+    String msg = ee.getMessage();
+    if (status != 0) {
+      //exit indicates a problem, log it
+      LOG.debug("Exiting with status {}: {}",  status, msg, ee);
+      LOG.info("Exiting with status {}: {}", status, msg);
+    }
     if (systemExitDisabled) {
-      ExitException ee = new ExitException(status, msg);
-      LOG.fatal("Terminate called", ee);
-      if (null == firstExitException) {
+      LOG.error("Terminate called", ee);
+      if (!terminateCalled()) {
         firstExitException = ee;
       }
       throw ee;
@@ -135,20 +217,26 @@ public final class ExitUtil {
 
   /**
    * 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);
+   * The exception argument is rethrown if JVM halting is disabled.
+   * @param ee the exception containing the status code, message and any stack
+   * trace.
+   * @throws HaltException if {@link Runtime#halt(int)} is disabled.
+   */
+  public static synchronized void halt(HaltException ee) throws HaltException {
+    int status = ee.getExitCode();
+    String msg = ee.getMessage();
+    try {
+      if (status != 0) {
+        //exit indicates a problem, log it
+        LOG.debug("Halt with status {}: {}", status, msg, ee);
+        LOG.info("Halt with status {}: {}", status, msg, msg);
+      }
+    } catch (Exception ignored) {
+      // ignore exceptions here, as it may be due to an out of memory situation
+    }
     if (systemHaltDisabled) {
-      HaltException ee = new HaltException(status, msg);
-      LOG.fatal("Halt called", ee);
-      if (null == firstHaltException) {
+      LOG.error("Halt called", ee);
+      if (!haltCalled()) {
         firstHaltException = ee;
       }
       throw ee;
@@ -157,47 +245,94 @@ public final class ExitUtil {
   }
 
   /**
-   * Like {@link terminate(int, String)} but uses the given throwable to
-   * initialize the ExitException.
-   *
-   * @param status
-   * @param t
-   *          throwable used to create the ExitException
-   * @throws ExitException
-   *           if System.exit is disabled for test purposes
+   * Like {@link #terminate(int, String)} but uses the given throwable to
+   * build the message to display or throw as an
+   * {@link ExitException}.
+   * <p>
+   * @param status exit code to use if the exception is not an ExitException.
+   * @param t throwable which triggered the termination. If this exception
+   * is an {@link ExitException} its status overrides that passed in.
+   * @throws ExitException if {@link System#exit(int)}  is disabled.
    */
   public static void terminate(int status, Throwable t) throws ExitException {
-    terminate(status, StringUtils.stringifyException(t));
+    if (t instanceof ExitException) {
+      terminate((ExitException) t);
+    } else {
+      terminate(new ExitException(status, t));
+    }
   }
 
   /**
    * Forcibly terminates the currently running Java virtual machine.
    *
-   * @param status
-   * @param t
-   * @throws ExitException
+   * @param status exit code to use if the exception is not a HaltException.
+   * @param t throwable which triggered the termination. If this exception
+   * is a {@link HaltException} its status overrides that passed in.
+   * @throws HaltException if {@link System#exit(int)}  is disabled.
    */
   public static void halt(int status, Throwable t) throws HaltException {
-    halt(status, StringUtils.stringifyException(t));
+    if (t instanceof HaltException) {
+      halt((HaltException) t);
+    } else {
+      halt(new HaltException(status, t));
+    }
   }
 
   /**
-   * Like {@link terminate(int, String)} without a message.
+   * Like {@link #terminate(int, Throwable)} without a message.
    *
-   * @param status
-   * @throws ExitException
-   *           if System.exit is disabled for test purposes
+   * @param status exit code
+   * @throws ExitException if {@link System#exit(int)} is disabled.
    */
   public static void terminate(int status) throws ExitException {
-    terminate(status, "ExitException");
+    terminate(status, "");
+  }
+
+  /**
+   * 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 {@code ExitException}
+   * @throws ExitException if {@link System#exit(int)} is disabled.
+   */
+  public static void terminate(int status, String msg) throws ExitException {
+    terminate(new ExitException(status, msg));
   }
 
   /**
    * Forcibly terminates the currently running Java virtual machine.
-   * @param status
-   * @throws ExitException
+   * @param status status code
+   * @throws HaltException if {@link Runtime#halt(int)} is disabled.
    */
   public static void halt(int status) throws HaltException {
-    halt(status, "HaltException");
+    halt(status, "");
+  }
+
+  /**
+   * Forcibly terminates the currently running Java virtual machine.
+   * @param status status code
+   * @param message message
+   * @throws HaltException if {@link Runtime#halt(int)} is disabled.
+   */
+  public static void halt(int status, String message) throws HaltException {
+    halt(new HaltException(status, message));
+  }
+
+  /**
+   * Handler for out of memory events -no attempt is made here
+   * to cleanly shutdown or support halt blocking; a robust
+   * printing of the event to stderr is all that can be done.
+   * @param oome out of memory event
+   */
+  public static void haltOnOutOfMemory(OutOfMemoryError oome) {
+    //After catching an OOM java says it is undefined behavior, so don't
+    //even try to clean up or we can get stuck on shutdown.
+    try {
+      System.err.println("Halting due to Out Of Memory Error...");
+    } catch (Throwable err) {
+      //Again we done want to exit because of logging issues.
+    }
+    Runtime.getRuntime().halt(-1);
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/373bb493/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/GenericOptionsParser.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/GenericOptionsParser.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/GenericOptionsParser.java
index cd1fc83..835206a 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/GenericOptionsParser.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/GenericOptionsParser.java
@@ -15,9 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.util;
-
-import java.io.File;
+package org.apache.hadoop.util;import java.io.File;
 import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.io.PrintStream;
@@ -118,6 +116,7 @@ public class GenericOptionsParser {
   private static final Log LOG = LogFactory.getLog(GenericOptionsParser.class);
   private Configuration conf;
   private CommandLine commandLine;
+  private final boolean parseSuccessful;
 
   /**
    * Create an options parser with the given options to parse the args.
@@ -171,7 +170,7 @@ public class GenericOptionsParser {
   public GenericOptionsParser(Configuration conf,
       Options options, String[] args) throws IOException {
     this.conf = conf;
-    parseGeneralOptions(options, args);
+    parseSuccessful = parseGeneralOptions(options, args);
   }
 
   /**
@@ -208,58 +207,72 @@ public class GenericOptionsParser {
   }
 
   /**
-   * Specify properties of each generic option
+   * Query for the parse operation succeeding.
+   * @return true if parsing the CLI was successful
+   */
+  public boolean isParseSuccessful() {
+    return parseSuccessful;
+  }
+
+  /**
+   * Specify properties of each generic option.
+   * <i>Important</i?: as {@link OptionBuilder} is not thread safe, subclasses
+   * must synchronize use on {@code OptionBuilder.class}
    */
   @SuppressWarnings("static-access")
-  private static synchronized Options buildGeneralOptions(Options opts) {
-    Option fs = OptionBuilder.withArgName("file:///|hdfs://namenode:port")
-        .hasArg()
-        .withDescription("specify default filesystem URL to use, "
-        + "overrides 'fs.defaultFS' property from configurations.")
-        .create("fs");
-    Option jt = OptionBuilder.withArgName("local|resourcemanager:port")
-    .hasArg()
-    .withDescription("specify a ResourceManager")
-    .create("jt");
-    Option oconf = OptionBuilder.withArgName("configuration file")
-    .hasArg()
-    .withDescription("specify an application configuration file")
-    .create("conf");
-    Option property = OptionBuilder.withArgName("property=value")
-    .hasArg()
-    .withDescription("use value for given property")
-    .create('D');
-    Option libjars = OptionBuilder.withArgName("paths")
-    .hasArg()
-    .withDescription("comma separated jar files to include in the classpath.")
-    .create("libjars");
-    Option files = OptionBuilder.withArgName("paths")
-    .hasArg()
-    .withDescription("comma separated files to be copied to the " +
-           "map reduce cluster")
-    .create("files");
-    Option archives = OptionBuilder.withArgName("paths")
-    .hasArg()
-    .withDescription("comma separated archives to be unarchived" +
-                     " on the compute machines.")
-    .create("archives");
-    
-    // file with security tokens
-    Option tokensFile = OptionBuilder.withArgName("tokensFile")
-    .hasArg()
-    .withDescription("name of the file with the tokens")
-    .create("tokenCacheFile");
+  protected Options buildGeneralOptions(Options opts) {
+    synchronized (OptionBuilder.class) {
+      Option fs = OptionBuilder.withArgName("file:///|hdfs://namenode:port")
+          .hasArg()
+          .withDescription("specify default filesystem URL to use, "
+          + "overrides 'fs.defaultFS' property from configurations.")
+          .create("fs");
+      Option jt = OptionBuilder.withArgName("local|resourcemanager:port")
+          .hasArg()
+          .withDescription("specify a ResourceManager")
+          .create("jt");
+      Option oconf = OptionBuilder.withArgName("configuration file")
+          .hasArg()
+          .withDescription("specify an application configuration file")
+          .create("conf");
+      Option property = OptionBuilder.withArgName("property=value")
+          .hasArg()
+          .withDescription("use value for given property")
+          .create('D');
+      Option libjars = OptionBuilder.withArgName("paths")
+          .hasArg()
+          .withDescription(
+              "comma separated jar files to include in the classpath.")
+          .create("libjars");
+      Option files = OptionBuilder.withArgName("paths")
+          .hasArg()
+          .withDescription("comma separated files to be copied to the " +
+              "map reduce cluster")
+          .create("files");
+      Option archives = OptionBuilder.withArgName("paths")
+          .hasArg()
+          .withDescription("comma separated archives to be unarchived" +
+              " on the compute machines.")
+          .create("archives");
+
+      // file with security tokens
+      Option tokensFile = OptionBuilder.withArgName("tokensFile")
+          .hasArg()
+          .withDescription("name of the file with the tokens")
+          .create("tokenCacheFile");
+
 
-    opts.addOption(fs);
-    opts.addOption(jt);
-    opts.addOption(oconf);
-    opts.addOption(property);
-    opts.addOption(libjars);
-    opts.addOption(files);
-    opts.addOption(archives);
-    opts.addOption(tokensFile);
+      opts.addOption(fs);
+      opts.addOption(jt);
+      opts.addOption(oconf);
+      opts.addOption(property);
+      opts.addOption(libjars);
+      opts.addOption(files);
+      opts.addOption(archives);
+      opts.addOption(tokensFile);
 
-    return opts;
+      return opts;
+    }
   }
 
   /**
@@ -368,7 +381,7 @@ public class GenericOptionsParser {
   }
 
   /**
-   * takes input as a comma separated list of files
+   * Takes input as a comma separated list of files
    * and verifies if they exist. It defaults for file:///
    * if the files specified do not have a scheme.
    * it returns the paths uri converted defaulting to file:///.
@@ -543,20 +556,24 @@ public class GenericOptionsParser {
    *
    * @param opts Options to use for parsing args.
    * @param args User-specified arguments
+   * @return true if the parse was successful
    */
-  private void parseGeneralOptions(Options opts, String[] args)
+  private boolean parseGeneralOptions(Options opts, String[] args)
       throws IOException {
     opts = buildGeneralOptions(opts);
     CommandLineParser parser = new GnuParser();
+    boolean parsed = false;
     try {
       commandLine = parser.parse(opts, preProcessForWindows(args), true);
       processGeneralOptions(commandLine);
+      parsed = true;
     } catch(ParseException e) {
       LOG.warn("options parsing failed: "+e.getMessage());
 
       HelpFormatter formatter = new HelpFormatter();
       formatter.printHelp("general options are: ", opts);
     }
+    return parsed;
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/373bb493/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/StringUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/StringUtils.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/StringUtils.java
index 1517612..72bd171 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/StringUtils.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/StringUtils.java
@@ -676,11 +676,11 @@ public class StringUtils {
    * @param msg content of the message
    * @return a message for logging
    */
-  private static String toStartupShutdownString(String prefix, String [] msg) {
+  public static String toStartupShutdownString(String prefix, String[] msg) {
     StringBuilder b = new StringBuilder(prefix);
     b.append("\n/************************************************************");
     for(String s : msg)
-      b.append("\n" + prefix + s);
+      b.append("\n").append(prefix).append(s);
     b.append("\n************************************************************/");
     return b.toString();
   }
@@ -711,21 +711,7 @@ public class StringUtils {
                                      final LogAdapter LOG) { 
     final String hostname = NetUtils.getHostname();
     final String classname = clazz.getSimpleName();
-    LOG.info(
-        toStartupShutdownString("STARTUP_MSG: ", new String[] {
-            "Starting " + classname,
-            "  user = " + System.getProperty("user.name"),
-            "  host = " + hostname,
-            "  args = " + Arrays.asList(args),
-            "  version = " + VersionInfo.getVersion(),
-            "  classpath = " + System.getProperty("java.class.path"),
-            "  build = " + VersionInfo.getUrl() + " -r "
-                         + VersionInfo.getRevision()  
-                         + "; compiled by '" + VersionInfo.getUser()
-                         + "' on " + VersionInfo.getDate(),
-            "  java = " + System.getProperty("java.version") }
-        )
-      );
+    LOG.info(createStartupShutdownMessage(classname, hostname, args));
 
     if (SystemUtils.IS_OS_UNIX) {
       try {
@@ -746,6 +732,29 @@ public class StringUtils {
   }
 
   /**
+   * Generate the text for the startup/shutdown message of processes.
+   * @param classname short classname of the class
+   * @param hostname hostname
+   * @param args Command arguments
+   * @return a string to log.
+   */
+  public static String createStartupShutdownMessage(String classname,
+      String hostname, String[] args) {
+    return toStartupShutdownString("STARTUP_MSG: ", new String[] {
+        "Starting " + classname,
+        "  host = " + hostname,
+        "  args = " + Arrays.asList(args),
+        "  version = " + VersionInfo.getVersion(),
+        "  classpath = " + System.getProperty("java.class.path"),
+        "  build = " + VersionInfo.getUrl() + " -r "
+                     + VersionInfo.getRevision()  
+                     + "; compiled by '" + VersionInfo.getUser()
+                     + "' on " + VersionInfo.getDate(),
+        "  java = " + System.getProperty("java.version") }
+    );
+  }
+
+  /**
    * The traditional binary prefixes, kilo, mega, ..., exa,
    * which can be represented by a 64-bit integer.
    * TraditionalBinaryPrefix symbol are case insensitive. 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/373bb493/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/service/BreakableService.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/service/BreakableService.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/service/BreakableService.java
index eeb1a31..d84447b 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/service/BreakableService.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/service/BreakableService.java
@@ -20,8 +20,6 @@
 package org.apache.hadoop.service;
 
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.service.AbstractService;
-import org.apache.hadoop.service.Service;
 
 /**
  * This is a service that can be configured to break on any of the lifecycle
@@ -69,12 +67,21 @@ public class BreakableService extends AbstractService {
     return counts[convert(state)];
   }
 
-  private void maybeFail(boolean fail, String action) {
+  private void maybeFail(boolean fail, String action) throws Exception {
     if (fail) {
-      throw new BrokenLifecycleEvent(this, action);
+      throw createFailureException(action);
     }
   }
 
+  /**
+   * Override point: create the exception to raise
+   * @param action action in progress
+   * @return the exception that will be thrown
+   */
+  protected Exception createFailureException(String action) {
+    return new BrokenLifecycleEvent(this, action);
+  }
+
   @Override
   protected void serviceInit(Configuration conf) throws Exception {
     inc(STATE.INITED);
@@ -83,13 +90,13 @@ public class BreakableService extends AbstractService {
   }
 
   @Override
-  protected void serviceStart() {
+  protected void serviceStart() throws Exception {
     inc(STATE.STARTED);
     maybeFail(failOnStart, "start");
   }
 
   @Override
-  protected void serviceStop() {
+  protected void serviceStop() throws Exception {
     inc(STATE.STOPPED);
     maybeFail(failOnStop, "stop");
   }
@@ -107,11 +114,11 @@ public class BreakableService extends AbstractService {
   }
 
   /**
-   * The exception explicitly raised on a failure
+   * The exception explicitly raised on a failure.
    */
   public static class BrokenLifecycleEvent extends RuntimeException {
 
-    final STATE state;
+    public final STATE state;
 
     public BrokenLifecycleEvent(Service service, String action) {
       super("Lifecycle Failure during " + action + " state is "

http://git-wip-us.apache.org/repos/asf/hadoop/blob/373bb493/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/service/launcher/AbstractServiceLauncherTestBase.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/service/launcher/AbstractServiceLauncherTestBase.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/service/launcher/AbstractServiceLauncherTestBase.java
new file mode 100644
index 0000000..127b0b3
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/service/launcher/AbstractServiceLauncherTestBase.java
@@ -0,0 +1,317 @@
+/*
+ * 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.service.launcher;
+
+import org.apache.commons.lang.StringUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.service.Service;
+import org.apache.hadoop.service.ServiceOperations;
+import static org.apache.hadoop.test.GenericTestUtils.*;
+import org.apache.hadoop.util.ExitCodeProvider;
+import org.apache.hadoop.util.ExitUtil;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.rules.TestName;
+import org.junit.rules.Timeout;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.util.Arrays;
+import java.util.List;
+
+public class AbstractServiceLauncherTestBase extends Assert implements
+    LauncherExitCodes {
+  private static final Logger LOG = LoggerFactory.getLogger(
+      AbstractServiceLauncherTestBase.class);
+  public static final String CONF_FILE_DIR = "target/launcher/conf";
+
+  /**
+   * A service which will be automatically stopped on teardown.
+   */
+  private Service serviceToTeardown;
+
+  /**
+   * All tests have a short life.
+   */
+  @Rule
+  public Timeout testTimeout = new Timeout(15000);
+
+  /**
+   * Rule to provide the method name.
+   */
+  @Rule
+  public TestName methodName = new TestName();
+
+  /**
+   * Turn off the exit util JVM exits, downgrading them to exception throws.
+   */
+  @BeforeClass
+  public static void disableJVMExits() {
+    ExitUtil.disableSystemExit();
+    ExitUtil.disableSystemHalt();
+  }
+
+  /**
+   * rule to name the thread JUnit.
+   */
+  @Before
+  public void nameThread() {
+    Thread.currentThread().setName("JUnit");
+  }
+
+  @After
+  public void stopService() {
+    ServiceOperations.stopQuietly(serviceToTeardown);
+  }
+
+  public void setServiceToTeardown(Service serviceToTeardown) {
+    this.serviceToTeardown = serviceToTeardown;
+  }
+
+  /**
+   * Assert that a service is in a state.
+   * @param service service
+   * @param expected expected state
+   */
+  protected void assertInState(Service service, Service.STATE expected) {
+    assertNotNull(service);
+    Service.STATE actual = service.getServiceState();
+    failif(actual != expected,
+        "Service %s in state %s expected state: %s", service.getName(), actual, expected);
+
+  }
+
+  /**
+   * Assert a service has stopped.
+   * @param service service
+   */
+  protected void assertStopped(Service service) {
+    assertInState(service, Service.STATE.STOPPED);
+  }
+
+  /**
+   * Assert that an exception code matches the value expected.
+   * @param expected expected value
+   * @param text text in exception -can be null
+   * @param e exception providing the actual value
+   */
+  protected void assertExceptionDetails(int expected,
+      String text,
+      ExitCodeProvider e) {
+    assertNotNull(e);
+    String toString = e.toString();
+    int exitCode = e.getExitCode();
+    boolean failed = expected != exitCode;
+    failed |= StringUtils.isNotEmpty(text)
+              && !StringUtils.contains(toString, text);
+    failif(failed,
+        "Expected exception with exit code %d and text \"%s\""
+            + " but got the exit code %d"
+            + " in \"%s\"",
+        expected, text,
+        exitCode, e);
+  }
+
+  /**
+   * Assert the launch come was a service creation failure.
+   * @param classname argument
+   */
+  protected void assertServiceCreationFails(String classname) {
+    assertLaunchOutcome(EXIT_SERVICE_CREATION_FAILURE, "", classname);
+  }
+
+  /**
+   * Assert a launch outcome.
+   * @param expected expected value
+   * @param text text in exception -can be null
+   * @param args CLI args
+   */
+  protected void assertLaunchOutcome(int expected,
+      String text,
+      String... args) {
+    try {
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Launching service with expected outcome {}", expected);
+        for (String arg : args) {
+          LOG.debug(arg);
+        }
+      }
+      ServiceLauncher.serviceMain(args);
+    } catch (ServiceLaunchException e) {
+      assertExceptionDetails(expected, text, e);
+    }
+  }
+
+  /**
+   * Assert a launch runs.
+   * @param args CLI args
+   */
+  protected void assertRuns(String... args) {
+    assertLaunchOutcome(0, "", args);
+  }
+
+  /**
+   * Init and start a service.
+   * @param service the service
+   * @return the service
+   */
+  protected <S extends Service> S run(S service) {
+    assertNotNull(service);
+    service.init(new Configuration());
+    service.start();
+    return service;
+  }
+
+  /**
+   * Save a configuration to a config file in the target dir.
+   * @param conf config
+   * @return absolute path
+   * @throws IOException problems
+   */
+  protected String configFile(Configuration conf) throws IOException {
+    File directory = new File(CONF_FILE_DIR);
+    directory.mkdirs();
+    File file = File.createTempFile("conf", ".xml", directory);
+    try(OutputStream fos = new FileOutputStream(file)) {
+      conf.writeXml(fos);
+    }
+    return file.getAbsolutePath();
+  }
+
+  /**
+   * Create a new config from key-val pairs.
+   * @param kvp a list of key, value, ...
+   * @return a new configuration
+   */
+  protected Configuration newConf(String... kvp) {
+    int len = kvp.length;
+    assertEquals("unbalanced keypair len of " + len, 0, len % 2);
+    Configuration conf = new Configuration(false);
+    for (int i = 0; i < len; i += 2) {
+      conf.set(kvp[i], kvp[i + 1]);
+    }
+    return conf;
+  }
+
+  /** varargs to list conversion. */
+  protected List<String> asList(String... args) {
+    return Arrays.asList(args);
+  }
+
+  /**
+   * Launch a service with the given list of arguments. Returns
+   * the service launcher, from which the created service can be extracted
+   * via {@link ServiceLauncher#getService()}.
+   * The service is has its execute() method called, but 
+   * @param serviceClass service class to create
+   * @param conf configuration
+   * @param args list of arguments
+   * @param execute execute/wait for the service to stop
+   * @param <S> service type
+   * @return the service launcher
+   * @throws ExitUtil.ExitException if the launch's exit code != 0
+   */
+  protected <S extends Service> ServiceLauncher<S> launchService(
+      Class serviceClass,
+      Configuration conf,
+      List<String> args,
+      boolean execute) throws ExitUtil.ExitException {
+    ServiceLauncher<S> serviceLauncher =
+        new ServiceLauncher<>(serviceClass.getName());
+    ExitUtil.ExitException exitException =
+        serviceLauncher.launchService(conf, args, false, execute);
+    if (exitException.getExitCode() == 0) {
+      // success
+      return serviceLauncher;
+    } else {
+      // launch failure
+      throw exitException;
+    }
+  }
+
+  /**
+   * Launch a service with the given list of arguments. Returns
+   * the service launcher, from which the created service can be extracted.
+   * via {@link ServiceLauncher#getService()}.
+   *
+   * This call DOES NOT call {@link LaunchableService#execute()} or wait for
+   * a simple service to finish. It returns the service that has been created,
+   * initialized and started.
+   * @param serviceClass service class to create
+   * @param conf configuration
+   * @param args varargs launch arguments
+   * @param <S> service type
+   * @return the service launcher
+   * @throws ExitUtil.ExitException  if the launch's exit code != 0
+   */
+  protected <S extends Service> ServiceLauncher<S> launchService(
+      Class serviceClass,
+      Configuration conf,
+      String... args) throws ExitUtil.ExitException {
+    return launchService(serviceClass, conf, Arrays.asList(args), false);
+  }
+
+  /**
+   * Launch expecting an exception.
+   * @param serviceClass service class to create
+   * @param conf configuration
+   * @param expectedText expected text; may be "" or null
+   * @param errorCode error code 
+   * @param args varargs launch arguments
+   * @return the exception returned if there was a match
+   * @throws AssertionError on a mismatch of expectation and actual
+   */
+  protected ExitUtil.ExitException launchExpectingException(Class serviceClass,
+      Configuration conf,
+      String expectedText,
+      int errorCode,
+      String... args) {
+    try {
+      ServiceLauncher<Service> launch = launchService(serviceClass,
+          conf,
+          Arrays.asList(args),
+          true);
+
+      failf("Expected an exception with error code %d and text \"%s\" "
+              + " -but the service completed with :%s",
+          errorCode, expectedText,
+          launch.getServiceException());
+      return null;
+    } catch (ExitUtil.ExitException e) {
+      int actualCode = e.getExitCode();
+      boolean condition = errorCode != actualCode ||
+             !StringUtils.contains(e.toString(), expectedText);
+      failif(condition,
+          "Expected an exception with error code %d and text \"%s\" "
+            + " -but the service threw an exception with exit code %d: %s",
+          errorCode, expectedText,
+          actualCode, e);
+
+      return e;
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/373bb493/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/service/launcher/ExitTrackingServiceLauncher.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/service/launcher/ExitTrackingServiceLauncher.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/service/launcher/ExitTrackingServiceLauncher.java
new file mode 100644
index 0000000..855ddfd
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/service/launcher/ExitTrackingServiceLauncher.java
@@ -0,0 +1,59 @@
+/*
+ * 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.service.launcher;
+
+import org.apache.hadoop.service.Service;
+import org.apache.hadoop.util.ExitUtil;
+
+/**
+ * Service launcher for testing: The exit operation has been overloaded to
+ * record the exit exception.
+ *
+ * It relies on the test runner to have disabled exits in the
+ * {@link ExitUtil} class.
+ * @param <S> type of service to launch
+ */
+public class ExitTrackingServiceLauncher<S extends Service> extends
+    ServiceLauncher<S> {
+
+  private ExitUtil.ExitException exitException;
+
+  public ExitTrackingServiceLauncher(String serviceClassName) {
+    super(serviceClassName);
+  }
+
+  @Override
+  protected void exit(ExitUtil.ExitException ee) {
+    exitException = ee;
+    super.exit(ee);
+  }
+
+  @Override
+  protected void exit(int exitCode, String message) {
+    exit(new ServiceLaunchException(exitCode, message));
+  }
+
+  public void bindCommandOptions() {
+    super.bindCommandOptions();
+  }
+
+  public ExitUtil.ExitException getExitException() {
+    return exitException;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/373bb493/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/service/launcher/TestServiceConf.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/service/launcher/TestServiceConf.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/service/launcher/TestServiceConf.java
new file mode 100644
index 0000000..6eb6372
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/service/launcher/TestServiceConf.java
@@ -0,0 +1,146 @@
+/*
+ * 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.service.launcher;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.service.Service;
+import org.apache.hadoop.service.launcher.testservices.LaunchableRunningService;
+import org.apache.hadoop.service.launcher.testservices.RunningService;
+import static org.apache.hadoop.service.launcher.LauncherArguments.*;
+import org.junit.Test;
+
+import java.io.File;
+import java.io.FileWriter;
+import java.util.List;
+
+/**
+ * Test how configuration files are loaded off the command line.
+ */
+public class TestServiceConf
+    extends AbstractServiceLauncherTestBase {
+
+  @Test
+  public void testRunService() throws Throwable {
+    assertRuns(LaunchableRunningService.NAME);
+  }
+
+  @Test
+  public void testConfPropagationOverInitBindings() throws Throwable {
+    Configuration conf = newConf(RunningService.FAIL_IN_RUN, "true");
+    assertLaunchOutcome(EXIT_FAIL,
+        "failed",
+        LaunchableRunningService.NAME,
+        ARG_CONF_PREFIXED,
+        configFile(conf));
+  }
+
+  @Test
+  public void testUnbalancedConfArg() throws Throwable {
+    assertLaunchOutcome(EXIT_COMMAND_ARGUMENT_ERROR,
+        E_PARSE_FAILED,
+        LaunchableRunningService.NAME,
+        ARG_CONF_PREFIXED);
+  }
+
+  @Test
+  public void testConfArgMissingFile() throws Throwable {
+    assertLaunchOutcome(EXIT_COMMAND_ARGUMENT_ERROR,
+        E_PARSE_FAILED,
+        LaunchableRunningService.NAME,
+        ARG_CONF_PREFIXED,
+        "no-file.xml");
+  }
+
+  @Test
+  public void testConfPropagation() throws Throwable {
+    Configuration conf = newConf(RunningService.FAIL_IN_RUN, "true");
+    assertLaunchOutcome(EXIT_EXCEPTION_THROWN,
+        RunningService.FAILURE_MESSAGE,
+        RunningService.NAME,
+        ARG_CONF_PREFIXED,
+        configFile(conf));
+  }
+
+  /**
+   * Low level conf value extraction test...just to make sure
+   * that all works at the lower level.
+   * @throws Throwable
+   */
+  @Test
+  public void testConfExtraction() throws Throwable {
+    ExitTrackingServiceLauncher<Service> launcher =
+      new ExitTrackingServiceLauncher<>(RunningService.NAME);
+    launcher.bindCommandOptions();
+    Configuration conf = newConf("propagated", "true");
+    assertEquals("true", conf.get("propagated", "unset"));
+
+    Configuration extracted = new Configuration(false);
+
+    List<String> argsList =
+        asList("Name", ARG_CONF_PREFIXED, configFile(conf));
+    List<String> args = launcher.extractCommandOptions(extracted,
+        argsList);
+    if (!args.isEmpty()) {
+      assertEquals("args beginning with " + args.get(0),
+          0, args.size());
+    }
+    assertEquals("true", extracted.get("propagated", "unset"));
+  }
+
+  @Test
+  public void testDualConfArgs() throws Throwable {
+    ExitTrackingServiceLauncher<Service> launcher =
+        new ExitTrackingServiceLauncher<>(RunningService.NAME);
+    launcher.bindCommandOptions();
+    String key1 = "key1";
+    Configuration conf1 = newConf(key1, "true");
+    String key2 = "file2";
+    Configuration conf2 = newConf(key2, "7");
+    Configuration extracted = new Configuration(false);
+
+    List<String> argsList =
+        asList("Name",
+            ARG_CONF_PREFIXED, configFile(conf1),
+            ARG_CONF_PREFIXED, configFile(conf2));
+
+    List<String> args = launcher.extractCommandOptions(extracted, argsList);
+    if (!args.isEmpty()) {
+      assertEquals("args beginning with " + args.get(0),
+          0, args.size());
+    }
+    assertTrue(extracted.getBoolean(key1, false));
+    assertEquals(7, extracted.getInt(key2, -1));
+  }
+
+  @Test
+  public void testConfArgWrongFiletype() throws Throwable {
+    new File(CONF_FILE_DIR).mkdirs();
+    File file = new File(CONF_FILE_DIR, methodName.getMethodName());
+    try (FileWriter fileWriter = new FileWriter(file)) {
+      fileWriter.write("not-a-conf-file");
+      fileWriter.close();
+    }
+    assertLaunchOutcome(EXIT_COMMAND_ARGUMENT_ERROR,
+        "",
+        RunningService.NAME,
+        ARG_CONF_PREFIXED,
+        file.getAbsolutePath());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/373bb493/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/service/launcher/TestServiceInterruptHandling.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/service/launcher/TestServiceInterruptHandling.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/service/launcher/TestServiceInterruptHandling.java
new file mode 100644
index 0000000..bd779e4
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/service/launcher/TestServiceInterruptHandling.java
@@ -0,0 +1,118 @@
+/*
+ * 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.service.launcher;
+
+import org.apache.hadoop.service.BreakableService;
+import org.apache.hadoop.service.launcher.testservices.FailureTestService;
+import org.apache.hadoop.util.ExitUtil;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Test service launcher interrupt handling.
+ */
+public class TestServiceInterruptHandling
+    extends AbstractServiceLauncherTestBase {
+
+  private static final Logger LOG = LoggerFactory.getLogger(
+      TestServiceInterruptHandling.class);
+
+  @Test
+  public void testRegisterAndRaise() throws Throwable {
+    InterruptCatcher catcher = new InterruptCatcher();
+    String name = IrqHandler.CONTROL_C;
+    IrqHandler irqHandler = new IrqHandler(name, catcher);
+    irqHandler.bind();
+    assertEquals(0, irqHandler.getSignalCount());
+    irqHandler.raise();
+    // allow for an async event
+    Thread.sleep(500);
+    IrqHandler.InterruptData data = catcher.interruptData;
+    assertNotNull("interrupt data", data);
+    assertEquals(name, data.getName());
+    assertEquals(1, irqHandler.getSignalCount());
+  }
+
+  @Test
+  public void testInterruptEscalationShutdown() throws Throwable {
+    ExitTrackingServiceLauncher<BreakableService> launcher =
+        new ExitTrackingServiceLauncher<>(BreakableService.class.getName());
+    BreakableService service = new BreakableService();
+    launcher.setService(service);
+
+    InterruptEscalator escalator =
+        new InterruptEscalator(launcher, 500);
+
+    // call the interrupt operation directly
+    try {
+      escalator.interrupted(new IrqHandler.InterruptData("INT", 3));
+      fail("Expected an exception to be raised in " + escalator);
+    } catch (ExitUtil.ExitException e) {
+      assertExceptionDetails(EXIT_INTERRUPTED, "", e);
+    }
+    //the service is now stopped
+    assertStopped(service);
+    assertTrue("isSignalAlreadyReceived() == false in " + escalator,
+        escalator.isSignalAlreadyReceived());
+    assertFalse("isForcedShutdownTimedOut() == true in " + escalator,
+        escalator.isForcedShutdownTimedOut());
+
+    // now interrupt it a second time and expect it to escalate to a halt
+    try {
+      escalator.interrupted(new IrqHandler.InterruptData("INT", 3));
+      fail("Expected an exception to be raised in " + escalator);
+    } catch (ExitUtil.HaltException e) {
+      assertExceptionDetails(EXIT_INTERRUPTED, "", e);
+    }
+  }
+
+  @Test
+  public void testBlockingShutdownTimeouts() throws Throwable {
+    ExitTrackingServiceLauncher<FailureTestService> launcher =
+        new ExitTrackingServiceLauncher<>(FailureTestService.class.getName());
+    FailureTestService service =
+        new FailureTestService(false, false, false, 2000);
+    launcher.setService(service);
+
+    InterruptEscalator escalator = new InterruptEscalator(launcher, 500);
+    // call the interrupt operation directly
+    try {
+      escalator.interrupted(new IrqHandler.InterruptData("INT", 3));
+      fail("Expected an exception to be raised from " + escalator);
+    } catch (ExitUtil.ExitException e) {
+      assertExceptionDetails(EXIT_INTERRUPTED, "", e);
+    }
+
+    assertTrue("isForcedShutdownTimedOut() == false in " + escalator,
+        escalator.isForcedShutdownTimedOut());
+  }
+
+  private static class InterruptCatcher implements IrqHandler.Interrupted {
+
+    public IrqHandler.InterruptData interruptData;
+
+    @Override
+    public void interrupted(IrqHandler.InterruptData data) {
+      LOG.info("Interrupt caught");
+      this.interruptData = data;
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/373bb493/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/service/launcher/TestServiceLauncher.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/service/launcher/TestServiceLauncher.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/service/launcher/TestServiceLauncher.java
new file mode 100644
index 0000000..f40051b
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/service/launcher/TestServiceLauncher.java
@@ -0,0 +1,213 @@
+/*
+ * 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.service.launcher;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.service.BreakableService;
+import org.apache.hadoop.service.launcher.testservices.FailingStopInStartService;
+import org.apache.hadoop.service.launcher.testservices.InitInConstructorLaunchableService;
+import org.apache.hadoop.service.launcher.testservices.LaunchableRunningService;
+import org.apache.hadoop.service.launcher.testservices.NoArgsAllowedService;
+import org.apache.hadoop.service.launcher.testservices.NullBindLaunchableService;
+import org.apache.hadoop.service.launcher.testservices.RunningService;
+import org.apache.hadoop.service.launcher.testservices.StoppingInStartLaunchableService;
+import org.apache.hadoop.service.launcher.testservices.StringConstructorOnlyService;
+
+import static org.apache.hadoop.service.launcher.LauncherArguments.*;
+
+import static org.apache.hadoop.test.GenericTestUtils.*;
+import static org.apache.hadoop.service.launcher.testservices.ExceptionInExecuteLaunchableService.*;
+
+import org.junit.Test;
+
+public class TestServiceLauncher extends AbstractServiceLauncherTestBase {
+
+  @Test
+  public void testRunService() throws Throwable {
+    assertRuns(RunningService.NAME);
+  }
+
+  @Test
+  public void testNullBindService() throws Throwable {
+    assertRuns(NullBindLaunchableService.NAME);
+  }
+
+  @Test
+  public void testServiceLaunchStringConstructor() throws Throwable {
+    assertRuns(StringConstructorOnlyService.NAME);
+  }
+
+  /**
+   * Test the behaviour of service stop logic.
+   */
+  @Test
+  public void testStopInStartup() throws Throwable {
+    FailingStopInStartService svc = new FailingStopInStartService();
+    svc.init(new Configuration());
+    svc.start();
+    assertStopped(svc);
+    Throwable cause = svc.getFailureCause();
+    assertNotNull(cause);
+    assertTrue(cause instanceof ServiceLaunchException);
+    assertTrue(svc.waitForServiceToStop(0));
+    ServiceLaunchException e = (ServiceLaunchException) cause;
+    assertEquals(FailingStopInStartService.EXIT_CODE, e.getExitCode());
+  }
+
+  @Test
+  public void testEx() throws Throwable {
+    assertLaunchOutcome(EXIT_EXCEPTION_THROWN,
+        OTHER_EXCEPTION_TEXT,
+        NAME);
+  }
+
+  /**
+   * This test verifies that exceptions in the
+   * {@link LaunchableService#execute()} method are relayed if an instance of
+   * an exit exceptions, and forwarded if not.
+   */
+  @Test
+  public void testServiceLaunchException() throws Throwable {
+    assertLaunchOutcome(EXIT_OTHER_FAILURE,
+        SLE_TEXT,
+        NAME,
+        ARG_THROW_SLE);
+  }
+
+  @Test
+  public void testIOE() throws Throwable {
+    assertLaunchOutcome(IOE_EXIT_CODE,
+        EXIT_IN_IOE_TEXT,
+        NAME,
+        ARG_THROW_IOE);
+  }
+
+  @Test
+  public void testThrowable() throws Throwable {
+    assertLaunchOutcome(EXIT_EXCEPTION_THROWN,
+        "java.lang.OutOfMemoryError",
+        NAME,
+        ARG_THROWABLE);
+  }
+
+  /**
+   * As the exception is doing some formatting tricks, these
+   * tests verify that exception arguments are being correctly
+   * used as initializers.
+   */
+  @Test
+  public void testBasicExceptionFormatting() throws Throwable {
+    ServiceLaunchException ex = new ServiceLaunchException(0, "%03x", 32);
+    assertExceptionContains("020", ex);
+  }
+
+  @Test
+  public void testNotEnoughArgsExceptionFormatting() throws Throwable {
+    ServiceLaunchException ex = new ServiceLaunchException(0, "%03x");
+    assertExceptionContains("%03x", ex);
+  }
+
+  @Test
+  public void testInnerCause() throws Throwable {
+
+    Exception cause = new Exception("cause");
+    ServiceLaunchException ex =
+        new ServiceLaunchException(0, "%03x: %s", 32, cause);
+    assertExceptionContains("020", ex);
+    assertExceptionContains("cause", ex);
+    assertSame(cause, ex.getCause());
+  }
+
+  @Test
+  public void testInnerCauseNotInFormat() throws Throwable {
+
+    Exception cause = new Exception("cause");
+    ServiceLaunchException ex =
+        new ServiceLaunchException(0, "%03x:", 32, cause);
+    assertExceptionContains("020", ex);
+    assertFalse(ex.getMessage().contains("cause"));
+    assertSame(cause, ex.getCause());
+  }
+
+  @Test
+  public void testServiceInitInConstructor() throws Throwable {
+    assertRuns(InitInConstructorLaunchableService.NAME);
+  }
+
+  @Test
+  public void testRunNoArgsAllowedService() throws Throwable {
+    assertRuns(NoArgsAllowedService.NAME);
+  }
+
+  @Test
+  public void testNoArgsOneArg() throws Throwable {
+    assertLaunchOutcome(EXIT_COMMAND_ARGUMENT_ERROR, "1",
+        NoArgsAllowedService.NAME, "one");
+  }
+
+  @Test
+  public void testNoArgsHasConfsStripped() throws Throwable {
+    assertRuns(
+        NoArgsAllowedService.NAME,
+        LauncherArguments.ARG_CONF_PREFIXED,
+        configFile(newConf()));
+  }
+
+  @Test
+  public void testRunLaunchableService() throws Throwable {
+    assertRuns(LaunchableRunningService.NAME);
+  }
+
+  @Test
+  public void testArgBinding() throws Throwable {
+    assertLaunchOutcome(EXIT_OTHER_FAILURE,
+        "",
+        LaunchableRunningService.NAME,
+        LaunchableRunningService.ARG_FAILING);
+  }
+
+  @Test
+  public void testStoppingInStartLaunchableService() throws Throwable {
+    assertRuns(StoppingInStartLaunchableService.NAME);
+  }
+
+  @Test
+  public void testShutdownHookNullReference() throws Throwable {
+    new ServiceShutdownHook(null).run();
+  }
+
+  @Test
+  public void testShutdownHook() throws Throwable {
+    BreakableService service = new BreakableService();
+    setServiceToTeardown(service);
+    ServiceShutdownHook hook = new ServiceShutdownHook(service);
+    hook.run();
+    assertStopped(service);
+  }
+
+  @Test
+  public void testFailingHookCaught() throws Throwable {
+    BreakableService service = new BreakableService(false, false, true);
+    setServiceToTeardown(service);
+    ServiceShutdownHook hook = new ServiceShutdownHook(service);
+    hook.run();
+    assertStopped(service);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/373bb493/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/service/launcher/TestServiceLauncherCreationFailures.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/service/launcher/TestServiceLauncherCreationFailures.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/service/launcher/TestServiceLauncherCreationFailures.java
new file mode 100644
index 0000000..c3506b3
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/service/launcher/TestServiceLauncherCreationFailures.java
@@ -0,0 +1,83 @@
+/*
+ * 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.service.launcher;
+
+import org.apache.hadoop.service.launcher.testservices.FailInConstructorService;
+import org.apache.hadoop.service.launcher.testservices.FailInInitService;
+import org.apache.hadoop.service.launcher.testservices.FailInStartService;
+import org.apache.hadoop.service.launcher.testservices.FailingStopInStartService;
+import org.junit.Test;
+
+/**
+ * Explore the ways in which the launcher is expected to (safely) fail.
+ */
+public class TestServiceLauncherCreationFailures extends
+    AbstractServiceLauncherTestBase {
+
+  public static final String SELF =
+      "org.apache.hadoop.service.launcher.TestServiceLauncherCreationFailures";
+
+  @Test
+  public void testNoArgs() throws Throwable {
+    try {
+      ServiceLauncher.serviceMain();
+    } catch (ServiceLaunchException e) {
+      assertExceptionDetails(EXIT_USAGE, "", e);
+    }
+  }
+
+  @Test
+  public void testUnknownClass() throws Throwable {
+    assertServiceCreationFails("no.such.classname");
+  }
+
+  @Test
+  public void testNotAService() throws Throwable {
+    assertServiceCreationFails(SELF);
+  }
+
+  @Test
+  public void testNoSimpleConstructor() throws Throwable {
+    assertServiceCreationFails(
+        "org.apache.hadoop.service.launcher.FailureTestService");
+  }
+
+  @Test
+  public void testFailInConstructor() throws Throwable {
+    assertServiceCreationFails(FailInConstructorService.NAME);
+  }
+
+  @Test
+  public void testFailInInit() throws Throwable {
+    assertLaunchOutcome(FailInInitService.EXIT_CODE, "",
+        FailInInitService.NAME);
+  }
+
+  @Test
+  public void testFailInStart() throws Throwable {
+    assertLaunchOutcome(FailInStartService.EXIT_CODE, "",
+        FailInStartService.NAME);
+  }
+
+  @Test
+  public void testFailInStopIsIgnored() throws Throwable {
+    assertRuns(FailingStopInStartService.NAME);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/373bb493/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/service/launcher/TestServiceLauncherInnerMethods.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/service/launcher/TestServiceLauncherInnerMethods.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/service/launcher/TestServiceLauncherInnerMethods.java
new file mode 100644
index 0000000..5869f34
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/service/launcher/TestServiceLauncherInnerMethods.java
@@ -0,0 +1,95 @@
+/*
+ * 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.service.launcher;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.service.BreakableService;
+import org.apache.hadoop.service.Service;
+import org.apache.hadoop.service.launcher.testservices.ExceptionInExecuteLaunchableService;
+import org.apache.hadoop.service.launcher.testservices.LaunchableRunningService;
+import org.apache.hadoop.service.launcher.testservices.NoArgsAllowedService;
+import org.junit.Test;
+
+import java.util.List;
+
+/**
+ * Test the inner launcher methods.
+ */
+@SuppressWarnings("ThrowableResultOfMethodCallIgnored")
+public class TestServiceLauncherInnerMethods extends
+    AbstractServiceLauncherTestBase {
+
+  @Test
+  public void testLaunchService() throws Throwable {
+    ServiceLauncher<NoArgsAllowedService> launcher =
+        launchService(NoArgsAllowedService.class, new Configuration());
+    NoArgsAllowedService service = launcher.getService();
+    assertNotNull("null service from " + launcher, service);
+    service.stop();
+  }
+
+  @Test
+  public void testLaunchServiceArgs() throws Throwable {
+    launchExpectingException(NoArgsAllowedService.class,
+        new Configuration(),
+        "arguments",
+        EXIT_COMMAND_ARGUMENT_ERROR,
+        "one",
+        "two");
+  }
+
+  @Test
+  public void testAccessLaunchedService() throws Throwable {
+    ServiceLauncher<LaunchableRunningService> launcher =
+        launchService(LaunchableRunningService.class, new Configuration());
+    LaunchableRunningService service = launcher.getService();
+    assertInState(service, Service.STATE.STARTED);
+    service.failInRun = true;
+    service.setExitCode(EXIT_CONNECTIVITY_PROBLEM);
+    assertEquals(EXIT_CONNECTIVITY_PROBLEM, service.execute());
+  }
+
+  @Test
+  public void testLaunchThrowableRaised() throws Throwable {
+    launchExpectingException(ExceptionInExecuteLaunchableService.class,
+        new Configuration(),
+        "java.lang.OutOfMemoryError", EXIT_EXCEPTION_THROWN,
+        ExceptionInExecuteLaunchableService.ARG_THROWABLE);
+  }
+
+  @Test
+  public void testBreakableServiceLifecycle() throws Throwable {
+    ServiceLauncher<BreakableService> launcher =
+        launchService(BreakableService.class, new Configuration());
+    BreakableService service = launcher.getService();
+    assertNotNull("null service from " + launcher, service);
+    service.stop();
+  }
+
+  @Test
+  public void testConfigLoading() throws Throwable {
+    ServiceLauncher<BreakableService> launcher =
+        new ServiceLauncher<>("BreakableService");
+    List<String> configurationsToCreate = launcher.getConfigurationsToCreate();
+    assertTrue(configurationsToCreate.size() > 1);
+    int created = launcher.loadConfigurationClasses();
+    assertEquals(1, created);
+  }
+
+}


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org