You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@slider.apache.org by st...@apache.org on 2015/12/16 20:04:38 UTC

[2/5] incubator-slider git commit: SLIDER-1027 add a kdiag command for kerberos diagnostics

SLIDER-1027 add a kdiag command for kerberos diagnostics


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

Branch: refs/heads/develop
Commit: a36b25d9c46346216c77ee204bae4cb19c060765
Parents: af6bd40
Author: Steve Loughran <st...@apache.org>
Authored: Wed Dec 16 15:07:51 2015 +0000
Committer: Steve Loughran <st...@apache.org>
Committed: Wed Dec 16 15:07:51 2015 +0000

----------------------------------------------------------------------
 .../apache/hadoop/security/KerberosDiags.java   | 303 +++++++++++++++++++
 .../org/apache/slider/client/SliderClient.java  |  62 +++-
 .../common/params/AbstractActionArgs.java       |  10 +
 .../slider/common/params/ActionKDiagArgs.java   |  31 ++
 .../apache/slider/common/params/Arguments.java  |  14 +-
 .../apache/slider/common/params/ClientArgs.java | 211 +++++++------
 .../apache/slider/common/tools/SliderUtils.java |   2 -
 7 files changed, 525 insertions(+), 108 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/a36b25d9/slider-core/src/main/java/org/apache/hadoop/security/KerberosDiags.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/hadoop/security/KerberosDiags.java b/slider-core/src/main/java/org/apache/hadoop/security/KerberosDiags.java
