瀏覽代碼

YARN-679. Add an entry point that can start any Yarn service. Contributed by Steve Loughran.

Junping Du 8 年之前
父節點
當前提交
373bb4931f
共有 38 個文件被更改,包括 4861 次插入148 次删除
  1. 56 10
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/service/ServiceStateException.java
  2. 78 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/service/launcher/AbstractLaunchableService.java
  3. 129 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/service/launcher/HadoopUncaughtExceptionHandler.java
  4. 216 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/service/launcher/InterruptEscalator.java
  5. 178 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/service/launcher/IrqHandler.java
  6. 95 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/service/launcher/LaunchableService.java
  7. 59 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/service/launcher/LauncherArguments.java
  8. 183 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/service/launcher/LauncherExitCodes.java
  9. 81 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/service/launcher/ServiceLaunchException.java
  10. 1044 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/service/launcher/ServiceLauncher.java
  11. 112 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/service/launcher/ServiceShutdownHook.java
  12. 462 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/service/launcher/package-info.java
  13. 35 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ExitCodeProvider.java
  14. 194 59
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ExitUtil.java
  15. 71 54
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/GenericOptionsParser.java
  16. 26 17
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/StringUtils.java
  17. 15 8
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/service/BreakableService.java
  18. 317 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/service/launcher/AbstractServiceLauncherTestBase.java
  19. 59 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/service/launcher/ExitTrackingServiceLauncher.java
  20. 146 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/service/launcher/TestServiceConf.java
  21. 118 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/service/launcher/TestServiceInterruptHandling.java
  22. 213 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/service/launcher/TestServiceLauncher.java
  23. 83 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/service/launcher/TestServiceLauncherCreationFailures.java
  24. 95 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/service/launcher/TestServiceLauncherInnerMethods.java
  25. 96 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/service/launcher/testservices/ExceptionInExecuteLaunchableService.java
  26. 33 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/service/launcher/testservices/FailInConstructorService.java
  27. 38 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/service/launcher/testservices/FailInInitService.java
  28. 37 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/service/launcher/testservices/FailInStartService.java
  29. 47 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/service/launcher/testservices/FailingStopInStartService.java
  30. 55 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/service/launcher/testservices/FailureTestService.java
  31. 63 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/service/launcher/testservices/InitInConstructorLaunchableService.java
  32. 111 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/service/launcher/testservices/LaunchableRunningService.java
  33. 64 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/service/launcher/testservices/NoArgsAllowedService.java
  34. 46 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/service/launcher/testservices/NullBindLaunchableService.java
  35. 84 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/service/launcher/testservices/RunningService.java
  36. 49 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/service/launcher/testservices/StoppingInStartLaunchableService.java
  37. 39 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/service/launcher/testservices/StringConstructorOnlyService.java
  38. 34 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/GenericTestUtils.java

+ 56 - 10
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/service/ServiceStateException.java

@@ -20,37 +20,83 @@ package org.apache.hadoop.service;
 
 
 import org.apache.hadoop.classification.InterfaceAudience.Public;
 import org.apache.hadoop.classification.InterfaceAudience.Public;
 import org.apache.hadoop.classification.InterfaceStability.Evolving;
 import org.apache.hadoop.classification.InterfaceStability.Evolving;
+import org.apache.hadoop.service.launcher.LauncherExitCodes;
+import org.apache.hadoop.util.ExitCodeProvider;
 
 
 /**
 /**
- * Exception that is raised on state change operations.
+ * Exception that can be raised on state change operations, whose
+ * exit code can be explicitly set, determined from that of any nested
+ * cause, or a default value of
+ * {@link  LauncherExitCodes#EXIT_SERVICE_LIFECYCLE_EXCEPTION}.
  */
  */
 @Public
 @Public
 @Evolving
 @Evolving