new file mode 100644
index 0000000..3d12857
--- /dev/null
+++ b/slider-core/src/main/java/org/apache/hadoop/security/KerberosDiags.java
@@ -0,0 +1,303 @@
+/*
+ * 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.security;
+
+import org.apache.commons.io.IOUtils;
+import org.apache.commons.lang.StringUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.security.token.TokenIdentifier;
+import org.apache.hadoop.util.ExitUtil;
+import org.apache.hadoop.util.Shell;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Closeable;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.util.Collection;
+import java.util.Date;
+import java.util.List;
+
+import static org.apache.hadoop.security.UserGroupInformation.*;
+import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.*;
+
+/**
+ * Kerberos diagnostics
+ * At some point this may move to hadoop core, so please keep use of slider
+ * methods and classes to ~0.
+ *
+ * This operation expands some of the diagnostic output of the security code,
+ * but not all. For completeness
+ *
+ * Set the environment variable {@code HADOOP_JAAS_DEBUG=true}
+ * Set the log level for {@code org.apache.hadoop.security=DEBUG}
+ */
+public class KerberosDiags implements Closeable {
+
+  private static final Logger LOG= LoggerFactory.getLogger(KerberosDiags.class);
+  public static final String KRB5_CCNAME = "KRB5CCNAME";
+  public static final String JAVA_SECURITY_KRB5_CONF
+    = "java.security.krb5.conf";
+  public static final String JAVA_SECURITY_KRB5_REALM
+    = "java.security.krb5.realm";
+  public static final String SUN_SECURITY_KRB5_DEBUG
+    = "sun.security.krb5.debug";
+  public static final String SUN_SECURITY_SPNEGO_DEBUG
+    = "sun.security.spnego.debug";
+
+  private final Configuration conf;
+  private final List<String> services;
+  private final PrintWriter out;
+  private final File keytab;
+  private final String principal;
+
+  @SuppressWarnings("IOResourceOpenedButNotSafelyClosed")
+  public KerberosDiags(Configuration conf,
+    PrintWriter out,
+    List<String> services,
+    File keytab, String principal) {
+    this.conf = conf;
+    this.services = services;
+    this.keytab = keytab;
+    this.principal = principal;
+    this.out = out;
+  }
+
+  @Override
+  public void close() throws IOException {
+    if (out != null) {
+      out.flush();
+    }
+  }
+
+  private void println(String format, Object... args) {
+    String msg = String.format(format, args);
+    if (out != null) {
+      out.println(msg);
+      out.flush();
+    } else {
+      LOG.info(msg);
+    }
+  }
+
+  private void title(String format, Object... args) {
+    println("");
+    println(format, args);
+    println("");
+  }
+
+  private void printSysprop(String key) {
+    println("%s = \"%s\"", key, System.getProperty(key, "(unset)"));
+  }
+
+  private void printConfOpt(String key) {
+    println("%s = \"%s\"", key, conf.get(key, "(unset)"));
+  }
+
+  private void printEnv(String key) {
+    String env = System.getenv(key);
+    println("%s = \"%s\"", key, env != null ? env : "(unset)");
+  }
+
+  private void dump(File file) throws IOException {
+    try(FileInputStream in = new FileInputStream(file)) {
+      for (String line: IOUtils.readLines(in)) {
+        println(line);
+      }
+    }
+    println("");
+  }
+
+  /**
+   * Execute diagnostics.
+   * <p>
+   * Things it would be nice if UGI made accessible
+   * <ol>
+   *   <li>A way to enable JAAS debug programatically</li>
+   *   <li>Acess to the TGT</li>
+   * </ol>
+   * @throws Exception
+   */
+  public void execute() throws Exception {
+    title("Kerberos Diagnostics scan at %s",
+      new Date(System.currentTimeMillis()));
+    boolean securityDisabled = SecurityUtil.getAuthenticationMethod(conf)
+      .equals(UserGroupInformation.AuthenticationMethod.SIMPLE);
+    failif(securityDisabled, "security disabled");
+    title("System Properties");
+    for (String prop : new String[]{
+      JAVA_SECURITY_KRB5_CONF,
+      JAVA_SECURITY_KRB5_REALM,
+      SUN_SECURITY_KRB5_DEBUG,
+      SUN_SECURITY_SPNEGO_DEBUG,
+    }) {
+      printSysprop(prop);
+    }
+
+    title("Environment Variables");
+    for (String env : new String[]{
+      "HADOOP_JAAS_DEBUG",
+      KRB5_CCNAME,
+      HADOOP_USER_NAME,
+      HADOOP_PROXY_USER,
+      HADOOP_TOKEN_FILE_LOCATION,
+    }) {
+      printEnv(env);
+    }
+    for (String prop : new String[]{
+      "hadoop.kerberos.kinit.command",
+      HADOOP_SECURITY_AUTHENTICATION,
+      HADOOP_SECURITY_AUTHORIZATION,
+      "hadoop.security.dns.interface",   // not in 2.6
+      "hadoop.security.dns.nameserver",  // not in 2.6
+      HADOOP_SSL_ENABLED_KEY,
+      HADOOP_RPC_PROTECTION,
+      HADOOP_SECURITY_SASL_PROPS_RESOLVER_CLASS,
+      HADOOP_SECURITY_CRYPTO_CODEC_CLASSES_KEY_PREFIX,
+      HADOOP_SECURITY_GROUP_MAPPING,
+    }) {
+      printConfOpt(prop);
+    }
+
+    System.setProperty("sun.security.krb5.debug", "true");
+    System.setProperty("sun.security.spnego.debug", "true");
+
+    title("Logging in");
+    dumpUser("Log in user", getLoginUser());
+
+    // locate KDC and dump it
+    if (!Shell.WINDOWS) {
+      title("Locating Kerberos configuration file");
+      String krbPath = "/etc/krb5.conf";
+      String jvmKrbPath = System.getProperty(JAVA_SECURITY_KRB5_CONF);
+      if (jvmKrbPath != null) {
+        println("Setting kerberos path from sysprop %s: %s",
+          JAVA_SECURITY_KRB5_CONF, jvmKrbPath);
+        krbPath = jvmKrbPath;
+      }
+
+      String krb5name = System.getenv(KRB5_CCNAME);
+      if (krb5name != null) {
+        println("Setting kerberos path from environment variable %s: %s",
+          KRB5_CCNAME, krb5name);
+        krbPath = krb5name;
+        if (jvmKrbPath != null) {
+          println("Warning - both %s and %s were set - %s takes priority",
+            JAVA_SECURITY_KRB5_CONF, KRB5_CCNAME, KRB5_CCNAME);
+        }
+      }
+
+      File krbFile = new File(krbPath);
+      println("Kerberos configuration file = %s", krbFile);
+      failif(!krbFile.exists(),
+        "Kerberos configuration file %s not found", krbFile);
+      dump(krbFile);
+    }
+
+    UserGroupInformation ugi;
+    String identity;
+    if (keytab != null) {
+      File kt = keytab.getCanonicalFile();
+      println("Using keytab %s principal %s", kt, principal);
+      identity = principal;
+      failif(!kt.exists(), "Keytab not found: %s", kt);
+      failif(!kt.isFile(), "Keytab is not a valid file: %s", kt);
+      failif(StringUtils.isEmpty(principal), "No principal defined");
+      ugi = loginUserFromKeytabAndReturnUGI(principal, kt.getPath());
+      dumpUser(identity, ugi);
+      title("Attempting to log in from keytab again");
+      // package scoped -hence the reason why this class must be in the
+      // hadoop.security package
+      setShouldRenewImmediatelyForTests(true);
+      // attempt a new login
+      ugi.reloginFromKeytab();
+//      dumpUser("Updated User", ugi);
+    } else {
+      println("No keytab: logging is as current user");
+      ugi = getLoginUser();
+      identity = "Login User";
+    }
+  }
+
+  private void dumpUser(String message, UserGroupInformation ugi) {
+    title(message);
+    println("UGI=%s", ugi);
+    title("Group names");
+    for (String name : ugi.getGroupNames()) {
+      println(name);
+    }
+    title("Credentials");
+    Credentials credentials = ugi.getCredentials();
+    List<Text> secretKeys = credentials.getAllSecretKeys();
+    title("Secret keys");
+    if (!secretKeys.isEmpty()) {
+      for (Text secret: secretKeys) {
+        println("%s", secret);
+      }
+    } else {
+      println("(none)");
+    }
+
+    title("Tokens");
+    Collection<Token<? extends TokenIdentifier>> tokens
+      = credentials.getAllTokens();
+    if (!tokens.isEmpty()) {
+      for (Token<? extends TokenIdentifier> token : tokens) {
+        println("%s", token);
+      }
+    } else {
+      println("(none)");
+    }
+  }
+
+  private void fail(String message, Object... args)
+    throws KerberosDiagsFailure {
+    throw new KerberosDiagsFailure(message, args);
+  }
+
+  private void failif(boolean condition, String message, Object... args)
+    throws KerberosDiagsFailure {
+    if (condition) {
+      throw new KerberosDiagsFailure(message, args);
+    }
+  }
+
+  /**
+   * Diags failures include an exit code 41, "unauth"
+   */
+  public static class KerberosDiagsFailure extends ExitUtil.ExitException {
+    public KerberosDiagsFailure( String message) {
+      super(41, message);
+    }
+
+    public KerberosDiagsFailure(String message, Object... args) {
+      this(String.format(message, args));
+    }
+
+    public KerberosDiagsFailure(Throwable throwable,
+      String message, Object... args) {
+      this(message, args);
+      initCause(throwable);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/a36b25d9/slider-core/src/main/java/org/apache/slider/client/SliderClient.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/client/SliderClient.java b/slider-core/src/main/java/org/apache/slider/client/SliderClient.java
index da94dd4..28727cb 100644
--- a/slider-core/src/main/java/org/apache/slider/client/SliderClient.java
+++ b/slider-core/src/main/java/org/apache/slider/client/SliderClient.java
@@ -90,6 +90,7 @@ import org.apache.slider.common.params.ActionFlexArgs;
 import org.apache.slider.common.params.ActionFreezeArgs;
 import org.apache.slider.common.params.ActionInstallKeytabArgs;
 import org.apache.slider.common.params.ActionInstallPackageArgs;
+import org.apache.slider.common.params.ActionKDiagArgs;
 import org.apache.slider.common.params.ActionKeytabArgs;
 import org.apache.slider.common.params.ActionKillContainerArgs;
 import org.apache.slider.common.params.ActionListArgs;
@@ -107,6 +108,7 @@ import org.apache.slider.common.params.CommonArgs;
 import org.apache.slider.common.params.LaunchArgsAccessor;
 import org.apache.slider.common.tools.ConfigHelper;
 import org.apache.slider.common.tools.Duration;
+import org.apache.hadoop.security.KerberosDiags;
 import org.apache.slider.common.tools.SliderFileSystem;
 import org.apache.slider.common.tools.SliderUtils;
 import org.apache.slider.common.tools.SliderVersionInfo;
@@ -178,6 +180,7 @@ import java.io.IOException;
 import java.io.InputStreamReader;
 import java.io.InterruptedIOException;
 import java.io.PrintStream;
+import java.io.PrintWriter;
 import java.io.StringWriter;
 import java.io.Writer;
 import java.net.InetAddress;
@@ -299,12 +302,12 @@ public class SliderClient extends AbstractSliderLaunchedService implements RunSe
     ConfigHelper.mergeConfigurations(conf, clientConf, SLIDER_CLIENT_XML, true);
     serviceArgs.applyDefinitions(conf);
     serviceArgs.applyFileSystemBinding(conf);
+    AbstractActionArgs coreAction = serviceArgs.getCoreAction();
     // init security with our conf
-    if (isHadoopClusterSecure(conf)) {
+    if (!coreAction.disableSecureLogin() && isHadoopClusterSecure(conf)) {
       forceLogin();
       initProcessSecurity(conf);
     }
-    AbstractActionArgs coreAction = serviceArgs.getCoreAction();
     if (coreAction.getHadoopServicesRequired()) {
       initHadoopBinding();
     }
@@ -389,12 +392,16 @@ public class SliderClient extends AbstractSliderLaunchedService implements RunSe
       case ACTION_HELP:
         log.info(serviceArgs.usage());
         break;
-      
+
+      case ACTION_KDIAG:
+        exitCode = actionKDiag(serviceArgs.getActionKDiagArgs());
+        break;
+
       case ACTION_KILL_CONTAINER:
         exitCode = actionKillContainer(clusterName,
             serviceArgs.getActionKillContainerArgs());
         break;
-      
+
       case ACTION_INSTALL_KEYTAB:
         exitCode = actionInstallKeytab(serviceArgs.getActionInstallKeytabArgs());
         break;
@@ -3393,12 +3400,9 @@ public class SliderClient extends AbstractSliderLaunchedService implements RunSe
           println(serviceRecordMarshal.toJson(instance));
         }
       }
-//      TODO JDK7
-    } catch (PathNotFoundException e) {
+    } catch (PathNotFoundException | NoRecordException e) {
       // no record at this path
       throw new NotFoundException(e, path);
-    } catch (NoRecordException e) {
-      throw new NotFoundException(e, path);
     }
     return EXIT_SUCCESS;
   }