-public class ServiceStateException extends RuntimeException {
+public class ServiceStateException extends RuntimeException implements
+    ExitCodeProvider {
 
 
   private static final long serialVersionUID = 1110000352259232646L;
   private static final long serialVersionUID = 1110000352259232646L;
 
 
+  /**
+   * Exit code.
+   */
+  private int exitCode ;
+
+  /**
+   * Instantiate
+   * @param message error message
+   */
   public ServiceStateException(String message) {
   public ServiceStateException(String message) {
-    super(message);
+    this(message, null);
   }
   }
 
 
+  /**
+   * Instantiate with a message and cause; if the cause has an exit code
+   * then it is used, otherwise the generic
+   * {@link LauncherExitCodes#EXIT_SERVICE_LIFECYCLE_EXCEPTION} exit code
+   * is used.
+   * @param message exception message
+   * @param cause optional inner cause
+   */
   public ServiceStateException(String message, Throwable cause) {
   public ServiceStateException(String message, Throwable cause) {
     super(message, cause);
     super(message, cause);
+    if(cause instanceof ExitCodeProvider) {
+      this.exitCode = ((ExitCodeProvider) cause).getExitCode();
+    } else {
+      this.exitCode = LauncherExitCodes.EXIT_SERVICE_LIFECYCLE_EXCEPTION;
+    }
+  }
+
+  /**
+   * Instantiate, using the specified exit code as the exit code
+   * of the exception, irrespetive of any exit code supplied by any inner
+   * cause.
+   *
+   * @param exitCode exit code to declare
+   * @param message exception message
+   * @param cause inner cause
+   */
+  public ServiceStateException(int exitCode,
+      String message,
+      Throwable cause) {
+    this(message, cause);
+    this.exitCode =  exitCode;
   }
   }
 
 
   public ServiceStateException(Throwable cause) {
   public ServiceStateException(Throwable cause) {
     super(cause);
     super(cause);
   }
   }
 
 
+  @Override
+  public int getExitCode() {
+    return exitCode;
+  }
+
   /**
   /**
    * Convert any exception into a {@link RuntimeException}.
    * Convert any exception into a {@link RuntimeException}.
-   * If the caught exception is already of that type, it is typecast to a
-   * {@link RuntimeException} and returned.
-   *
    * All other exception types are wrapped in a new instance of
    * All other exception types are wrapped in a new instance of
-   * ServiceStateException
+   * {@code ServiceStateException}.
    * @param fault exception or throwable
    * @param fault exception or throwable
-   * @return a ServiceStateException to rethrow
+   * @return a {@link RuntimeException} to rethrow
    */
    */
   public static RuntimeException convert(Throwable fault) {
   public static RuntimeException convert(Throwable fault) {
     if (fault instanceof RuntimeException) {
     if (fault instanceof RuntimeException) {
@@ -66,10 +112,10 @@ public class ServiceStateException extends RuntimeException {
    * {@link RuntimeException} and returned.
    * {@link RuntimeException} and returned.
    *
    *
    * All other exception types are wrapped in a new instance of
    * All other exception types are wrapped in a new instance of
-   * ServiceStateException
+   * {@code ServiceStateException}.
    * @param text text to use if a new exception is created
    * @param text text to use if a new exception is created
    * @param fault exception or throwable
    * @param fault exception or throwable
-   * @return a ServiceStateException to rethrow
+   * @return a {@link RuntimeException} to rethrow
    */
    */
   public static RuntimeException convert(String text, Throwable fault) {
   public static RuntimeException convert(String text, Throwable fault) {
     if (fault instanceof RuntimeException) {
     if (fault instanceof RuntimeException) {

+ 78 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/service/launcher/AbstractLaunchableService.java

@@ -0,0 +1,78 @@
+/*
+ * 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 java.util.List;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.service.AbstractService;
+
+/**
+ * Subclass of {@link AbstractService} that provides basic implementations
+ * of the {@link LaunchableService} methods.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public abstract class AbstractLaunchableService extends AbstractService
+    implements LaunchableService {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(AbstractLaunchableService.class);
+
+  /**
+   * Construct an instance with the given name.
+   */
+  protected AbstractLaunchableService(String name) {
+    super(name);
+  }
+
+  /**
+   * {@inheritDoc}
+   * <p>
+   * The base implementation logs all arguments at the debug level,
+   * then returns the passed in config unchanged.
+   */
+  
+  @Override
+  public Configuration bindArgs(Configuration config, List<String> args) throws
+      Exception {
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Service {} passed in {} arguments:", getName(), args.size());
+      for (String arg : args) {
+        LOG.debug(arg);
+      }
+    }
+    return config;
+  }
+
+  /**
+   * {@inheritDoc}
+   * <p>
+   * The action is to signal success by returning the exit code 0.
+   */
+  @Override
+  public int execute() throws Exception {
+    return LauncherExitCodes.EXIT_SUCCESS;
+  }
+}

+ 129 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/service/launcher/HadoopUncaughtExceptionHandler.java

@@ -0,0 +1,129 @@
+/*
+ * 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 java.lang.Thread.UncaughtExceptionHandler;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.classification.InterfaceStability.Evolving;
+import org.apache.hadoop.util.ExitUtil;
+import org.apache.hadoop.util.ShutdownHookManager;
+
+/**
+ * This class is intended to be installed by calling 
+ * {@link Thread#setDefaultUncaughtExceptionHandler(UncaughtExceptionHandler)}
+ * in the main entry point. 
+ *
+ * The base class will always attempt to shut down the process if an Error
+ * was raised; the behavior on a standard Exception, raised outside 
+ * process shutdown, is simply to log it. 
+ *
+ * (Based on the class {@code YarnUncaughtExceptionHandler})
+ */
+@SuppressWarnings("UseOfSystemOutOrSystemErr")
+@Public
+@Evolving
+public class HadoopUncaughtExceptionHandler
+    implements UncaughtExceptionHandler {
+
+  /**
+   * Logger.
+   */
+  private static final Logger LOG = LoggerFactory.getLogger(
+      HadoopUncaughtExceptionHandler.class);
+
+  /**
+   * Delegate for simple exceptions.
+   */
+  private final UncaughtExceptionHandler delegate;
+
+  /**
+   * Create an instance delegating to the supplied handler if
+   * the exception is considered "simple".
+   * @param delegate a delegate exception handler.
+   */
+  public HadoopUncaughtExceptionHandler(UncaughtExceptionHandler delegate) {
+    this.delegate = delegate;
+  }
+
+  /**
+   * Basic exception handler -logs simple exceptions, then continues.
+   */
+  public HadoopUncaughtExceptionHandler() {
+    this(null);
+  }
+
+  /**
+   * Uncaught exception handler.
+   * If an error is raised: shutdown
+   * The state of the system is unknown at this point -attempting
+   * a clean shutdown is dangerous. Instead: exit
+   * @param thread thread that failed
+   * @param exception the raised exception
+   */
+  @Override
+  public void uncaughtException(Thread thread, Throwable exception) {
+    if (ShutdownHookManager.get().isShutdownInProgress()) {
+      LOG.error("Thread {} threw an error during shutdown: {}.",
+          thread.toString(),
+          exception,
+          exception);
+    } else if (exception instanceof Error) {
+      try {
+        LOG.error("Thread {} threw an error: {}. Shutting down",
+            thread.toString(),
+            exception,
+            exception);
+      } catch (Throwable err) {
+        // We don't want to not exit because of an issue with logging
+      }
+      if (exception instanceof OutOfMemoryError) {
+        // 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 don't want to exit because of logging issues.
+        }
+        ExitUtil.haltOnOutOfMemory((OutOfMemoryError) exception);
+      } else {
+        // error other than OutOfMemory
+        ExitUtil.ExitException ee =
+            ServiceLauncher.convertToExitException(exception);
+        ExitUtil.terminate(ee.status, ee);
+      }
+    } else {
+      // simple exception in a thread. There's a policy decision here:
+      // terminate the process vs. keep going after a thread has failed
+      // base implementation: do nothing but log
+      LOG.error("Thread {} threw an exception: {}",
+          thread.toString(),
+          exception,
+          exception);
+      if (delegate != null) {
+        delegate.uncaughtException(thread, exception);
+      }
+    }
+
+  }
+
+}

+ 216 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/service/launcher/InterruptEscalator.java

@@ -0,0 +1,216 @@
+/*
+ * 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 java.lang.ref.WeakReference;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import com.google.common.base.Preconditions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.service.Service;
+import org.apache.hadoop.util.ExitUtil;
+
+import static org.apache.hadoop.service.launcher.LauncherExitCodes.EXIT_INTERRUPTED;
+
+/**
+ * Handles interrupts by shutting down a service, escalating if the service
+ * does not shut down in time, or when other interrupts are received.
+ * <ol>
+ *   <li>The service is given a time in milliseconds to stop:
+ *   if it exceeds this it the process exits anyway.</li>
+ *   <li>the exit operation used is {@link ServiceLauncher#exit(int, String)}
+ *   with the exit code {@link LauncherExitCodes#EXIT_INTERRUPTED}</li>
+ *   <li>If a second shutdown signal is received during the shutdown
+ *   process, {@link ExitUtil#halt(int)} is invoked. This handles the 
+ *   problem of blocking shutdown hooks.</li>
+ * </ol>
+ *
+ */
+
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public class InterruptEscalator implements IrqHandler.Interrupted {
+  private static final Logger LOG = LoggerFactory.getLogger(
+      InterruptEscalator.class);
+
+  /**
+   * Flag to indicate when a shutdown signal has already been received.
+   * This allows the operation to be escalated.
+   */
+  private final AtomicBoolean signalAlreadyReceived = new AtomicBoolean(false);
+
+  private final WeakReference<ServiceLauncher> ownerRef;
+
+  private final int shutdownTimeMillis;
+
+  /**
+   * Previous interrupt handlers. These are not queried.
+   */
+  private final List<IrqHandler> interruptHandlers = new ArrayList<>(2);
+  private boolean forcedShutdownTimedOut;
+
+  public InterruptEscalator(ServiceLauncher owner, int shutdownTimeMillis) {
+    Preconditions.checkArgument(owner != null, "null owner");
+    this.ownerRef = new WeakReference<>(owner);
+    this.shutdownTimeMillis = shutdownTimeMillis;
+  }
+
+  private ServiceLauncher getOwner() {
+    return ownerRef.get();
+  }
+
+  private Service getService() {
+    ServiceLauncher owner = getOwner();
+    return owner != null ? owner.getService() : null;
+  }
+
+  @Override
+  public String toString() {
+    final StringBuilder sb = new StringBuilder("InterruptEscalator{");
+    sb.append(" signalAlreadyReceived=").append(signalAlreadyReceived.get());
+    ServiceLauncher owner = ownerRef.get();
+    if (owner != null) {
+      sb.append(", owner= ").append(owner.toString());
+    }
+    sb.append(", shutdownTimeMillis=").append(shutdownTimeMillis);
+    sb.append(", forcedShutdownTimedOut=").append(forcedShutdownTimedOut);
+    sb.append('}');
+    return sb.toString();
+  }
+
+  @Override
+  public void interrupted(IrqHandler.InterruptData interruptData) {
+    String message = "Service interrupted by " + interruptData.toString();
+    LOG.warn(message);
+    if (!signalAlreadyReceived.compareAndSet(false, true)) {
+      message = "Repeated interrupt: escalating to a JVM halt";
+      LOG.warn(message);
+      // signal already received. On a second request to a hard JVM
+      // halt and so bypass any blocking shutdown hooks.
+      ExitUtil.halt(LauncherExitCodes.EXIT_INTERRUPTED, message);
+    }
+    Service service = getService();
+    if (service != null) {
+      //start an async shutdown thread with a timeout
+      ServiceForcedShutdown shutdown =
+          new ServiceForcedShutdown(service, shutdownTimeMillis);
+      Thread thread = new Thread(shutdown);
+      thread.setDaemon(true);
+      thread.setName("Service Forced Shutdown");
+      thread.start();
+      //wait for that thread to finish
+      try {
+        thread.join(shutdownTimeMillis);
+      } catch (InterruptedException ignored) {
+        //ignored
+      }
+      forcedShutdownTimedOut = !shutdown.getServiceWasShutdown();
+      if (forcedShutdownTimedOut) {
+        LOG.warn("Service did not shut down in time");
+      }
+    }
+    ExitUtil.terminate(EXIT_INTERRUPTED, message);
+  }
+
+  /**
+   * Register an interrupt handler.
+   * @param signalName signal name
+   * @throws IllegalArgumentException if the registration failed
+   */
+  public synchronized void register(String signalName) {
+    IrqHandler handler = new IrqHandler(signalName, this);
+    handler.bind();
+    interruptHandlers.add(handler);
+  }
+
+  /**
+   * Look up the handler for a signal.
+   * @param signalName signal name
+   * @return a handler if found
+   */
+  public synchronized IrqHandler lookup(String signalName) {
+    for (IrqHandler irqHandler : interruptHandlers) {
+      if (irqHandler.getName().equals(signalName)) {
+        return irqHandler;
+      }
+    }
+    return null;
+  }
+
+  /**
+   * Flag set if forced shut down timed out.
+   * @return true if a shutdown was attempted and it timed out
+   */
+  public boolean isForcedShutdownTimedOut() {
+    return forcedShutdownTimedOut;
+  }
+
+  /**
+   * Flag set if a signal has been received.
+   * @return true if there has been one interrupt already.
+   */
+  public boolean isSignalAlreadyReceived() {
+    return signalAlreadyReceived.get();
+  }
+
+  /**
+   * Forced shutdown runnable.
+   */
+  protected static class ServiceForcedShutdown implements Runnable {
+
+    private final int shutdownTimeMillis;
+    private final AtomicBoolean serviceWasShutdown =
+        new AtomicBoolean(false);
+    private Service service;
+
+    public ServiceForcedShutdown(Service service, int shutdownTimeMillis) {
+      this.shutdownTimeMillis = shutdownTimeMillis;
+      this.service = service;
+    }
+
+    /**
+     * Shutdown callback: stop the service and set an atomic boolean
+     * if it stopped within the shutdown time.
+     */
+    @Override
+    public void run() {
+      if (service != null) {
+        service.stop();
+        serviceWasShutdown.set(
+            service.waitForServiceToStop(shutdownTimeMillis));
+      } else {
+        serviceWasShutdown.set(true);
+      }
+    }
+
+    /**
+     * Probe for the service being shutdown.
+     * @return true if the service has been shutdown in the runnable
+     */
+    private boolean getServiceWasShutdown() {
+      return serviceWasShutdown.get();
+    }
+  }
+}

+ 178 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/service/launcher/IrqHandler.java

@@ -0,0 +1,178 @@
+/*
+ * 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 java.util.concurrent.atomic.AtomicInteger;
+
+import com.google.common.base.Preconditions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import sun.misc.Signal;
+import sun.misc.SignalHandler;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * Handler of interrupts that relays them to a registered
+ * implementation of {@link IrqHandler.Interrupted}.
+ *
+ * This class bundles up all the compiler warnings about abuse of sun.misc
+ * interrupt handling code into one place.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+@SuppressWarnings("UseOfSunClasses")
+public final class IrqHandler implements SignalHandler {
+  private static final Logger LOG = LoggerFactory.getLogger(IrqHandler.class);
+  
+  /**
+   * Definition of the Control-C handler name: {@value}.
+   */
+  public static final String CONTROL_C = "INT";
+
+  /**
+   * Definition of default <code>kill</code> signal: {@value}.
+   */
+  public static final String SIGTERM = "TERM";
+
+  /**
+   * Signal name.
+   */
+  private final String name;
+
+  /**
+   * Handler to relay to.
+   */
+  private final Interrupted handler;
+
+  /** Count of how many times a signal has been raised. */
+  private final AtomicInteger signalCount = new AtomicInteger(0);
+
+  /**
+   * Stored signal.
+   */
+  private Signal signal;
+
+  /**
+   * Create an IRQ handler bound to the specific interrupt.
+   * @param name signal name
+   * @param handler handler
+   */
+  public IrqHandler(String name, Interrupted handler) {
+    Preconditions.checkArgument(name != null, "Null \"name\"");
+    Preconditions.checkArgument(handler != null, "Null \"handler\"");
+    this.handler = handler;
+    this.name = name;
+  }
+
+  /**
+   * Bind to the interrupt handler.
+   * @throws IllegalArgumentException if the exception could not be set
+   */
+  void bind() {
+    Preconditions.checkState(signal == null, "Handler already bound");
+    try {
+      signal = new Signal(name);
+      Signal.handle(signal, this);
+    } catch (IllegalArgumentException e) {
+      throw new IllegalArgumentException(
+          "Could not set handler for signal \"" + name + "\"."
+          + "This can happen if the JVM has the -Xrs set.",
+          e);
+    }
+  }
+
+  /**
+   * @return the signal name.
+   */
+  public String getName() {
+    return name;
+  }
+
+  /**
+   * Raise the signal.
+   */
+  public void raise() {
+    Signal.raise(signal);
+  }
+
+  @Override
+  public String toString() {
+    return "IrqHandler for signal " + name;
+  }
+
+  /**
+   * Handler for the JVM API for signal handling.
+   * @param s signal raised
+   */
+  @Override
+  public void handle(Signal s) {
+    signalCount.incrementAndGet();
+    InterruptData data = new InterruptData(s.getName(), s.getNumber());
+    LOG.info("Interrupted: {}", data);
+    handler.interrupted(data);
+  }
+
+  /**
+   * Get the count of how many times a signal has been raised.
+   * @return the count of signals
+   */
+  public int getSignalCount() {
+    return signalCount.get();
+  }
+  
+  /**
+   * Callback issues on an interrupt.
+   */
+  public interface Interrupted {
+
+    /**
+     * Handle an interrupt.
+     * @param interruptData data
+     */
+    void interrupted(InterruptData interruptData);
+  }
+
+  /**
+   * Interrupt data to pass on.
+   */
+  public static class InterruptData {
+    private final String name;
+    private final int number;
+
+    public InterruptData(String name, int number) {
+      this.name = name;
+      this.number = number;
+    }
+
+    public String getName() {
+      return name;
+    }
+
+    public int getNumber() {
+      return number;
+    }
+
+    @Override
+    public String toString() {
+      return "signal " + name + '(' + number + ')';
+    }
+  }
+}

+ 95 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/service/launcher/LaunchableService.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 java.util.List;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.service.Service;
+
+/**
+ * An interface which services can implement to have their
+ * execution managed by the ServiceLauncher.
+ * <p>
+ * The command line options will be passed down before the 
+ * {@link Service#init(Configuration)} operation is invoked via an
+ * invocation of {@link LaunchableService#bindArgs(Configuration, List)}
+ * After the service has been successfully started via {@link Service#start()}
+ * the {@link LaunchableService#execute()} method is called to execute the 
+ * service. When this method returns, the service launcher will exit, using
+ * the return code from the method as its exit option.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public interface LaunchableService extends Service {
+
+  /**
+   * Propagate the command line arguments.
+   * <p>
+   * This method is called before {@link #init(Configuration)};
+   * Any non-null configuration that is returned from this operation
+   * becomes the one that is passed on to that {@link #init(Configuration)}
+   * operation.
+   * <p>
+   * This permits implementations to change the configuration before
+   * the init operation. As the ServiceLauncher only creates
+   * an instance of the base {@link Configuration} class, it is
+   * recommended to instantiate any subclass (such as YarnConfiguration)
+   * that injects new resources.
+   * <p>
+   * @param config the initial configuration build up by the
+   * service launcher.
+   * @param args list of arguments passed to the command line
+   * after any launcher-specific commands have been stripped.
+   * @return the configuration to init the service with.
+   * Recommended: pass down the config parameter with any changes
+   * @throws Exception any problem
+   */
+  Configuration bindArgs(Configuration config, List<String> args)
+        throws Exception;
+
+  /**
+   * Run a service. This method is called after {@link Service#start()}.
+   * <p>
+   * The return value becomes the exit code of the launched process.
+   * <p>
+   * If an exception is raised, the policy is:
+   * <ol>
+   *   <li>Any subset of {@link org.apache.hadoop.util.ExitUtil.ExitException}:
+   *   the exception is passed up unmodified.
+   *   </li>
+   *   <li>Any exception which implements
+   *   {@link org.apache.hadoop.util.ExitCodeProvider}:
+   *   A new {@link ServiceLaunchException} is created with the exit code
+   *   and message of the thrown exception; the thrown exception becomes the
+   *   cause.</li>
+   *   <li>Any other exception: a new {@link ServiceLaunchException} is created
+   *   with the exit code {@link LauncherExitCodes#EXIT_EXCEPTION_THROWN} and
+   *   the message of the original exception (which becomes the cause).</li>
+   * </ol>
+   * @return the exit code
+   * @throws org.apache.hadoop.util.ExitUtil.ExitException an exception passed
+   *  up as the exit code and error text.
+   * @throws Exception any exception to report. If it provides an exit code
+   * this is used in a wrapping exception.
+   */
+  int execute() throws Exception;
+}

+ 59 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/service/launcher/LauncherArguments.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;
+
+/**
+ * Standard launcher arguments. These are all from
+ * the {@code GenericOptionsParser}, simply extracted to constants.
+ */
+public interface LauncherArguments {
+  /**
+   * Name of the configuration argument on the CLI.
+   * Value: {@value}
+   */
+  String ARG_CONF = "conf";
+  String ARG_CONF_SHORT = "conf";
+
+  /**
+   * prefixed version of {@link #ARG_CONF}.
+   * Value: {@value}
+   */
+  String ARG_CONF_PREFIXED = "--" + ARG_CONF;
+
+  /**
+   * Name of a configuration class which is loaded before any
+   * attempt is made to load the class.
+   * <p>
+   * Value: {@value}
+   */
+  String ARG_CONFCLASS = "hadoopconf";
+  String ARG_CONFCLASS_SHORT = "hadoopconf";
+
+  /**
+   * Prefixed version of {@link #ARG_CONFCLASS}.
+   * Value: {@value}
+   */
+  String ARG_CONFCLASS_PREFIXED = "--" + ARG_CONFCLASS;
+
+  /**
+   * Error string on a parse failure.
+   * Value: {@value}
+   */
+  String E_PARSE_FAILED = "Failed to parse: ";
+}

+ 183 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/service/launcher/LauncherExitCodes.java

@@ -0,0 +1,183 @@
+/*
+ * 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.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * Common Exit codes.
+ * <p>
+ * Codes with a YARN prefix are YARN-related.
+ * <p>
+ * Many of the exit codes are designed to resemble HTTP error codes,
+ * squashed into a single byte. e.g 44 , "not found" is the equivalent
+ * of 404. The various 2XX HTTP error codes aren't followed;
+ * the Unix standard of "0" for success is used.
+ * <pre>
+ *    0-10: general command issues
+ *   30-39: equivalent to the 3XX responses, where those responses are
+ *          considered errors by the application.
+ *   40-49: client-side/CLI/config problems
+ *   50-59: service-side problems.
+ *   60+  : application specific error codes
+ * </pre>
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public interface LauncherExitCodes {
+
+  /**
+   * Success: {@value}.
+   */
+  int EXIT_SUCCESS = 0;
+
+  /**
+   * Generic "false/fail" response: {@value}.
+   * The operation worked but the result was not "true" from the viewpoint
+   * of the executed code.
+   */
+  int EXIT_FAIL = -1;
+
+  /**
+   * Exit code when a client requested service termination: {@value}.
+   */
+  int EXIT_CLIENT_INITIATED_SHUTDOWN = 1;
+
+  /**
+   * Exit code when targets could not be launched: {@value}.
+   */
+  int EXIT_TASK_LAUNCH_FAILURE = 2;
+
+  /**
+   * Exit code when a control-C, kill -3, signal was picked up: {@value}.
+   */
+  int EXIT_INTERRUPTED = 3;
+
+  /**
+   * Exit code when something happened but we can't be specific: {@value}.
+   */
+  int EXIT_OTHER_FAILURE = 5;
+
+  /**
+   * Exit code when the command line doesn't parse: {@value}, or
+   * when it is otherwise invalid.
+   * <p>
+   * Approximate HTTP equivalent: {@code 400 Bad Request}
+   */
+  int EXIT_COMMAND_ARGUMENT_ERROR = 40;
+
+  /**
+   * The request requires user authentication: {@value}.
+   * <p>
+   * approximate HTTP equivalent: Approximate HTTP equivalent: {@code 401 Unauthorized}
+   */
+  int EXIT_UNAUTHORIZED = 41;
+
+  /**
+   * Exit code when a usage message was printed: {@value}.
+   */
+  int EXIT_USAGE = 42;
+
+  /**
+   * Forbidden action: {@value}.
+   * <p>
+   * Approximate HTTP equivalent: Approximate HTTP equivalent: {@code 403: Forbidden}
+   */
+  int EXIT_FORBIDDEN = 43;
+
+  /**
+   * Something was not found: {@value}.
+   * <p>
+   * Approximate HTTP equivalent: {@code 404: Not Found}
+   */
+  int EXIT_NOT_FOUND = 44;
+
+  /**
+   * The operation is not allowed: {@value}.
+   * <p>
+   * Approximate HTTP equivalent: {@code 405: Not allowed}
+   */
+  int EXIT_OPERATION_NOT_ALLOWED = 45;
+
+  /**
+   * The command is somehow not acceptable: {@value}.
+   * <p>
+   * Approximate HTTP equivalent: {@code 406: Not Acceptable}
+   */
+  int EXIT_NOT_ACCEPTABLE = 46;
+
+  /**
+   * Exit code on connectivity problems: {@value}.
+   * <p>
+   * Approximate HTTP equivalent: {@code 408: Request Timeout}
+   */
+  int EXIT_CONNECTIVITY_PROBLEM = 48;
+
+  /**
+   * Exit code when the configurations in valid/incomplete: {@value}.
+   * <p>
+   * Approximate HTTP equivalent: {@code 409: Conflict}
+   */
+  int EXIT_BAD_CONFIGURATION = 49;
+
+  /**
+   * Exit code when an exception was thrown from the service: {@value}.
+   * <p>
+   * Approximate HTTP equivalent: {@code 500 Internal Server Error}
+   */
+  int EXIT_EXCEPTION_THROWN = 50;
+
+  /**
+   * Unimplemented feature: {@value}.
+   * <p>
+   * Approximate HTTP equivalent: {@code 501: Not Implemented}
+   */
+  int EXIT_UNIMPLEMENTED = 51;
+
+  /**
+   * Service Unavailable; it may be available later: {@value}.
+   * <p>
+   * Approximate HTTP equivalent: {@code 503 Service Unavailable}
+   */
+  int EXIT_SERVICE_UNAVAILABLE = 53;
+
+  /**
+   * The application does not support, or refuses to support this
+   * version: {@value}.
+   * <p>
+   * If raised, this is expected to be raised server-side and likely due
+   * to client/server version incompatibilities.
+   * <p>
+   * Approximate HTTP equivalent: {@code 505: Version Not Supported}
+   */
+  int EXIT_UNSUPPORTED_VERSION = 55;
+
+  /**
+   * The service instance could not be created: {@value}.
+   */
+  int EXIT_SERVICE_CREATION_FAILURE = 56;
+ 
+  /**
+   * The service instance could not be created: {@value}.
+   */
+  int EXIT_SERVICE_LIFECYCLE_EXCEPTION = 57;
+  
+}

+ 81 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/service/launcher/ServiceLaunchException.java

@@ -0,0 +1,81 @@
+/*
+ * 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 java.util.Locale;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.util.ExitCodeProvider;
+import org.apache.hadoop.util.ExitUtil;
+
+/**
+ * A service launch exception that includes an exit code.
+ * <p>
+ * When caught by the ServiceLauncher, it will convert that
+ * into a process exit code.
+ * 
+ * The {@link #ServiceLaunchException(int, String, Object...)} constructor
+ * generates formatted exceptions.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+
+public class ServiceLaunchException extends ExitUtil.ExitException
+    implements ExitCodeProvider, LauncherExitCodes {
+
+  /**
+   * Create an exception with the specific exit code.
+   * @param exitCode exit code
+   * @param cause cause of the exception
+   */
+  public ServiceLaunchException(int exitCode, Throwable cause) {
+    super(exitCode, cause);
+  }
+
+  /**
+   * Create an exception with the specific exit code and text.
+   * @param exitCode exit code
+   * @param message message to use in exception
+   */
+  public ServiceLaunchException(int exitCode, String message) {
+    super(exitCode, message);
+  }
+
+  /**
+   * Create a formatted exception.
+   * <p>
+   * This uses {@link String#format(String, Object...)}
+   * to build the formatted exception in the ENGLISH locale.
+   * <p>
+   * If the last argument is a throwable, it becomes the cause of the exception.
+   * It will also be used as a parameter for the format.
+   * @param exitCode exit code
+   * @param format format for message to use in exception
+   * @param args list of arguments
+   */
+  public ServiceLaunchException(int exitCode, String format, Object... args) {
+    super(exitCode, String.format(Locale.ENGLISH, format, args));
+    if (args.length > 0 && (args[args.length - 1] instanceof Throwable)) {
+      initCause((Throwable) args[args.length - 1]);
+    }
+  }
+
+}

+ 1044 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/service/launcher/ServiceLauncher.java

@@ -0,0 +1,1044 @@
+/*
+ * 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 java.io.File;
+import java.io.IOException;
+import java.net.URL;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.commons.cli.CommandLine;
+import org.apache.commons.cli.Option;
+import org.apache.commons.cli.OptionBuilder;
+import org.apache.commons.cli.Options;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.service.Service;
+import org.apache.hadoop.util.ExitCodeProvider;
+import org.apache.hadoop.util.ExitUtil;
+import org.apache.hadoop.util.GenericOptionsParser;
+import org.apache.hadoop.util.StringUtils;
+
+/**
+ * A class to launch any YARN service by name.
+ *
+ * It's designed to be subclassed for custom entry points.
+ *
+ * Workflow:
+ * <ol>
+ *   <li>An instance of the class is created. It must be of the type
+ *   {@link Service}</li>
+ *   <li>If it implements
+ *   {@link LaunchableService#bindArgs(Configuration, List)},
+ *    it is given the binding args off the CLI after all general configuration
+ *    arguments have been stripped.</li>
+ *   <li>Its {@link Service#init(Configuration)} and {@link Service#start()}
+ *   methods are called.</li>
+ *   <li>If it implements it, {@link LaunchableService#execute()}
+ *   is called and its return code used as the exit code.</li>
+ *   <li>Otherwise: it waits for the service to stop, assuming that the
+ *   {@link Service#start()} method spawns one or more thread
+ *   to perform work</li>
+ *   <li>If any exception is raised and provides an exit code,
+ *   that is, it implements {@link ExitCodeProvider},
+ *   the return value of {@link ExitCodeProvider#getExitCode()},
+ *   becomes the exit code of the command.</li>
+ * </ol>
+ * Error and warning messages are logged to {@code stderr}.
+ * 
+ * @param <S> service class to cast the generated service to.
+ */
+@SuppressWarnings("UseOfSystemOutOrSystemErr")
+public class ServiceLauncher<S extends Service>
+    implements LauncherExitCodes, LauncherArguments,
+    Thread.UncaughtExceptionHandler {
+
+  /**
+   * Logger.
+   */
+  private static final Logger LOG =
+      LoggerFactory.getLogger(ServiceLauncher.class);
+
+  /**
+   * Priority for the shutdown hook: {@value}.
+   */
+  protected static final int SHUTDOWN_PRIORITY = 30;
+
+  /**
+   * The name of this class.
+   */
+  public static final String NAME = "ServiceLauncher";
+
+  protected static final String USAGE_NAME = "Usage: " + NAME;
+  protected static final String USAGE_SERVICE_ARGUMENTS =
+      "service-classname <service arguments>";
+
+  /**
+   * Usage message.
+   *
+   * Text: {@value}
+   */
+  public static final String USAGE_MESSAGE =
+      USAGE_NAME
+          + " [" + ARG_CONF_PREFIXED + " <conf file>]"
+          + " [" + ARG_CONFCLASS_PREFIXED + " <configuration classname>]"
+          + " "  + USAGE_SERVICE_ARGUMENTS;
+
+  /**
+   * The shutdown time on an interrupt: {@value}.
+   */
+  private static final int SHUTDOWN_TIME_ON_INTERRUPT = 30 * 1000;
+
+  /**
+   * The launched service.
+   *
+   * Invalid until the service has been created. 
+   */
+  private volatile S service;
+  
+  /**
+   * Exit code of the service.
+   *
+   * Invalid until a service has
+   * executed or stopped, depending on the service type.
+   */
+  private int serviceExitCode;
+  
+  /**
+   * Any exception raised during execution.
+   */
+  private ExitUtil.ExitException serviceException;
+
+  /**
+   * The interrupt escalator for the service.
+   */
+  private InterruptEscalator interruptEscalator;
+
+  /**
+   * Configuration used for the service.
+   */
+  private Configuration configuration;
+
+  /**
+   * Text description of service for messages.
+   */
+  private String serviceName;
+
+  /**
+   * Classname for the service to create.; empty string otherwise.
+   */
+  private String serviceClassName = "";
+
+  /**
+   * List of the standard configurations to create (and so load in properties).
+   * The values are Hadoop, HDFS and YARN configurations.
+   */
+  protected static final String[] DEFAULT_CONFIGS = {
+      "org.apache.hadoop.conf.Configuration",
+      "org.apache.hadoop.hdfs.HdfsConfiguration",
+      "org.apache.hadoop.yarn.conf.YarnConfiguration"
+  };
+
+  /**
+   * List of classnames to load to configuration before creating a
+   * {@link Configuration} instance.
+   */
+  private List<String> confClassnames = new ArrayList<>(DEFAULT_CONFIGS.length);
+
+  /**
+   * URLs of configurations to load into the configuration instance created.
+   */
+  private List<URL> confResourceUrls = new ArrayList<>(1);
+
+  /** Command options. Preserved for usage statements. */
+  private Options commandOptions;
+
+  /**
+   * Create an instance of the launcher.
+   * @param serviceClassName classname of the service
+   */
+  public ServiceLauncher(String serviceClassName) {
+    this(serviceClassName, serviceClassName);
+  }
+
+  /**
+   * Create an instance of the launcher.
+   * @param serviceName name of service for text messages
+   * @param serviceClassName classname of the service
+   */
+  public ServiceLauncher(String serviceName, String serviceClassName) {
+    this.serviceClassName = serviceClassName;
+    this.serviceName = serviceName;
+    // set up initial list of configurations
+    confClassnames.addAll(Arrays.asList(DEFAULT_CONFIGS));
+  }
+
+  /**
+   * Get the service.
+   *
+   * Null until
+   * {@link #coreServiceLaunch(Configuration, List, boolean, boolean)}
+   * has completed.
+   * @return the service
+   */
+  public final S getService() {
+    return service;
+  }
+
+  /**
+   * Setter is to give subclasses the ability to manipulate the service.
+   * @param s the new service
+   */
+  protected void setService(S s) {
+    this.service = s;
+  }
+
+  /**
+   * Get the configuration constructed from the command line arguments.
+   * @return the configuration used to create the service
+   */
+  public final Configuration getConfiguration() {
+    return configuration;
+  }
+
+  /**
+   * The exit code from a successful service execution.
+   * @return the exit code. 
+   */
+  public final int getServiceExitCode() {
+    return serviceExitCode;
+  }
+
+  /**
+   * Get the exit exception used to end this service.
+   * @return an exception, which will be null until the service
+   * has exited (and {@code System.exit} has not been called)
+   */
+  public final ExitUtil.ExitException getServiceException() {
+    return serviceException;
+  }
+
+  /**
+   * Probe for service classname being defined.
+   * @return true if the classname is set
+   */
+  private boolean isClassnameDefined() {
+    return serviceClassName != null && !serviceClassName.isEmpty();
+  }
+
+  @Override
+  public String toString() {
+    final StringBuilder sb = new StringBuilder("\"ServiceLauncher for \"");
+    sb.append(serviceName);
+    if (isClassnameDefined()) {
+      sb.append(", serviceClassName='").append(serviceClassName).append('\'');
+    }
+    if (service != null) {
+      sb.append(", service=").append(service);
+    }
+    return sb.toString();
+  }
+
+  /**
+   * Launch the service and exit.
+   *
+   * <ol>
+   * <li>Parse the command line.</li> 
+   * <li>Build the service configuration from it.</li>
+   * <li>Start the service.</li>.
+   * <li>If it is a {@link LaunchableService}: execute it</li>
+   * <li>Otherwise: wait for it to finish.</li>
+   * <li>Exit passing the status code to the {@link #exit(int, String)}
+   * method.</li>
+   * </ol>
+   * @param args arguments to the service. {@code arg[0]} is 
+   * assumed to be the service classname.
+   */
+  public void launchServiceAndExit(List<String> args) {
+    StringBuilder builder = new StringBuilder();
+    for (String arg : args) {
+      builder.append('"').append(arg).append("\" ");
+    }
+    String argumentString = builder.toString();
+    if (LOG.isDebugEnabled()) {
+      LOG.debug(startupShutdownMessage(serviceName, args));
+      LOG.debug(argumentString);
+    }
+    registerFailureHandling();
+    // set up the configs, using reflection to push in the -site.xml files
+    loadConfigurationClasses();
+    Configuration conf = createConfiguration();
+    for (URL resourceUrl : confResourceUrls) {
+      conf.addResource(resourceUrl);
+    }
+    bindCommandOptions();
+    ExitUtil.ExitException exitException;
+    try {
+      List<String> processedArgs = extractCommandOptions(conf, args);
+      exitException = launchService(conf, processedArgs, true, true);
+    } catch (ExitUtil.ExitException e) {
+      exitException = e;
+      noteException(exitException);
+    }
+    if (exitException.getExitCode() != 0) {
+      // something went wrong. Print the usage and commands
+      System.err.println(getUsageMessage());
+      System.err.println("Command: " + argumentString);
+    }
+    System.out.flush();
+    System.err.flush();
+    exit(exitException);
+  }
+
+  /**
+   * Set the {@link #commandOptions} field to the result of
+   * {@link #createOptions()}; protected for subclasses and test access.
+   */
+  protected void bindCommandOptions() {
+    commandOptions = createOptions();
+  }
+
+  /**
+   * Record that an Exit Exception has been raised.
+   * Save it to {@link #serviceException}, with its exit code in
+   * {@link #serviceExitCode}
+   * @param exitException exception
+   */
+  void noteException(ExitUtil.ExitException exitException) {
+    LOG.debug("Exception raised", exitException);
+    serviceExitCode = exitException.getExitCode();
+    serviceException = exitException;
+  }
+
+  /**
+   * Get the usage message, ideally dynamically.
+   * @return the usage message
+   */
+  protected String getUsageMessage() {
+    String message =   USAGE_MESSAGE;
+    if (commandOptions != null) {
+      message = USAGE_NAME
+          + " " + commandOptions.toString()
+          + " " + USAGE_SERVICE_ARGUMENTS;
+    }
+    return message;
+  }
+
+  /**
+   * Override point: create an options instance to combine with the 
+   * standard options set.
+   * <i>Important. Synchronize uses of {@link OptionBuilder}</i>
+   * with {@code OptionBuilder.class}
+   * @return the new options
+   */
+  @SuppressWarnings("static-access")
+  protected Options createOptions() {
+    synchronized (OptionBuilder.class) {
+      Options options = new Options();
+      Option oconf = OptionBuilder.withArgName("configuration file")
+          .hasArg()
+          .withDescription("specify an application configuration file")
+          .withLongOpt(ARG_CONF)
+          .create(ARG_CONF_SHORT);
+      Option confclass = OptionBuilder.withArgName("configuration classname")
+          .hasArg()
+          .withDescription(
+              "Classname of a Hadoop Configuration subclass to load")
+          .withLongOpt(ARG_CONFCLASS)
+          .create(ARG_CONFCLASS_SHORT);
+      Option property = OptionBuilder.withArgName("property=value")
+          .hasArg()
+          .withDescription("use value for given property")
+          .create('D');
+      options.addOption(oconf);
+      options.addOption(property);
+      options.addOption(confclass);
+      return options;
+    }
+  }
+
+  /**
+   * Override point: create the base configuration for the service.
+   *
+   * Subclasses can override to create HDFS/YARN configurations etc.
+   * @return the configuration to use as the service initializer.
+   */
+  protected Configuration createConfiguration() {
+    return new Configuration();
+  }
+
+  /**
+   * Override point: Get a list of configuration classes to create.
+   * @return the array of configs to attempt to create. If any are off the
+   * classpath, that is logged
+   */
+  @SuppressWarnings("ReturnOfCollectionOrArrayField")
+  protected List<String> getConfigurationsToCreate() {
+    return confClassnames;
+  }
+
+  /**
+   * This creates all the configurations defined by
+   * {@link #getConfigurationsToCreate()} , ensuring that
+   * the resources have been pushed in.
+   * If one cannot be loaded it is logged and the operation continues
+   * except in the case that the class does load but it isn't actually
+   * a subclass of {@link Configuration}.
+   * @throws ExitUtil.ExitException if a loaded class is of the wrong type
+   */
+  @VisibleForTesting
+  public int loadConfigurationClasses() {
+    List<String> toCreate = getConfigurationsToCreate();
+    int loaded = 0;
+    for (String classname : toCreate) {
+      try {
+        Class<?> loadClass = getClassLoader().loadClass(classname);
+        Object instance = loadClass.getConstructor().newInstance();
+        if (!(instance instanceof Configuration)) {
+          throw new ExitUtil.ExitException(EXIT_SERVICE_CREATION_FAILURE,
+              "Could not create " + classname
+              + " because it is not a Configuration class/subclass");
+        }
+        loaded++;
+      } catch (ClassNotFoundException e) {
+        // class could not be found -implies it is not on the current classpath
+        LOG.debug("Failed to load {} because it is not on the classpath",
+            classname);
+      } catch (ExitUtil.ExitException e) {
+        // rethrow
+        throw e;
+      } catch (Exception e) {
+        // any other exception
+        LOG.info("Failed to create {}", classname, e);
+      }
+    }
+    return loaded;
+  }
+
+  /**
+   * Launch a service catching all exceptions and downgrading them to exit codes
+   * after logging.
+   *
+   * Sets {@link #serviceException} to this value.
+   * @param conf configuration to use
+   * @param processedArgs command line after the launcher-specific arguments
+   * have been stripped out.
+   * @param addShutdownHook should a shutdown hook be added to terminate
+   * this service on shutdown. Tests should set this to false.
+   * @param execute execute/wait for the service to stop.
+   * @return an exit exception, which will have a status code of 0 if it worked
+   */
+  @VisibleForTesting
+  public ExitUtil.ExitException launchService(Configuration conf,
+      List<String> processedArgs,
+      boolean addShutdownHook,
+      boolean execute) {
+    
+    ExitUtil.ExitException exitException;
+    
+    try {
+      int exitCode = coreServiceLaunch(conf, processedArgs, addShutdownHook,
+          execute);
+      if (service != null) {
+        // check to see if the service failed
+        Throwable failure = service.getFailureCause();
+        if (failure != null) {
+          // the service exited with a failure.
+          // check what state it is in
+          Service.STATE failureState = service.getFailureState();
+          if (failureState == Service.STATE.STOPPED) {
+            // the failure occurred during shutdown, not important enough
+            // to bother the user as it may just scare them
+            LOG.debug("Failure during shutdown: {} ", failure, failure);
+          } else {
+            //throw it for the catch handlers to deal with
+            throw failure;
+          }
+        }
+      }
+      String name = getServiceName();
+
+      if (exitCode == 0) {
+        exitException = new ServiceLaunchException(exitCode,
+            "%s succeeded",
+            name);
+      } else {
+        exitException = new ServiceLaunchException(exitCode,
+            "%s failed ", name);
+      }
+      // either the service succeeded, or an error raised during shutdown, 
+      // which we don't worry that much about
+    } catch (ExitUtil.ExitException ee) {
+      // exit exceptions are passed through unchanged
+      exitException = ee;
+    } catch (Throwable thrown) {
+      exitException = convertToExitException(thrown);
+    }
+    noteException(exitException);
+    return exitException;
+  }
+
+  /**
+   * Launch the service.
+   *
+   * All exceptions that occur are propagated upwards.
+   *
+   * If the method returns a status code, it means that it got as far starting
+   * the service, and if it implements {@link LaunchableService}, that the 
+   * method {@link LaunchableService#execute()} has completed. 
+   *
+   * After this method returns, the service can be retrieved returned by
+   * {@link #getService()}.
+   *
+   * @param conf configuration
+   * @param processedArgs arguments after the configuration parameters
+   * have been stripped out.
+   * @param addShutdownHook should a shutdown hook be added to terminate
+   * this service on shutdown. Tests should set this to false.
+   * @param execute execute/wait for the service to stop
+   * @throws ClassNotFoundException classname not on the classpath
+   * @throws IllegalAccessException not allowed at the class
+   * @throws InstantiationException not allowed to instantiate it
+   * @throws InterruptedException thread interrupted
+   * @throws ExitUtil.ExitException any exception defining the status code.
+   * @throws Exception any other failure -if it implements
+   * {@link ExitCodeProvider} then it defines the exit code for any
+   * containing exception
+   */
+
+  protected int coreServiceLaunch(Configuration conf,
+      List<String> processedArgs,
+      boolean addShutdownHook,
+      boolean execute) throws Exception {
+
+    // create the service instance
+    instantiateService(conf);
+    ServiceShutdownHook shutdownHook = null;
+
+    // and the shutdown hook if requested
+    if (addShutdownHook) {
+      shutdownHook = new ServiceShutdownHook(service);
+      shutdownHook.register(SHUTDOWN_PRIORITY);
+    }
+    String name = getServiceName();
+    LOG.debug("Launched service {}", name);
+    LaunchableService launchableService = null;
+
+    if (service instanceof LaunchableService) {
+      // it's a LaunchableService, pass in the conf and arguments before init)
+      LOG.debug("Service {} implements LaunchableService", name);
+      launchableService = (LaunchableService) service;
+      if (launchableService.isInState(Service.STATE.INITED)) {
+        LOG.warn("LaunchableService {}" 
+            + " initialized in constructor before CLI arguments passed in",
+            name);
+      }
+      Configuration newconf = launchableService.bindArgs(configuration,
+          processedArgs);
+      if (newconf != null) {
+        configuration = newconf;
+      }
+    }
+
+    //some class constructors init; here this is picked up on.
+    if (!service.isInState(Service.STATE.INITED)) {
+      service.init(configuration);
+    }
+    int exitCode;
+
+    try {
+      // start the service
+      service.start();
+      exitCode = EXIT_SUCCESS;
+      if (execute && service.isInState(Service.STATE.STARTED)) {
+        if (launchableService != null) {
+          // assume that runnable services are meant to run from here
+          try {
+            exitCode = launchableService.execute();
+            LOG.debug("Service {} execution returned exit code {}",
+                name, exitCode);
+          } finally {
+            // then stop the service
+            service.stop();
+          }
+        } else {
+          //run the service until it stops or an interrupt happens
+          // on a different thread.
+          LOG.debug("waiting for service threads to terminate");
+          service.waitForServiceToStop(0);
+        }
+      }
+    } finally {
+      if (shutdownHook != null) {
+        shutdownHook.unregister();
+      }
+    }
+    return exitCode;
+  }
+
+  /**
+   * Instantiate the service defined in {@code serviceClassName}.
+   *
+   * Sets the {@code configuration} field
+   * to the the value of {@code conf},
+   * and the {@code service} field to the service created.
+   *
+   * @param conf configuration to use
+   */
+  @SuppressWarnings("unchecked")
+  public Service instantiateService(Configuration conf) {
+    Preconditions.checkArgument(conf != null, "null conf");
+    Preconditions.checkArgument(serviceClassName != null,
+        "null service classname");
+    Preconditions.checkArgument(!serviceClassName.isEmpty(),
+        "undefined service classname");
+    configuration = conf;
+
+    // Instantiate the class. this requires the service to have a public
+    // zero-argument or string-argument constructor
+    Object instance;
+    try {
+      Class<?> serviceClass = getClassLoader().loadClass(serviceClassName);
+      try {
+        instance = serviceClass.getConstructor().newInstance();
+      } catch (NoSuchMethodException noEmptyConstructor) {
+        // no simple constructor, fall back to a string
+        LOG.debug("No empty constructor {}", noEmptyConstructor,
+            noEmptyConstructor);
+        instance = serviceClass.getConstructor(String.class)
+                               .newInstance(serviceClassName);
+      }
+    } catch (Exception e) {
+      throw serviceCreationFailure(e);
+    }
+    if (!(instance instanceof Service)) {
+      //not a service
+      throw new ServiceLaunchException(
+          LauncherExitCodes.EXIT_SERVICE_CREATION_FAILURE,
+          "Not a service class: \"%s\"", serviceClassName);
+    }
+
+    // cast to the specific instance type of this ServiceLauncher
+    service = (S) instance;
+    return service;
+  }
+
+  /**
+   * Convert an exception to an {@code ExitException}.
+   *
+   * This process may just be a simple pass through, otherwise a new
+   * exception is created with an exit code, the text of the supplied
+   * exception, and the supplied exception as an inner cause.
+   * 
+   * <ol>
+   *   <li>If is already the right type, pass it through.</li>
+   *   <li>If it implements {@link ExitCodeProvider#getExitCode()},
+   *   the exit code is extracted and used in the new exception.</li>
+   *   <li>Otherwise, the exit code
+   *   {@link LauncherExitCodes#EXIT_EXCEPTION_THROWN} is used.</li>
+   * </ol>
+   *  
+   * @param thrown the exception thrown
+   * @return an {@code ExitException} with a status code
+   */
+  protected static ExitUtil.ExitException convertToExitException(
+      Throwable thrown) {
+    ExitUtil.ExitException exitException;
+    // get the exception message
+    String message = thrown.toString();
+    int exitCode;
+    if (thrown instanceof ExitCodeProvider) {
+      // the exception provides a status code -extract it
+      exitCode = ((ExitCodeProvider) thrown).getExitCode();
+      message = thrown.getMessage();
+      if (message == null) {
+        // some exceptions do not have a message; fall back
+        // to the string value.
+        message = thrown.toString();
+      }
+    } else {
+      // no exception code: use the default
+      exitCode = EXIT_EXCEPTION_THROWN;
+    }
+    // construct the new exception with the original message and
+    // an exit code
+    exitException = new ServiceLaunchException(exitCode, message);
+    exitException.initCause(thrown);
+    return exitException;
+  }
+
+  /**
+   * Generate an exception announcing a failure to create the service.
+   * @param exception inner exception.
+   * @return a new exception, with the exit code
+   * {@link LauncherExitCodes#EXIT_SERVICE_CREATION_FAILURE}
+   */
+  protected ServiceLaunchException serviceCreationFailure(Exception exception) {
+    return new ServiceLaunchException(EXIT_SERVICE_CREATION_FAILURE, exception);
+  }
+  
+  /**
+   * Override point: register this class as the handler for the control-C
+   * and SIGINT interrupts.
+   *
+   * Subclasses can extend this with extra operations, such as
+   * an exception handler:
+   * <pre>
+   *  Thread.setDefaultUncaughtExceptionHandler(
+   *     new YarnUncaughtExceptionHandler());
+   * </pre>
+   */
+  protected void registerFailureHandling() {
+    try {
+      interruptEscalator = new InterruptEscalator(this,
+          SHUTDOWN_TIME_ON_INTERRUPT);
+      interruptEscalator.register(IrqHandler.CONTROL_C);
+      interruptEscalator.register(IrqHandler.SIGTERM);
+    } catch (IllegalArgumentException e) {
+      // downgrade interrupt registration to warnings
+      LOG.warn("{}", e, e);
+    }
+    Thread.setDefaultUncaughtExceptionHandler(
+        new HadoopUncaughtExceptionHandler(this));
+  }
+
+  /**
+   * Handler for uncaught exceptions: terminate the service.
+   * @param thread thread
+   * @param exception exception
+   */
+  @Override
+  public void uncaughtException(Thread thread, Throwable exception) {
+    LOG.error("Uncaught exception in thread {} -exiting", thread, exception);
+    exit(convertToExitException(exception));
+  }
+
+  /**
+   * Get the service name via {@link Service#getName()}.
+   *
+   * If the service is not instantiated, the classname is returned instead.
+   * @return the service name
+   */
+  public String getServiceName() {
+    Service s = service;
+    String name = null;
+    if (s != null) {
+      try {
+        name = s.getName();
+      } catch (Exception ignored) {
+        // ignored
+      }
+    }
+    if (name != null) {
+      return "service " + name;
+    } else {
+      return "service " + serviceName;
+    }
+  }
+  
+  /**
+   * Print a warning message.
+   * <p>
+   * This tries to log to the log's warn() operation.
+   * If the log at that level is disabled it logs to system error
+   * @param text warning text
+   */
+  protected void warn(String text) {
+    if (LOG.isWarnEnabled()) {
+      LOG.warn(text);
+    } else {
+      System.err.println(text);
+    }
+  }
+
+  /**
+   * Report an error. 
+   * <p>
+   * This tries to log to {@code LOG.error()}.
+   * <p>
+   * If that log level is disabled disabled the message
+   * is logged to system error along with {@code thrown.toString()}
+   * @param message message for the user
+   * @param thrown the exception thrown
+   */
+  protected void error(String message, Throwable thrown) {
+    String text = "Exception: " + message;
+    if (LOG.isErrorEnabled()) {
+      LOG.error(text, thrown);
+    } else {
+      System.err.println(text);
+      if (thrown != null) {
+        System.err.println(thrown.toString());
+      }
+    }
+  }
+  
+  /**
+   * Exit the JVM.
+   *
+   * This is method can be overridden for testing, throwing an 
+   * exception instead. Any subclassed method MUST raise an 
+   * {@code ExitException} instance/subclass.
+   * The service launcher code assumes that after this method is invoked,
+   * no other code in the same method is called.
+   * @param exitCode code to exit
+   */
+  protected void exit(int exitCode, String message) {
+    ExitUtil.terminate(exitCode, message);
+  }
+
+  /**
+   * Exit the JVM using an exception for the exit code and message,
+   * invoking {@link ExitUtil#terminate(ExitUtil.ExitException)}.
+   *
+   * This is the standard way a launched service exits.
+   * An error code of 0 means success -nothing is printed.
+   *
+   * If {@link ExitUtil#disableSystemExit()} has been called, this
+   * method will throw the exception.
+   *
+   * The method <i>may</i> be subclassed for testing
+   * @param ee exit exception
+   * @throws ExitUtil.ExitException if ExitUtil exceptions are disabled
+   */
+  protected void exit(ExitUtil.ExitException ee) {
+    ExitUtil.terminate(ee);
+  }
+
+  /**
+   * Override point: get the classloader to use.
+   * @return the classloader for loading a service class.
+   */
+  protected ClassLoader getClassLoader() {
+    return this.getClass().getClassLoader();
+  }
+
+  /**
+   * Extract the command options and apply them to the configuration,
+   * building an array of processed arguments to hand down to the service.
+   *
+   * @param conf configuration to update.
+   * @param args main arguments. {@code args[0]}is assumed to be
+   * the service classname and is skipped.
+   * @return the remaining arguments
+   * @throws ExitUtil.ExitException if JVM exiting is disabled.
+   */
+  public List<String> extractCommandOptions(Configuration conf,
+      List<String> args) {
+    int size = args.size();
+    if (size <= 1) {
+      return new ArrayList<>(0);
+    }
+    List<String> coreArgs = args.subList(1, size);
+
+    return parseCommandArgs(conf, coreArgs);
+  }
+
+  /**
+   * Parse the command arguments, extracting the service class as the last
+   * element of the list (after extracting all the rest).
+   *
+   * The field {@link #commandOptions} field must already have been set.
+   * @param conf configuration to use
+   * @param args command line argument list
+   * @return the remaining arguments
+   * @throws ServiceLaunchException if processing of arguments failed
+   */
+  protected List<String> parseCommandArgs(Configuration conf,
+      List<String> args) {
+    Preconditions.checkNotNull(commandOptions,
+        "Command options have not been created");
+    StringBuilder argString = new StringBuilder(args.size() * 32);
+    for (String arg : args) {
+      argString.append("\"").append(arg).append("\" ");
+    }
+    LOG.debug("Command line: {}", argString);
+    try {
+      String[] argArray = args.toArray(new String[args.size()]);
+      // parse this the standard way. This will
+      // update the configuration in the parser, and potentially
+      // patch the user credentials
+      GenericOptionsParser parser = createGenericOptionsParser(conf, argArray);
+      if (!parser.isParseSuccessful()) {
+        throw new ServiceLaunchException(EXIT_COMMAND_ARGUMENT_ERROR,
+            E_PARSE_FAILED + " %s", argString);
+      }
+      CommandLine line = parser.getCommandLine();
+      List<String> remainingArgs = Arrays.asList(parser.getRemainingArgs());
+      LOG.debug("Remaining arguments {}", remainingArgs);
+
+      // Scan the list of configuration files
+      // and bail out if they don't exist
+      if (line.hasOption(ARG_CONF)) {
+        String[] filenames = line.getOptionValues(ARG_CONF);
+        verifyConfigurationFilesExist(filenames);
+        // Add URLs of files as list of URLs to load
+        for (String filename : filenames) {
+          File file = new File(filename);
+          LOG.debug("Configuration files {}", file);
+          confResourceUrls.add(file.toURI().toURL());
+        }
+      }
+      if (line.hasOption(ARG_CONFCLASS)) {
+        // new resources to instantiate as configurations
+        List<String> classnameList = Arrays.asList(
+            line.getOptionValues(ARG_CONFCLASS));
+        LOG.debug("Configuration classes {}", classnameList);
+        confClassnames.addAll(classnameList);
+      }
+      // return the remainder
+      return remainingArgs;
+    } catch (IOException e) {
+      // parsing problem: convert to a command argument error with
+      // the original text
+      throw new ServiceLaunchException(EXIT_COMMAND_ARGUMENT_ERROR, e);
+    } catch (RuntimeException e) {
+      // lower level issue such as XML parse failure
+      throw new ServiceLaunchException(EXIT_COMMAND_ARGUMENT_ERROR,
+          E_PARSE_FAILED + " %s : %s", argString, e);
+    }
+  }
+
+  /**
+   * Override point: create a generic options parser or subclass thereof.
+   * @param conf Hadoop configuration
+   * @param argArray array of arguments
+   * @return a generic options parser to parse the arguments
+   * @throws IOException on any failure
+   */
+  protected GenericOptionsParser createGenericOptionsParser(Configuration conf,
+      String[] argArray) throws IOException {
+    return new MinimalGenericOptionsParser(conf, commandOptions, argArray);
+  }
+
+  /**
+   * Verify that all the specified filenames exist.
+   * @param filenames a list of files
+   * @throws ServiceLaunchException if a file is not found
+   */
+  protected void verifyConfigurationFilesExist(String[] filenames) {
+    if (filenames == null) {
+      return;
+    }
+    for (String filename : filenames) {
+      File file = new File(filename);
+      LOG.debug("Conf file {}", file.getAbsolutePath());
+      if (!file.exists()) {
+        // no configuration file
+        throw new ServiceLaunchException(EXIT_NOT_FOUND,
+            ARG_CONF_PREFIXED + ": configuration file not found: %s",
+            file.getAbsolutePath());
+      }
+    }
+  }
+
+  /**
+   * Build a log message for starting up and shutting down. 
+   * @param classname the class of the server
+   * @param args arguments
+   */
+  protected static String startupShutdownMessage(String classname,
+      List<String> args) {
+    final String hostname = NetUtils.getHostname();
+
+    return StringUtils.createStartupShutdownMessage(classname, hostname,
+        args.toArray(new String[args.size()]));
+  }
+
+  /**
+   * Exit with a printed message. 
+   * @param status status code
+   * @param message message message to print before exiting
+   * @throws ExitUtil.ExitException if exceptions are disabled
+   */
+  protected static void exitWithMessage(int status, String message) {
+    ExitUtil.terminate(new ServiceLaunchException(status, message));
+  }
+
+  /**
+   * Exit with the usage exit code {@link #EXIT_USAGE}
+   * and message {@link #USAGE_MESSAGE}.
+   * @throws ExitUtil.ExitException if exceptions are disabled
+   */
+  protected static void exitWithUsageMessage() {
+    exitWithMessage(EXIT_USAGE, USAGE_MESSAGE);
+  }
+
+  /**
+   * This is the JVM entry point for the service launcher.
+   *
+   * Converts the arguments to a list, then invokes {@link #serviceMain(List)}
+   * @param args command line arguments.
+   */
+  public static void main(String[] args) {
+    serviceMain(Arrays.asList(args));
+  }
+
+  /**
+   * Varargs version of the entry point for testing and other in-JVM use.
+   * Hands off to {@link #serviceMain(List)}
+   * @param args command line arguments.
+   */
+  public static void serviceMain(String... args) {
+    serviceMain(Arrays.asList(args));
+  }
+
+  /* ====================================================================== */
+  /**
+   * The real main function, which takes the arguments as a list.
+   * Argument 0 MUST be the service classname
+   * @param argsList the list of arguments
+   */
+  /* ====================================================================== */
+
+  public static void serviceMain(List<String> argsList) {
+    if (argsList.isEmpty()) {
+      // no arguments: usage message
+      exitWithUsageMessage();
+    } else {
+      ServiceLauncher<Service> serviceLauncher =
+          new ServiceLauncher<>(argsList.get(0));
+      serviceLauncher.launchServiceAndExit(argsList);
+    }
+  }
+
+  /**
+   * A generic options parser which does not parse any of the traditional
+   * Hadoop options.
+   */
+  protected static class MinimalGenericOptionsParser
+      extends GenericOptionsParser {
+    public MinimalGenericOptionsParser(Configuration conf,
+        Options options, String[] args) throws IOException {
+      super(conf, options, args);
+    }
+
+    @Override
+    protected Options buildGeneralOptions(Options opts) {
+      return opts;
+    }
+  }
+}

+ 112 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/service/launcher/ServiceShutdownHook.java

@@ -0,0 +1,112 @@
+/*
+ * 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 java.lang.ref.WeakReference;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.service.Service;
+import org.apache.hadoop.util.ShutdownHookManager;
+
+/**
+ * JVM Shutdown hook for Service which will stop the
+ * Service gracefully in case of JVM shutdown.
+ * This hook uses a weak reference to the service,
+ * and when shut down, calls {@link Service#stop()} if the reference is valid.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public class ServiceShutdownHook implements Runnable {
+  private static final Logger LOG = LoggerFactory.getLogger(
+      ServiceShutdownHook.class);
+
+  /**
+   * A weak reference to the service.
+   */
+  private final WeakReference<Service> serviceRef;
+
+  /**
+   * Create an instance.
+   * @param service the service
+   */
+  public ServiceShutdownHook(Service service) {
+    serviceRef = new WeakReference<>(service);
+  }
+
+  /**
+   * Register the service for shutdown with Hadoop's
+   * {@link ShutdownHookManager}.
+   * @param priority shutdown hook priority
+   */
+  public synchronized void register(int priority) {
+    unregister();
+    ShutdownHookManager.get().addShutdownHook(this, priority);
+  }
+
+  /**
+   * Unregister the hook.
+   */
+  public synchronized void unregister() {
+    try {
+      ShutdownHookManager.get().removeShutdownHook(this);
+    } catch (IllegalStateException e) {
+      LOG.info("Failed to unregister shutdown hook: {}", e, e);
+    }
+  }
+
+  /**
+   * Shutdown handler.
+   * Query the service hook reference -if it is still valid the 
+   * {@link Service#stop()} operation is invoked.
+   */
+  @Override
+  public void run() {
+    shutdown();
+  }
+
+  /**
+   * Shutdown operation.
+   * <p>
+   * Subclasses may extend it, but it is primarily
+   * made available for testing.
+   * @return true if the service was stopped and no exception was raised.
+   */
+  protected boolean shutdown() {
+    Service service;
+    boolean result = false;
+    synchronized (this) {
+      service = serviceRef.get();
+      serviceRef.clear();
+    }
+    if (service != null) {
+      try {
+        // Stop the  Service
+        service.stop();
+        result = true;
+      } catch (Throwable t) {
+        LOG.info("Error stopping {}", service.getName(), t);
+      }
+    }
+    return result;
+  }
+}

+ 462 - 0
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;

+ 35 - 0
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();
+}

+ 194 - 59
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ExitUtil.java

@@ -17,41 +17,123 @@
  */
  */
 package org.apache.hadoop.util;
 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.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 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
 @InterfaceStability.Unstable
 public final class ExitUtil {
 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 systemExitDisabled = false;
   private static volatile boolean systemHaltDisabled = false;
   private static volatile boolean systemHaltDisabled = false;
   private static volatile ExitException firstExitException;
   private static volatile ExitException firstExitException;
   private static volatile HaltException firstHaltException;
   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;
     private static final long serialVersionUID = 1L;
+    /**
+     * The status code.
+     */
     public final int status;
     public final int status;
 
 
     public ExitException(int status, String msg) {
     public ExitException(int status, String msg) {
       super(msg);
       super(msg);
       this.status = status;
       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;
     private static final long serialVersionUID = 1L;
     public final int status;
     public final int status;
 
 
+    public HaltException(int status, Throwable cause) {
+      super(cause);
+      this.status = status;
+    }
+
     public HaltException(int status, String msg) {
     public HaltException(int status, String msg) {
       super(msg);
       super(msg);
       this.status = status;
       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() {
   public static boolean terminateCalled() {
     // Either we set this member or we actually called System#exit
     // 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() {
   public static boolean haltCalled() {
     return firstHaltException != null;
     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() {
   public static ExitException getFirstExitException() {
     return firstExitException;
     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() {
   public static HaltException getFirstHaltException() {
     return firstHaltException;
     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) {
     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;
         firstExitException = ee;
       }
       }
       throw ee;
       throw ee;
@@ -135,20 +217,26 @@ public final class ExitUtil {
 
 
   /**
   /**
    * Forcibly terminates the currently running Java virtual machine.
    * 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) {
     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;
         firstHaltException = ee;
       }
       }
       throw 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 {
   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.
    * 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 {
   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 {
   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.
    * 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 {
   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);
   }
   }
 }
 }

+ 71 - 54
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
  * See the License for the specific language governing permissions and
  * limitations under the License.
  * 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.FileNotFoundException;
 import java.io.IOException;
 import java.io.IOException;
 import java.io.PrintStream;
 import java.io.PrintStream;
@@ -118,6 +116,7 @@ public class GenericOptionsParser {
   private static final Log LOG = LogFactory.getLog(GenericOptionsParser.class);
   private static final Log LOG = LogFactory.getLog(GenericOptionsParser.class);
   private Configuration conf;
   private Configuration conf;
   private CommandLine commandLine;
   private CommandLine commandLine;
+  private final boolean parseSuccessful;
 
 
   /**
   /**
    * Create an options parser with the given options to parse the args.
    * Create an options parser with the given options to parse the args.
@@ -171,7 +170,7 @@ public class GenericOptionsParser {
   public GenericOptionsParser(Configuration conf,
   public GenericOptionsParser(Configuration conf,
       Options options, String[] args) throws IOException {
       Options options, String[] args) throws IOException {
     this.conf = conf;
     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")
   @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:///
    * and verifies if they exist. It defaults for file:///
    * if the files specified do not have a scheme.
    * if the files specified do not have a scheme.
    * it returns the paths uri converted defaulting to file:///.
    * 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 opts Options to use for parsing args.
    * @param args User-specified arguments
    * @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 {
       throws IOException {
     opts = buildGeneralOptions(opts);
     opts = buildGeneralOptions(opts);
     CommandLineParser parser = new GnuParser();
     CommandLineParser parser = new GnuParser();
+    boolean parsed = false;
     try {
     try {
       commandLine = parser.parse(opts, preProcessForWindows(args), true);
       commandLine = parser.parse(opts, preProcessForWindows(args), true);
       processGeneralOptions(commandLine);
       processGeneralOptions(commandLine);
+      parsed = true;
     } catch(ParseException e) {
     } catch(ParseException e) {
       LOG.warn("options parsing failed: "+e.getMessage());
       LOG.warn("options parsing failed: "+e.getMessage());
 
 
       HelpFormatter formatter = new HelpFormatter();
       HelpFormatter formatter = new HelpFormatter();
       formatter.printHelp("general options are: ", opts);
       formatter.printHelp("general options are: ", opts);
     }
     }
+    return parsed;
   }
   }
 
 
   /**
   /**

+ 26 - 17
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
    * @param msg content of the message
    * @return a message for logging
    * @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);
     StringBuilder b = new StringBuilder(prefix);
     b.append("\n/************************************************************");
     b.append("\n/************************************************************");
     for(String s : msg)
     for(String s : msg)
-      b.append("\n" + prefix + s);
+      b.append("\n").append(prefix).append(s);
     b.append("\n************************************************************/");
     b.append("\n************************************************************/");
     return b.toString();
     return b.toString();
   }
   }
@@ -711,21 +711,7 @@ public class StringUtils {
                                      final LogAdapter LOG) { 
                                      final LogAdapter LOG) { 
     final String hostname = NetUtils.getHostname();
     final String hostname = NetUtils.getHostname();
     final String classname = clazz.getSimpleName();
     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) {
     if (SystemUtils.IS_OS_UNIX) {
       try {
       try {
@@ -745,6 +731,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,
    * The traditional binary prefixes, kilo, mega, ..., exa,
    * which can be represented by a 64-bit integer.
    * which can be represented by a 64-bit integer.

+ 15 - 8
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/service/BreakableService.java

@@ -20,8 +20,6 @@
 package org.apache.hadoop.service;
 package org.apache.hadoop.service;
 
 
 import org.apache.hadoop.conf.Configuration;
 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
  * 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)];
     return counts[convert(state)];
   }
   }
 
 
-  private void maybeFail(boolean fail, String action) {
+  private void maybeFail(boolean fail, String action) throws Exception {
     if (fail) {
     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
   @Override
   protected void serviceInit(Configuration conf) throws Exception {
   protected void serviceInit(Configuration conf) throws Exception {
     inc(STATE.INITED);
     inc(STATE.INITED);
@@ -83,13 +90,13 @@ public class BreakableService extends AbstractService {
   }
   }
 
 
   @Override
   @Override
-  protected void serviceStart() {
+  protected void serviceStart() throws Exception {
     inc(STATE.STARTED);
     inc(STATE.STARTED);
     maybeFail(failOnStart, "start");
     maybeFail(failOnStart, "start");
   }
   }
 
 
   @Override
   @Override
-  protected void serviceStop() {
+  protected void serviceStop() throws Exception {
     inc(STATE.STOPPED);
     inc(STATE.STOPPED);
     maybeFail(failOnStop, "stop");
     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 {
   public static class BrokenLifecycleEvent extends RuntimeException {
 
 
-    final STATE state;
+    public final STATE state;
 
 
     public BrokenLifecycleEvent(Service service, String action) {
     public BrokenLifecycleEvent(Service service, String action) {
       super("Lifecycle Failure during " + action + " state is "
       super("Lifecycle Failure during " + action + " state is "

+ 317 - 0
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;
+    }
+  }
+
+}

+ 59 - 0
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;
+  }
+}

+ 146 - 0
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());
+  }
+
+}

+ 118 - 0
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;
+    }
+  }
+
+}

+ 213 - 0
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);
+  }
+
+}

+ 83 - 0
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);
+  }
+
+}

+ 95 - 0
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);
+  }
+
+}

+ 96 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/service/launcher/testservices/ExceptionInExecuteLaunchableService.java

@@ -0,0 +1,96 @@
+/*
+ * 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.testservices;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.service.launcher.AbstractLaunchableService;
+import org.apache.hadoop.service.launcher.LauncherExitCodes;
+import org.apache.hadoop.service.launcher.ServiceLaunchException;
+import org.apache.hadoop.util.ExitCodeProvider;
+
+import java.io.IOException;
+import java.util.List;
+
+/**
+ * Raise an exception in the execute() method; the exception type can
+ * be configured from the CLI.
+ */
+public class ExceptionInExecuteLaunchableService extends
+    AbstractLaunchableService {
+
+  public static final String NAME =
+      "org.apache.hadoop.service.launcher.testservices.ExceptionInExecuteLaunchableService";
+  public static final String ARG_THROW_SLE = "--SLE";
+  public static final String ARG_THROW_IOE = "--IOE";
+  public static final String ARG_THROWABLE = "--throwable";
+  public static final String SLE_TEXT = "SLE raised in execute()";
+  public static final String OTHER_EXCEPTION_TEXT = "Other exception";
+
+  public static final String EXIT_IN_IOE_TEXT = "Exit in IOE";
+  public static final int IOE_EXIT_CODE = 64;
+  private ExType exceptionType = ExType.EX;
+
+  public ExceptionInExecuteLaunchableService() {
+    super("ExceptionInExecuteLaunchedService");
+  }
+
+  @Override
+  public Configuration bindArgs(Configuration config, List<String> args) throws
+      Exception {
+    if (args.contains(ARG_THROW_SLE)) {
+      exceptionType = ExType.SLE;
+    } else if (args.contains(ARG_THROW_IOE)) {
+      exceptionType = ExType.IOE;
+    } else if (args.contains(ARG_THROWABLE)) {
+      exceptionType = ExType.THROWABLE;
+    }
+    return super.bindArgs(config, args);
+  }
+
+  @Override
+  public int execute() throws Exception {
+    switch (exceptionType) {
+    case SLE:
+      throw new ServiceLaunchException(LauncherExitCodes.EXIT_OTHER_FAILURE,
+          SLE_TEXT);
+    case IOE:
+      throw new IOECodedException();
+    case THROWABLE:
+      throw new OutOfMemoryError("OOM");
+    case EX:
+    default:
+      throw new Exception(OTHER_EXCEPTION_TEXT);
+    }
+  }
+
+  enum ExType {EX, SLE, IOE, THROWABLE}
+
+  public static class IOECodedException extends IOException implements
+      ExitCodeProvider {
+
+    public IOECodedException() {
+      super(EXIT_IN_IOE_TEXT);
+    }
+
+    @Override
+    public int getExitCode() {
+      return IOE_EXIT_CODE;
+    }
+  }
+}

+ 33 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/service/launcher/testservices/FailInConstructorService.java

@@ -0,0 +1,33 @@
+/*
+ * 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.testservices;
+
+/**
+ * Service which fails in its constructor.
+ */
+public class FailInConstructorService extends FailureTestService {
+
+  public static final String NAME =
+      "org.apache.hadoop.service.launcher.testservices.FailInConstructorService";
+
+  public FailInConstructorService() {
+    super(false, false, false, 0);
+    throw new NullPointerException("oops");
+  }
+}

+ 38 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/service/launcher/testservices/FailInInitService.java

@@ -0,0 +1,38 @@
+/*
+ * 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.testservices;
+
+/**
+ * Service which fails in its init() operation.
+ */
+public class FailInInitService extends FailureTestService {
+  public static final String NAME =
+      "org.apache.hadoop.service.launcher.testservices.FailInInitService";
+  public static final int EXIT_CODE = -1;
+
+  public FailInInitService() {
+    super(true, false, false, 0
+    );
+  }
+
+  @Override
+  int getExitCode() {
+    return EXIT_CODE;
+  }
+}

+ 37 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/service/launcher/testservices/FailInStartService.java

@@ -0,0 +1,37 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.service.launcher.testservices;
+
+/**
+ * Service which fails in its start() operation.
+ */
+public class FailInStartService extends FailureTestService {
+  public static final String NAME =
+      "org.apache.hadoop.service.launcher.testservices.FailInStartService";
+  public static final int EXIT_CODE = -2;
+
+  public FailInStartService() {
+    super(false, true, false, 0);
+  }
+
+  @Override
+  int getExitCode() {
+    return EXIT_CODE;
+  }
+}

+ 47 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/service/launcher/testservices/FailingStopInStartService.java

@@ -0,0 +1,47 @@
+/*
+ * 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.testservices;
+
+/**
+ * This service stops during its start operation.
+ */
+public class FailingStopInStartService extends FailureTestService {
+  public static final String NAME =
+      "org.apache.hadoop.service.launcher.testservices.FailingStopInStartService";
+  public static final int EXIT_CODE = -4;
+
+  public FailingStopInStartService() {
+    super(false, false, true, 0);
+  }
+
+  @Override
+  protected void serviceStart() throws Exception {
+    super.serviceStart();
+    try {
+      stop();
+    } catch (Exception e) {
+      //this is secretly swallowed
+    }
+  }
+
+  @Override
+  int getExitCode() {
+    return EXIT_CODE;
+  }
+}

+ 55 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/service/launcher/testservices/FailureTestService.java

@@ -0,0 +1,55 @@
+/*
+ * 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.testservices;
+
+import org.apache.hadoop.service.BreakableService;
+import org.apache.hadoop.service.launcher.ServiceLaunchException;
+
+/**
+ * Launcher test service that does not take CLI arguments.
+ */
+public class FailureTestService extends BreakableService {
+
+  private final int delay;
+
+  public FailureTestService(boolean failOnInit,
+      boolean failOnStart,
+      boolean failOnStop,
+      int delay) {
+    super(failOnInit, failOnStart, failOnStop);
+    this.delay = delay;
+  }
+
+  @Override
+  protected void serviceStop() throws Exception {
+    if (delay > 0) {
+      Thread.sleep(delay);
+    }
+    super.serviceStop();
+  }
+
+  @Override
+  protected Exception createFailureException(String action) {
+    return new ServiceLaunchException(getExitCode(), toString());
+  }
+
+  int getExitCode() {
+    return -1;
+  }
+}

+ 63 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/service/launcher/testservices/InitInConstructorLaunchableService.java

@@ -0,0 +1,63 @@
+/*
+ * 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.testservices;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.service.launcher.AbstractLaunchableService;
+import org.junit.Assert;
+
+import java.util.List;
+
+/**
+ * Init in the constructor and make sure that it isn't inited again.
+ */
+public class InitInConstructorLaunchableService extends
+    AbstractLaunchableService {
+
+  public static final String NAME =
+      "org.apache.hadoop.service.launcher.testservices.InitInConstructorLaunchableService";
+  private final Configuration originalConf = new Configuration();
+
+  public InitInConstructorLaunchableService() {
+    super("InitInConstructorLaunchableService");
+    init(originalConf);
+  }
+
+  @Override
+  public void init(Configuration conf) {
+    Assert.assertEquals(STATE.NOTINITED, getServiceState());
+    super.init(conf);
+  }
+
+  @Override
+  public Configuration bindArgs(Configuration config, List<String> args)
+      throws Exception {
+    Assert.assertEquals(STATE.INITED, getServiceState());
+    Assert.assertTrue(isInState(STATE.INITED));
+    Assert.assertNotSame(getConfig(), config);
+    return null;
+  }
+
+  @Override
+  public int execute() throws Exception {
+    Assert.assertEquals(STATE.STARTED, getServiceState());
+    Assert.assertSame(originalConf, getConfig());
+    return super.execute();
+  }
+}

+ 111 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/service/launcher/testservices/LaunchableRunningService.java

@@ -0,0 +1,111 @@
+/*
+ * 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.testservices;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.service.launcher.LaunchableService;
+import org.apache.hadoop.service.launcher.LauncherExitCodes;
+import org.junit.Assert;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.List;
+
+/**
+ * A service which implements {@link LaunchableService}.
+ * It
+ * <ol>
+ *   <li>does nothing in its {@link #serviceStart()}</li>
+ *   <li>does its sleep+ maybe fail operation in its {@link #execute()}
+ *   method</li>
+ *   <li>gets the failing flag from the argument {@link #ARG_FAILING} first,
+ *   the config file second.</li>
+ *   <li>returns 0 for a successful execute</li>
+ *   <li>returns a configurable exit code for a failing execute</li>
+ *   <li>generates a new configuration in {@link #bindArgs(Configuration, List)}
+ *   to verify that these propagate.</li>
+ * </ol>
+ */
+public class LaunchableRunningService extends RunningService implements
+    LaunchableService {
+  public static final String NAME =
+      "org.apache.hadoop.service.launcher.testservices.LaunchableRunningService";
+  public static final String ARG_FAILING = "--failing";
+  public static final String EXIT_CODE_PROP = "exit.code";
+  private static final Logger LOG =
+      LoggerFactory.getLogger(LaunchableRunningService.class);
+  private int exitCode = 0;
+
+  public LaunchableRunningService() {
+    this("LaunchableRunningService");
+  }
+
+  public LaunchableRunningService(String name) {
+    super(name);
+  }
+
+  @Override
+  public Configuration bindArgs(Configuration config, List<String> args) throws
+      Exception {
+    Assert.assertEquals(STATE.NOTINITED, getServiceState());
+    for (String arg : args) {
+      LOG.info(arg);
+    }
+    Configuration newConf = new Configuration(config);
+    if (args.contains(ARG_FAILING)) {
+      LOG.info("CLI contains " + ARG_FAILING);
+      failInRun = true;
+      newConf.setInt(EXIT_CODE_PROP, LauncherExitCodes.EXIT_OTHER_FAILURE);
+    }
+    return newConf;
+  }
+
+  @Override
+  protected void serviceInit(Configuration conf) throws Exception {
+    super.serviceInit(conf);
+    if (conf.getBoolean(FAIL_IN_RUN, false)) {
+      //if the conf value says fail, the exit code goes to it too
+      exitCode = LauncherExitCodes.EXIT_FAIL;
+    }
+    // the exit code can be read off the property
+    exitCode = conf.getInt(EXIT_CODE_PROP, exitCode);
+  }
+
+  @Override
+  protected void serviceStart() throws Exception {
+    // no-op
+  }
+
+  @Override
+  public int execute() throws Exception {
+    Thread.sleep(delayTime);
+    if (failInRun) {
+      return exitCode;
+    }
+    return 0;
+  }
+
+  public int getExitCode() {
+    return exitCode;
+  }
+
+  public void setExitCode(int exitCode) {
+    this.exitCode = exitCode;
+  }
+}

+ 64 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/service/launcher/testservices/NoArgsAllowedService.java

@@ -0,0 +1,64 @@
+/*
+ * 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.testservices;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.service.launcher.AbstractLaunchableService;
+import org.apache.hadoop.service.launcher.LauncherExitCodes;
+import org.apache.hadoop.service.launcher.ServiceLaunchException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.List;
+import java.util.Map;
+
+/**
+ * service that does not allow any arguments.
+ */
+public class NoArgsAllowedService extends AbstractLaunchableService {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(NoArgsAllowedService.class);
+
+  public NoArgsAllowedService() {
+    super("NoArgsAllowedService");
+  }
+
+  public static final String NAME =
+      "org.apache.hadoop.service.launcher.testservices.NoArgsAllowedService";
+  
+  @Override
+  public Configuration bindArgs(Configuration config, List<String> args)
+      throws Exception {
+    Configuration configuration = super.bindArgs(config, args);
+    if (!args.isEmpty()) {
+      StringBuilder argsList = new StringBuilder();
+      for (String arg : args) {
+        argsList.append('"').append(arg).append("\" ");
+      }
+      LOG.error("Got {} arguments: {}", args.size(), argsList);
+      throw new ServiceLaunchException(
+          LauncherExitCodes.EXIT_COMMAND_ARGUMENT_ERROR,
+          "Expected 0 arguments but got %d: %s",
+          args.size(),
+          argsList);
+    }
+    return configuration;
+  }
+}

+ 46 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/service/launcher/testservices/NullBindLaunchableService.java

@@ -0,0 +1,46 @@
+/*
+ * 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.testservices;
+
+import org.apache.hadoop.conf.Configuration;
+
+import java.util.List;
+
+/**
+ * An extension of {@link LaunchableRunningService} which returns null from
+ * the {@link #bindArgs(Configuration, List)} method.
+ */
+public class NullBindLaunchableService extends LaunchableRunningService {
+  public static final String NAME =
+      "org.apache.hadoop.service.launcher.testservices.NullBindLaunchableService";
+
+  public NullBindLaunchableService() {
+    this("NullBindLaunchableService");
+  }
+
+  public NullBindLaunchableService(String name) {
+    super(name);
+  }
+
+  @Override
+  public Configuration bindArgs(Configuration config, List<String> args)
+      throws Exception {
+    return null;
+  }
+}

+ 84 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/service/launcher/testservices/RunningService.java

@@ -0,0 +1,84 @@
+/*
+ * 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.testservices;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.service.AbstractService;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class RunningService extends AbstractService implements Runnable {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(RunningService.class);
+  public static final String NAME =
+      "org.apache.hadoop.service.launcher.testservices.RunningService";
+  public static final int DELAY = 100;
+
+  /**
+   * Property on delay times.
+   */
+  public static final String DELAY_TIME = "delay.time";
+  public static final String FAIL_IN_RUN = "fail.runnable";
+  public static final String FAILURE_MESSAGE = "FAIL_IN_RUN";
+  private boolean interrupted;
+
+  public int delayTime = DELAY;
+  public boolean failInRun;
+
+  public RunningService() {
+    super("RunningService");
+  }
+
+  public RunningService(String name) {
+    super(name);
+  }
+
+  @Override
+  protected void serviceInit(Configuration conf) throws Exception {
+    super.serviceInit(conf);
+    delayTime = getConfig().getInt(DELAY_TIME, delayTime);
+    failInRun = getConfig().getBoolean(FAIL_IN_RUN, failInRun);
+  }
+
+  @Override
+  protected void serviceStart() throws Exception {
+    Thread thread = new Thread(this);
+    thread.setName(getName());
+    thread.start();
+  }
+
+  @Override
+  public void run() {
+    try {
+      Thread.sleep(delayTime);
+      if (failInRun) {
+        noteFailure(new Exception(FAILURE_MESSAGE));
+      }
+    } catch (InterruptedException e) {
+      interrupted = true;
+      LOG.info("Interrupted");
+    }
+    stop();
+  }
+
+  public boolean isInterrupted() {
+    return interrupted;
+  }
+
+}

+ 49 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/service/launcher/testservices/StoppingInStartLaunchableService.java

@@ -0,0 +1,49 @@
+/*
+ * 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.testservices;
+
+import org.apache.hadoop.service.launcher.AbstractLaunchableService;
+import org.apache.hadoop.service.launcher.LauncherExitCodes;
+import org.apache.hadoop.service.launcher.ServiceLaunchException;
+
+/**
+ * Try to stop() in service start; in execute() raise an exception.
+ */
+public class StoppingInStartLaunchableService
+    extends AbstractLaunchableService {
+
+  public static final String NAME =
+      "org.apache.hadoop.service.launcher.testservices.StoppingInStartLaunchableService";
+  public StoppingInStartLaunchableService(String name) {
+    super(name);
+  }
+
+  @Override
+  protected void serviceStart() throws Exception {
+    super.serviceStart();
+    stop();
+    }
+
+  @Override
+  public int execute() throws Exception {
+    throw new ServiceLaunchException(
+        LauncherExitCodes.EXIT_SERVICE_LIFECYCLE_EXCEPTION,
+        "Should not have been executed");
+  }
+}

+ 39 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/service/launcher/testservices/StringConstructorOnlyService.java

@@ -0,0 +1,39 @@
+/*
+ * 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.testservices;
+
+import org.apache.hadoop.service.launcher.AbstractLaunchableService;
+
+/**
+ * Service that only has one constructor that takes a string.
+ * This is the standard base class of a YARN service, so handle it
+ * in the launch
+ */
+public class StringConstructorOnlyService extends AbstractLaunchableService {
+  
+
+  public StringConstructorOnlyService(String name) {
+    super(name);
+  }
+
+  public static final String NAME =
+      "org.apache.hadoop.service.launcher.testservices.StringConstructorOnlyService";
+  
+
+}

+ 34 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/GenericTestUtils.java

@@ -31,6 +31,7 @@ import java.lang.management.ThreadInfo;
 import java.lang.management.ThreadMXBean;
 import java.lang.management.ThreadMXBean;
 import java.lang.reflect.InvocationTargetException;
 import java.lang.reflect.InvocationTargetException;
 import java.util.Arrays;
 import java.util.Arrays;
+import java.util.Locale;
 import java.util.Random;
 import java.util.Random;
 import java.util.Set;
 import java.util.Set;
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.CountDownLatch;
@@ -692,4 +693,37 @@ public abstract class GenericTestUtils {
       bld.append(" + ").append(l).append("\n");
       bld.append(" + ").append(l).append("\n");
     }
     }
   }
   }
+
+  /**
+   * Formatted fail, via {@link String#format(String, Object...)}.
+   * @param format format string
+   * @param args argument list. If the last argument is a throwable, it
+   * is used as the inner cause of the exception
+   * @throws AssertionError with the formatted message
+   */
+  public static void failf(String format, Object... args) {
+    String message = String.format(Locale.ENGLISH, format, args);
+    AssertionError error = new AssertionError(message);
+    int len = args.length;
+    if (len > 0 && args[len - 1] instanceof Throwable) {
+      error.initCause((Throwable) args[len - 1]);
+    }
+    throw error;
+  }
+
+  /**
+   * Conditional formatted fail, via {@link String#format(String, Object...)}.
+   * @param condition condition: if true the method fails
+   * @param format format string
+   * @param args argument list. If the last argument is a throwable, it
+   * is used as the inner cause of the exception
+   * @throws AssertionError with the formatted message
+   */
+  public static void failif(boolean condition,
+      String format,
+      Object... args) {
+    if (condition) {
+      failf(format, args);
+    }
+  }
 }
 }