@@ -3772,6 +3776,48 @@ public class SliderClient extends AbstractSliderLaunchedService implements RunSe
   }
 
   /**
+   * Kerberos Diagnostics
+   * @param args CLI arguments
+   * @return exit code
+   * @throws SliderException
+   * @throws IOException
+   */
+  private int actionKDiag(ActionKDiagArgs args)
+    throws Exception {
+    PrintWriter out = new PrintWriter(System.err);
+    boolean closeStream = false;
+    if (args.out != null) {
+      out = new PrintWriter(new FileOutputStream(args.out));
+      closeStream = true;
+    }
+    try {
+      KerberosDiags kdiags = new KerberosDiags(getConfig(),
+        out,
+        args.services,
+        args.keytab,
+        args.principal);
+      kdiags.execute();
+    } catch (KerberosDiags.KerberosDiagsFailure e) {
+      log.error(e.toString());
+      log.debug(e.toString(), e);
+      if (args.fail) {
+        throw e;
+      }
+    } catch (Exception e) {
+      log.error("Kerberos Diagnostics", e);
+      if (args.fail) {
+        throw e;
+      }
+    } finally {
+      if (closeStream) {
+        out.flush();
+        out.close();
+      }
+    }
+    return 0;
+  }
+
+  /**
    * Log a service record instance
    * @param instance record
    * @param verbose verbose logging of all external endpoints

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/a36b25d9/slider-core/src/main/java/org/apache/slider/common/params/AbstractActionArgs.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/common/params/AbstractActionArgs.java b/slider-core/src/main/java/org/apache/slider/common/params/AbstractActionArgs.java
index d0b1693..e3cb288 100644
--- a/slider-core/src/main/java/org/apache/slider/common/params/AbstractActionArgs.java
+++ b/slider-core/src/main/java/org/apache/slider/common/params/AbstractActionArgs.java
@@ -165,4 +165,14 @@ public abstract class AbstractActionArgs extends ArgOps implements Arguments {
   public boolean getHadoopServicesRequired() {
     return true;
   }
+
+  /**
+   * Flag to disable secure login.
+   * This MUST only be set if the action is bypassing security or setting
+   * it itself
+   * @return true if login at slider client init time is to be skipped
+   */
+  public boolean disableSecureLogin() {
+    return false;
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/a36b25d9/slider-core/src/main/java/org/apache/slider/common/params/ActionKDiagArgs.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/common/params/ActionKDiagArgs.java b/slider-core/src/main/java/org/apache/slider/common/params/ActionKDiagArgs.java
index 912b300..874eac1 100644
--- a/slider-core/src/main/java/org/apache/slider/common/params/ActionKDiagArgs.java
+++ b/slider-core/src/main/java/org/apache/slider/common/params/ActionKDiagArgs.java
@@ -20,6 +20,9 @@ package org.apache.slider.common.params;
 
 import com.beust.jcommander.Parameter;
 import com.beust.jcommander.Parameters;
+import org.apache.slider.common.tools.SliderUtils;
+import org.apache.slider.core.exceptions.BadCommandArgumentsException;
+import org.apache.slider.core.exceptions.UsageException;
 
 import java.io.File;
 import java.util.ArrayList;
@@ -50,4 +53,32 @@ public class ActionKDiagArgs extends AbstractActionArgs {
   @Parameter(names = {ARG_KEYTAB}, description = "keytab to use")
   public File keytab;
 
+  @Parameter(names = {ARG_PRINCIPAL}, description = "principal to log in from a keytab")
+  public String principal;
+
+  @Override
+  public int getMinParams() {
+    return 0;
+  }
+
+  @Override
+  public boolean getHadoopServicesRequired() {
+    return false;
+  }
+
+  @Override
+  public boolean disableSecureLogin() {
+    return true;
+  }
+
+  @Override
+  public void validate() throws BadCommandArgumentsException, UsageException {
+    super.validate();
+    if (keytab != null && SliderUtils.isUnset(principal)) {
+      throw new UsageException("Missing argument " + ARG_PRINCIPAL);
+    }
+    if (keytab == null && SliderUtils.isSet(principal)) {
+      throw new UsageException("Missing argument " + ARG_KEYTAB);
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/a36b25d9/slider-core/src/main/java/org/apache/slider/common/params/Arguments.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/common/params/Arguments.java b/slider-core/src/main/java/org/apache/slider/common/params/Arguments.java
index 34ff8bf..62e5e0d 100644
--- a/slider-core/src/main/java/org/apache/slider/common/params/Arguments.java
+++ b/slider-core/src/main/java/org/apache/slider/common/params/Arguments.java
@@ -41,9 +41,11 @@ public interface Arguments {
   String ARG_COMP_OPT= "--compopt";
   String ARG_COMP_OPT_SHORT = "--co";
   String ARG_CONFIG = "--config";
+  String ARG_CONTAINERS = "--containers";
   String ARG_CREDENTIALS = "--credentials";
   String ARG_DEBUG = "--debug";
   String ARG_DEFINE = "-D";
+  String ARG_DELETE = "--delete";
   String ARG_DEST = "--dest";
   String ARG_DESTDIR = "--destdir";
   String ARG_DESTFILE = "--destfile";
@@ -66,11 +68,12 @@ public interface Arguments {
   String ARG_HOSTNAME = "--hostname";
   String ARG_ID = "--id";
   String ARG_IMAGE = "--image";
+  String ARG_INSTALL = "--install";
   String ARG_INTERNAL = "--internal";
   String ARG_KEYTAB = "--keytab";
   String ARG_KEYSTORE = "--keystore";
-  String ARG_KEYTABINSTALL = "--install";
-  String ARG_KEYTABDELETE = "--delete";
+  String ARG_KEYTABINSTALL = ARG_INSTALL;
+  String ARG_KEYTABDELETE = ARG_DELETE;
   String ARG_KEYTABLIST = "--list";
   String ARG_LABEL = "--label";
   String ARG_LEVEL = "--level";
@@ -79,7 +82,6 @@ public interface Arguments {
   String ARG_LISTEXP = "--listexp";
   String ARG_LISTFILES = "--listfiles";
   String ARG_LIVE = "--live";
-  String ARG_CONTAINERS = "--containers";
   String ARG_MANAGER = "--manager";
   String ARG_MANAGER_SHORT = "--m";
   String ARG_MESSAGE = "--message";
@@ -93,10 +95,10 @@ public interface Arguments {
   String ARG_PACKAGE = "--package";
   String ARG_PASSWORD = "--password";
   String ARG_PATH = "--path";
-  String ARG_PKGDELETE = "--delete";
-  String ARG_INSTALL = "--install";
+  String ARG_PKGDELETE = ARG_DELETE;
   String ARG_PKGINSTANCES = "--instances";
-  String ARG_PKGLIST = "--list";
+  String ARG_PKGLIST = ARG_LIST;
+  String ARG_PRINCIPAL = "--principal";
   String ARG_PROVIDER = "--provider";
   String ARG_QUEUE = "--queue";
   String ARG_REPLACE_PKG = "--replacepkg";

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/a36b25d9/slider-core/src/main/java/org/apache/slider/common/params/ClientArgs.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/common/params/ClientArgs.java b/slider-core/src/main/java/org/apache/slider/common/params/ClientArgs.java
index 3a9afef..e5e6888 100644
--- a/slider-core/src/main/java/org/apache/slider/common/params/ClientArgs.java
+++ b/slider-core/src/main/java/org/apache/slider/common/params/ClientArgs.java
@@ -230,104 +230,131 @@ public class ClientArgs extends CommonArgs {
 
   /**
    * Look at the chosen action and bind it as the core action for the operation.
-   * In theory this could be done by introspecting on the list of actions and 
-   * choosing it without the switch statement. In practise this switch, while
-   * verbose, is easier to debug. And in JDK7, much simpler.
    * @throws SliderException bad argument or similar
    */
   @Override
   public void applyAction() throws SliderException {
     String action = getAction();
-    if (SliderActions.ACTION_BUILD.equals(action)) {
-      bindCoreAction(actionBuildArgs);
-      //its a builder, so set those actions too
-      buildingActionArgs = actionBuildArgs;
-    } else if (SliderActions.ACTION_CREATE.equals(action)) {
-      bindCoreAction(actionCreateArgs);
-      //its a builder, so set those actions too
-      buildingActionArgs = actionCreateArgs;
-
-    } else if (SliderActions.ACTION_FREEZE.equals(action)) {
-      bindCoreAction(actionFreezeArgs);
-
-    } else if (SliderActions.ACTION_THAW.equals(action)) {
-      bindCoreAction(actionThawArgs);
-
-    } else if (SliderActions.ACTION_AM_SUICIDE.equals(action)) {
-      bindCoreAction(actionAMSuicideArgs);
-
-    } else if (SliderActions.ACTION_DEPENDENCY.equals(action)) {
-      bindCoreAction(actionDependencyArgs);
-
-    } else if (SliderActions.ACTION_DESTROY.equals(action)) {
-      bindCoreAction(actionDestroyArgs);
-
-    } else if (SliderActions.ACTION_DIAGNOSTICS.equals(action)) {
-      bindCoreAction(actionDiagnosticArgs);
-
-    } else if (SliderActions.ACTION_EXISTS.equals(action)) {
-      bindCoreAction(actionExistsArgs);
-
-    } else if (SliderActions.ACTION_FLEX.equals(action)) {
-      bindCoreAction(actionFlexArgs);
-
-    } else if (SliderActions.ACTION_HELP.equals(action)) {
-      bindCoreAction(actionHelpArgs);
-
-    } else if (SliderActions.ACTION_INSTALL_PACKAGE.equals(action)) {
-      bindCoreAction(actionInstallPackageArgs);
-
-    } else if (SliderActions.ACTION_KDIAG.equals(action)) {
-      bindCoreAction(actionKDiagArgs);
-
-    } else if (SliderActions.ACTION_KEYTAB.equals(action)) {
-      bindCoreAction(actionKeytabArgs);
-
-    } else if (SliderActions.ACTION_PACKAGE.equals(action)) {
-      bindCoreAction(actionPackageArgs);
-
-    } else if (SliderActions.ACTION_CLIENT.equals(action)) {
-      bindCoreAction(actionClientArgs);
-
-    } else if (SliderActions.ACTION_INSTALL_KEYTAB.equals(action)) {
-      bindCoreAction(actionInstallKeytabArgs);
-
-    } else if (SliderActions.ACTION_KILL_CONTAINER.equals(action)) {
-      bindCoreAction(actionKillContainerArgs);
-
-    } else if (SliderActions.ACTION_LIST.equals(action)) {
-      bindCoreAction(actionListArgs);
-
-    } else if (SliderActions.ACTION_LOOKUP.equals(action)) {
-      bindCoreAction(actionLookupArgs);
-
-    } else if (SliderActions.ACTION_NODES.equals(action)) {
-      bindCoreAction(actionNodesArgs);
-
-    } else if (SliderActions.ACTION_REGISTRY.equals(action)) {
-      bindCoreAction(actionRegistryArgs);
-
-    } else if (SliderActions.ACTION_RESOLVE.equals(action)) {
-      bindCoreAction(actionResolveArgs);
-
-    } else if (SliderActions.ACTION_STATUS.equals(action)) {
-      bindCoreAction(actionStatusArgs);
-
-    } else if (SliderActions.ACTION_UPDATE.equals(action)) {
-      bindCoreAction(actionUpdateArgs);
-
-    } else if (SliderActions.ACTION_UPGRADE.equals(action)) {
-      bindCoreAction(actionUpgradeArgs);
-
-    } else if (SliderActions.ACTION_VERSION.equals(action)) {
-      bindCoreAction(actionVersionArgs);
-
-    } else if (SliderUtils.isUnset(action)) {
-      bindCoreAction(actionHelpArgs);
+    if (SliderUtils.isUnset(action)) {
+      action = ACTION_HELP;
+    }
+    switch (action) {
+      case ACTION_BUILD:
+        bindCoreAction(actionBuildArgs);
+        //its a builder, so set those actions too
+        buildingActionArgs = actionBuildArgs;
+        break;
+
+      case ACTION_CREATE:
+        bindCoreAction(actionCreateArgs);
+        //its a builder, so set those actions too
+        buildingActionArgs = actionCreateArgs;
+        break;
+
+      case ACTION_FREEZE:
+        bindCoreAction(actionFreezeArgs);
+        break;
+
+      case ACTION_THAW:
+        bindCoreAction(actionThawArgs);
+        break;
+
+      case ACTION_AM_SUICIDE:
+        bindCoreAction(actionAMSuicideArgs);
+        break;
+
+      case ACTION_CLIENT:
+        bindCoreAction(actionClientArgs);
+        break;
+
+      case ACTION_DEPENDENCY:
+        bindCoreAction(actionDependencyArgs);
+        break;
+
+      case ACTION_DESTROY:
+        bindCoreAction(actionDestroyArgs);
+        break;
+
+      case ACTION_DIAGNOSTICS:
+        bindCoreAction(actionDiagnosticArgs);
+        break;
+
+      case ACTION_EXISTS:
+        bindCoreAction(actionExistsArgs);
+        break;
+
+      case ACTION_FLEX:
+        bindCoreAction(actionFlexArgs);
+        break;
+
+      case ACTION_HELP:
+        bindCoreAction(actionHelpArgs);
+        break;
+
+      case ACTION_INSTALL_KEYTAB:
+        bindCoreAction(actionInstallKeytabArgs);
+        break;
+
+      case ACTION_INSTALL_PACKAGE:
+        bindCoreAction(actionInstallPackageArgs);
+        break;
+
+      case ACTION_KDIAG:
+        bindCoreAction(actionKDiagArgs);
+        break;
+
+      case ACTION_KEYTAB:
+        bindCoreAction(actionKeytabArgs);
+        break;
+
+      case ACTION_KILL_CONTAINER:
+        bindCoreAction(actionKillContainerArgs);
+        break;
+
+      case ACTION_LIST:
+        bindCoreAction(actionListArgs);
+        break;
+
+      case ACTION_LOOKUP:
+        bindCoreAction(actionLookupArgs);
+        break;
+
+      case ACTION_NODES:
+        bindCoreAction(actionNodesArgs);
+        break;
+
+      case ACTION_PACKAGE:
+        bindCoreAction(actionPackageArgs);
+        break;
+
+      case ACTION_REGISTRY:
+        bindCoreAction(actionRegistryArgs);
+        break;
+
+      case ACTION_RESOLVE:
+        bindCoreAction(actionResolveArgs);
+        break;
+
+      case ACTION_STATUS:
+        bindCoreAction(actionStatusArgs);
+        break;
+
+      case ACTION_UPDATE:
+        bindCoreAction(actionUpdateArgs);
+        break;
+
+      case ACTION_UPGRADE:
+        bindCoreAction(actionUpgradeArgs);
+        break;
+
+      case ACTION_VERSION:
+        bindCoreAction(actionVersionArgs);
+        break;
 
-    } else {
-      throw new BadCommandArgumentsException(ErrorStrings.ERROR_UNKNOWN_ACTION
-                                             + " " + action);
+      default:
+        throw new BadCommandArgumentsException(ErrorStrings.ERROR_UNKNOWN_ACTION
+        + " " + action);
     }
   }
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/a36b25d9/slider-core/src/main/java/org/apache/slider/common/tools/SliderUtils.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/common/tools/SliderUtils.java b/slider-core/src/main/java/org/apache/slider/common/tools/SliderUtils.java
index a456198..8d74692 100644
--- a/slider-core/src/main/java/org/apache/slider/common/tools/SliderUtils.java
+++ b/slider-core/src/main/java/org/apache/slider/common/tools/SliderUtils.java
@@ -1261,8 +1261,6 @@ public final class SliderUtils {
         conf.get(CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHENTICATION));
     log.debug("hadoop.security.authorization={}",
         conf.get(CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHORIZATION));
-/*    SecurityUtil.setAuthenticationMethod(
-        UserGroupInformation.AuthenticationMethod.KERBEROS, conf);*/
     UserGroupInformation.setConfiguration(conf);
     UserGroupInformation authUser = UserGroupInformation.getCurrentUser();
     log.debug("Authenticating as {}", authUser);