You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@slider.apache.org by go...@apache.org on 2016/03/16 21:57:57 UTC

[06/50] incubator-slider git commit: SLIDER-1035 extend kdiag and add category to exceptions. Goal is to target hadoop

SLIDER-1035 extend kdiag and add category to exceptions. Goal is to target hadoop


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

Branch: refs/heads/feature/SLIDER-906_docker_support
Commit: 4fa7c18d3b541dc468026afb2f3fe6fb84878848
Parents: be14bdd
Author: Steve Loughran <st...@apache.org>
Authored: Sun Jan 17 18:45:39 2016 -0800
Committer: Steve Loughran <st...@apache.org>
Committed: Sun Jan 17 18:45:39 2016 -0800

----------------------------------------------------------------------
 .../apache/hadoop/security/KerberosDiags.java   | 274 +++++++++++++++----
 .../org/apache/slider/client/SliderClient.java  |  14 +-
 .../slider/common/params/ActionKDiagArgs.java   |   6 +
 .../apache/slider/common/params/Arguments.java  |   2 +
 .../apache/slider/client/TestDiagnostics.groovy |   1 +
 5 files changed, 233 insertions(+), 64 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/4fa7c18d/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
index 2a8344d..d0fba8e 100644
--- a/slider-core/src/main/java/org/apache/hadoop/security/KerberosDiags.java
+++ b/slider-core/src/main/java/org/apache/hadoop/security/KerberosDiags.java
@@ -35,11 +35,16 @@ import java.io.File;
 import java.io.FileInputStream;
 import java.io.IOException;
 import java.io.PrintWriter;
+import java.lang.reflect.InvocationTargetException;
+import java.net.InetAddress;
+import java.security.NoSuchAlgorithmException;
 import java.util.Collection;
 import java.util.Date;
 import java.util.List;
+import java.util.regex.Pattern;
 
 import static org.apache.hadoop.security.UserGroupInformation.*;
+import static org.apache.hadoop.security.authentication.util.KerberosUtil.*;
 import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.*;
 
 /**
@@ -69,23 +74,40 @@ public class KerberosDiags implements Closeable {
     = "java.security.auth.login.config";
   public static final String KERBEROS_KINIT_COMMAND
     = "hadoop.kerberos.kinit.command";
+  public static final String HADOOP_AUTHENTICATION_IS_DISABLED
+      = "Hadoop authentication is disabled";
+  public static final String UNSET = "(unset)";
 
   private final Configuration conf;
   private final List<String> services;
   private final PrintWriter out;
   private final File keytab;
   private final String principal;
+  private final long minKeyLength;
+  private final boolean securityRequired;
+
+  public static final String CAT_JVM = "JVM";
+  public static final String CAT_JAAS = "JAAS";
+  public static final String CAT_CONFIG = "CONFIG";
+  public static final String CAT_LOGIN = "LOGIN";
+  public static final String CAT_KERBEROS = "KERBEROS";
+  public static final String CAT_SASL = "SASL";
 
   @SuppressWarnings("IOResourceOpenedButNotSafelyClosed")
   public KerberosDiags(Configuration conf,
-    PrintWriter out,
-    List<String> services,
-    File keytab, String principal) {
+      PrintWriter out,
+      List<String> services,
+      File keytab,
+      String principal,
+      long minKeyLength,
+      boolean securityRequired) {
     this.conf = conf;
     this.services = services;
     this.keytab = keytab;
     this.principal = principal;
     this.out = out;
+    this.minKeyLength = minKeyLength;
+    this.securityRequired = securityRequired;
   }
 
   @Override
@@ -107,19 +129,17 @@ public class KerberosDiags implements Closeable {
    */
   @SuppressWarnings("deprecation")
   public boolean execute() throws Exception {
+
     title("Kerberos Diagnostics scan at %s",
-      new Date(System.currentTimeMillis()));
-    int aesLen = Cipher.getMaxAllowedKeyLength("AES");
-    println("Maximum AES encryption key length %d", aesLen);
-    failif (aesLen < 256,
-      "Java Cryptography Extensions are not installed on this JVM."
-        +" Kerberos will not work.");
-    boolean securityDisabled = SecurityUtil.getAuthenticationMethod(conf)
-      .equals(UserGroupInformation.AuthenticationMethod.SIMPLE);
-    if (securityDisabled) {
-      println("security disabled");
-      return false;
-    }
+        new Date(System.currentTimeMillis()));
+
+    // check that the machine has a name
+    println("Hostname: %s",
+        InetAddress.getLocalHost().getCanonicalHostName());
+
+    // Fail fast on a JVM without JCE installed.
+    validateKeyLength();
+
     title("System Properties");
     for (String prop : new String[]{
       JAVA_SECURITY_KRB5_CONF,
@@ -141,6 +161,7 @@ public class KerberosDiags implements Closeable {
     }) {
       printEnv(env);
     }
+
     for (String prop : new String[]{
       KERBEROS_KINIT_COMMAND,
       HADOOP_SECURITY_AUTHENTICATION,
@@ -154,16 +175,25 @@ public class KerberosDiags implements Closeable {
       HADOOP_SECURITY_GROUP_MAPPING,
       "hadoop.security.impersonation.provider.class",    // not in 2.6
       "dfs.data.transfer.protection" // HDFS
-
     }) {
       printConfOpt(prop);
     }
+
+    // check that authentication is enabled
+    if (SecurityUtil.getAuthenticationMethod(conf)
+        .equals(AuthenticationMethod.SIMPLE)) {
+      println(HADOOP_AUTHENTICATION_IS_DISABLED);
+      failif(securityRequired, CAT_CONFIG, HADOOP_AUTHENTICATION_IS_DISABLED);
+      // no security, skip rest of test
+      return false;
+    }
+
+    validateJVM();
     validateKrb5File();
     validateSasl(HADOOP_SECURITY_SASL_PROPS_RESOLVER_CLASS);
     validateSasl("dfs.data.transfer.saslproperties.resolver.class");
-    validateKinit();
+    validateKinitExecutable();
     validateJAAS();
-
     // now the big test: login, then try again
     boolean krb5Debug = getAndSet(SUN_SECURITY_KRB5_DEBUG);
     boolean spnegoDebug = getAndSet(SUN_SECURITY_SPNEGO_DEBUG);
@@ -187,6 +217,44 @@ public class KerberosDiags implements Closeable {
   }
 
   /**
+   * Fail fast on a JVM without JCE installed.
+   *
+   * This is a recurrent problem
+   * (that is: it keeps creeping back with JVM updates);
+   * a fast failure is the best tactic
+   * @throws NoSuchAlgorithmException
+   */
+
+  protected void validateKeyLength() throws NoSuchAlgorithmException {
+    int aesLen = Cipher.getMaxAllowedKeyLength("AES");
+    println("Maximum AES encryption key length %d", aesLen);
+    failif (aesLen < minKeyLength,
+        CAT_JVM,
+        "Java Cryptography Extensions are not installed on this JVM."
+        +" Maximum supported key length %s - minimum required %d",
+        aesLen, minKeyLength);
+  }
+
+  /**
+   *
+   */
+  protected void validateJVM() {
+    println("JVM Kerberos Login Module = %s",
+        getKrb5LoginModuleName());
+    try {
+      println("Default Realm = %s",
+          getDefaultRealm());
+    } catch (ClassNotFoundException
+        | InvocationTargetException
+        | IllegalAccessException
+        | NoSuchMethodException e) {
+
+      throw new KerberosDiagsFailure(CAT_JVM, e,
+          "Failed to get default realm");
+    }
+  }
+
+  /**
    * Locate the krb5.conf file and dump it. No-op on windows
    * @throws IOException
    */
@@ -215,11 +283,28 @@ public class KerberosDiags implements Closeable {
       File krbFile = new File(krbPath);
       println("Kerberos configuration file = %s", krbFile);
       failif(!krbFile.exists(),
-        "Kerberos configuration file %s not found", krbFile);
+          CAT_KERBEROS,
+          "Kerberos configuration file %s not found", krbFile);
       dump(krbFile);
     }
   }
 
+  /**
+   * Dump a keytab: list all principals
+   * @param keytabFile the keytab file
+   * @throws IOException IO problems
+   */
+  private void dumpKeytab(File keytabFile) throws IOException {
+    title("Examining keytab %s", keytabFile);
+    String[] names = getPrincipalNames(keytabFile.getCanonicalPath(),
+        Pattern.compile(".*"));
+    println("keytab entry count: %d", names.length);
+    for (String name : names) {
+      println(name);
+    }
+    println("-----");
+  }
+
   private void loginFromKeytab() throws IOException {
     UserGroupInformation ugi;
     String identity;
@@ -227,9 +312,12 @@ public class KerberosDiags implements Closeable {
       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");
+      failif(!kt.exists(), CAT_LOGIN, "Keytab not found: %s", kt);
+      failif(!kt.isFile(), CAT_LOGIN, "Keytab is not a valid file: %s", kt);
+      dumpKeytab(kt);
+
+      failif(StringUtils.isEmpty(principal), CAT_KERBEROS,
+          "No principal defined");
       ugi = loginUserFromKeytabAndReturnUGI(principal, kt.getPath());
       dumpUser(identity, ugi);
       validateUser(principal, ugi);
@@ -248,7 +336,7 @@ public class KerberosDiags implements Closeable {
   private void dumpUser(String message, UserGroupInformation ugi)
     throws IOException {
     title(message);
-    println("UGI=%s", ugi);
+    println("UGI instance = %s", ugi);
     println("Has kerberos credentials: %b", ugi.hasKerberosCredentials());
     println("Authentication method: %s", ugi.getAuthenticationMethod());
     println("Real Authentication method: %s",
@@ -274,21 +362,29 @@ public class KerberosDiags implements Closeable {
 
   private void validateUser(String message, UserGroupInformation user) {
     failif(!user.hasKerberosCredentials(),
-      "%s: No kerberos credentials for  %s", message, user);
+        CAT_LOGIN, "%s: No kerberos credentials for  %s", message, user);
     failif(user.getAuthenticationMethod() == null,
-      "%s: Null AuthenticationMethod for %s", message, user);
+        CAT_LOGIN, "%s: Null AuthenticationMethod for %s", message, user);
   }
 
-  private void validateKinit() {
+  /**
+   * A cursory look at the kinit executable: exists, size > 0.
+   */
+  private void validateKinitExecutable() {
     String kinit = conf.getTrimmed(KERBEROS_KINIT_COMMAND, "");
     if (!kinit.isEmpty()) {
       File kinitPath = new File(kinit);
       println("%s = %s", KERBEROS_KINIT_COMMAND, kinitPath);
       if (kinitPath.isAbsolute()) {
-        failif(!kinitPath.exists(), "%s executable does not exist: %s",
-          KERBEROS_KINIT_COMMAND, kinitPath);
-        failif(!kinitPath.isFile(), "%s path does not refer to a file: %s",
-          KERBEROS_KINIT_COMMAND, kinitPath);
+        failif(!kinitPath.exists(), CAT_KERBEROS,
+            "%s executable does not exist: %s",
+            KERBEROS_KINIT_COMMAND, kinitPath);
+        failif(!kinitPath.isFile(), CAT_KERBEROS,
+            "%s path does not refer to a file: %s",
+            KERBEROS_KINIT_COMMAND, kinitPath);
+        failif(kinitPath.length() == 0, CAT_KERBEROS,
+            "%s file is empty: %s",
+            KERBEROS_KINIT_COMMAND, kinitPath);
       } else {
         println("Executable %s is relative -must be on the PATH", kinit);
         printEnv("PATH");
@@ -296,32 +392,47 @@ public class KerberosDiags implements Closeable {
     }
   }
 
+  /**
+   * Try to load the SASL resolver.
+   * @param saslPropsResolverKey key for the SASL resolver
+   */
   private void validateSasl(String saslPropsResolverKey) {
     title("Resolving SASL property %s", saslPropsResolverKey);
     String saslPropsResolver = conf.getTrimmed(saslPropsResolverKey);
     try {
       Class<? extends SaslPropertiesResolver> resolverClass = conf.getClass(
-        saslPropsResolverKey,
-        SaslPropertiesResolver.class, SaslPropertiesResolver.class);
-      println("Resolver is %s", resolverClass.toString());
+          saslPropsResolverKey,
+          SaslPropertiesResolver.class, SaslPropertiesResolver.class);
+      println("Resolver is %s", resolverClass);
     } catch (RuntimeException e) {
-      throw new KerberosDiagsFailure(e, "Failed to load %s class %s",
-        saslPropsResolverKey, saslPropsResolver);
+      throw new KerberosDiagsFailure(CAT_SASL, e,
+          "Failed to load %s class %s",
+          saslPropsResolverKey, saslPropsResolver);
     }
   }
 
+  /**
+   * Validate any JAAS entry referenced in the {@link #SUN_SECURITY_JAAS_FILE}
+   * property.
+   */
   private void validateJAAS() {
     String jaasFilename = System.getProperty(SUN_SECURITY_JAAS_FILE);
-    if (jaasFilename!=null) {
+    if (jaasFilename != null) {
       title("JAAS");
       File jaasFile = new File(jaasFilename);
       println("JAAS file is defined in %s: %s",
           SUN_SECURITY_JAAS_FILE, jaasFile);
-      failif(!jaasFile.exists(), "JAAS file does not exist: %s", jaasFile);
-      failif(!jaasFile.isFile(), "JAAS file is not a file: %s", jaasFile);
+      failif(!jaasFile.exists(), CAT_JAAS,
+          "JAAS file does not exist: %s", jaasFile);
+      failif(!jaasFile.isFile(), CAT_JAAS,
+          "Specified JAAS file is not a file: %s", jaasFile);
     }
   }
 
+  /**
+   * Dump all tokens of a user
+   * @param user user
+   */
   public void dumpTokens(UserGroupInformation user) {
     Collection<Token<? extends TokenIdentifier>> tokens
       = user.getCredentials().getAllTokens();
@@ -355,7 +466,15 @@ public class KerberosDiags implements Closeable {
     System.err.flush();
   }
 
+  /**
+   * Print a line of output. This goes to any output file, or
+   * is logged at info. The output is flushed before and after, to
+   * try and stay in sync with JRE logging.
+   * @param format format string
+   * @param args any arguments
+   */
   private void println(String format, Object... args) {
+    flush();
     String msg = String.format(format, args);
     if (out != null) {
       out.println(msg);
@@ -365,6 +484,11 @@ public class KerberosDiags implements Closeable {
     flush();
   }
 
+  /**
+   * Print a title entry
+   * @param format format string
+   * @param args any arguments
+   */
   private void title(String format, Object... args) {
     println("");
     println("");
@@ -373,20 +497,38 @@ public class KerberosDiags implements Closeable {
     println("");
   }
 
-  private void printSysprop(String key) {
-    println("%s = \"%s\"", key, System.getProperty(key, "(unset)"));
+  /**
+   * Print a system property, or {@link #UNSET} if unset.
+   * @param property property to print
+   */
+  private void printSysprop(String property) {
+    println("%s = \"%s\"", property,
+        System.getProperty(property, UNSET));
   }
 
-  private void printConfOpt(String key) {
-    println("%s = \"%s\"", key, conf.get(key, "(unset)"));
+  /**
+   * Print a configuration option, or {@link #UNSET} if unset.
+   * @param option option to print
+   */
+  private void printConfOpt(String option) {
+    println("%s = \"%s\"", option, conf.get(option, UNSET));
   }
 
-
-  private void printEnv(String key) {
-    String env = System.getenv(key);
-    println("%s = \"%s\"", key, env != null ? env : "(unset)");
+  /**
+   * Print an environment variable's name and value; printing
+   * {@link #UNSET} if it is not set
+   * @param variable environment variable
+   */
+  private void printEnv(String variable) {
+    String env = System.getenv(variable);
+    println("%s = \"%s\"", variable, env != null ? env : UNSET);
   }
 
+  /**
+   * Dump any file to standard out; add a trailing newline
+   * @param file file to dump
+   * @throws IOException IO problems
+   */
   private void dump(File file) throws IOException {
     try (FileInputStream in = new FileInputStream(file)) {
       for (String line : IOUtils.readLines(in)) {
@@ -398,46 +540,62 @@ public class KerberosDiags implements Closeable {
 
   /**
    * Format and raise a failure
+   *
+   * @param category category for exception
    * @param message string formatting message
    * @param args any arguments for the formatting
    * @throws KerberosDiagsFailure containing the formatted text
    */
-  private void fail(String message, Object... args)
+  private void fail(String category, String message, Object... args)
     throws KerberosDiagsFailure {
-    throw new KerberosDiagsFailure(message, args);
+    throw new KerberosDiagsFailure(category, message, args);
   }
 
   /**
    * Conditional failure with string formatted arguments
    * @param condition failure condition
+   * @param category category for exception
    * @param message string formatting message
    * @param args any arguments for the formatting
    * @throws KerberosDiagsFailure containing the formatted text
    *         if the condition was met
    */
-  private void failif(boolean condition, String message, Object... args)
+  private void failif(boolean condition,
+      String category,
+      String message,
+      Object... args)
     throws KerberosDiagsFailure {
     if (condition) {
-      fail(message, args);
+      fail(category, message, args);
     }
   }
 
   /**
-   * Diagnostics failures return the exit code 41, "unauthorized"
+   * Diagnostics failures return the exit code 41, "unauthorized".
+   *
+   * They have a category, initially for testing: the category can be
+   * validated without having to match on the entire string.
    */
   public static class KerberosDiagsFailure extends ExitUtil.ExitException {
-    public KerberosDiagsFailure( String message) {
-      super(41, message);
+    private final String category;
+
+    public KerberosDiagsFailure(String category, String message) {
+      super(41, category + ":" + message);
+      this.category = category;
     }
 
-    public KerberosDiagsFailure(String message, Object... args) {
-      this(String.format(message, args));
+    public KerberosDiagsFailure(String category, String message, Object... args) {
+      this(category, String.format(message, args));
     }
 
-    public KerberosDiagsFailure(Throwable throwable,
-      String message, Object... args) {
-      this(message, args);
+    public KerberosDiagsFailure(String category, Throwable throwable,
+        String message, Object... args) {
+      this(category, message, args);
       initCause(throwable);
     }
+
+    public String getCategory() {
+      return category;
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/4fa7c18d/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 28727cb..7e644d2 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
@@ -20,7 +20,6 @@ package org.apache.slider.client;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.io.Files;
-
 import org.apache.commons.collections.CollectionUtils;
 import org.apache.commons.io.IOUtils;
 import org.apache.commons.lang.ArrayUtils;
@@ -45,6 +44,7 @@ import org.apache.hadoop.registry.client.types.Endpoint;
 import org.apache.hadoop.registry.client.types.RegistryPathStatus;
 import org.apache.hadoop.registry.client.types.ServiceRecord;
 import org.apache.hadoop.registry.client.types.yarn.YarnRegistryAttributes;
+import org.apache.hadoop.security.KerberosDiags;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.alias.CredentialProvider;
 import org.apache.hadoop.security.alias.CredentialProviderFactory;
@@ -108,7 +108,6 @@ 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;
@@ -3782,6 +3781,7 @@ public class SliderClient extends AbstractSliderLaunchedService implements RunSe
    * @throws SliderException
    * @throws IOException
    */
+  @SuppressWarnings("IOResourceOpenedButNotSafelyClosed")
   private int actionKDiag(ActionKDiagArgs args)
     throws Exception {
     PrintWriter out = new PrintWriter(System.err);
@@ -3792,10 +3792,12 @@ public class SliderClient extends AbstractSliderLaunchedService implements RunSe
     }
     try {
       KerberosDiags kdiags = new KerberosDiags(getConfig(),
-        out,
-        args.services,
-        args.keytab,
-        args.principal);
+          out,
+          args.services,
+          args.keytab,
+          args.principal,
+          args.keylen,
+          args.secure);
       kdiags.execute();
     } catch (KerberosDiags.KerberosDiagsFailure e) {
       log.error(e.toString());

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/4fa7c18d/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 874eac1..5332c4b 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
@@ -53,9 +53,15 @@ public class ActionKDiagArgs extends AbstractActionArgs {
   @Parameter(names = {ARG_KEYTAB}, description = "keytab to use")
   public File keytab;
 
+  @Parameter(names = {ARG_KEYLEN}, description = "minimum key length")
+  public int keylen = 256;
+
   @Parameter(names = {ARG_PRINCIPAL}, description = "principal to log in from a keytab")
   public String principal;
 
+  @Parameter(names = {ARG_SECURE}, description = "Is security required")
+  public boolean secure = false;
+
   @Override
   public int getMinParams() {
     return 0;

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/4fa7c18d/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 62e5e0d..d133f25 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
@@ -70,6 +70,7 @@ public interface Arguments {
   String ARG_IMAGE = "--image";
   String ARG_INSTALL = "--install";
   String ARG_INTERNAL = "--internal";
+  String ARG_KEYLEN = "--keylen";
   String ARG_KEYTAB = "--keytab";
   String ARG_KEYSTORE = "--keystore";
   String ARG_KEYTABINSTALL = ARG_INSTALL;
@@ -108,6 +109,7 @@ public interface Arguments {
   String ARG_RESOURCE_MANAGER = "--rm";
   String ARG_RESOURCE_OPT = "--resopt";
   String ARG_RESOURCE_OPT_SHORT = "-ro";
+  String ARG_SECURE = "--secure";
   String ARG_SERVICETYPE = "--servicetype";
   String ARG_SERVICES = "--services";
   String ARG_SLIDER = "--slider";

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/4fa7c18d/slider-core/src/test/groovy/org/apache/slider/client/TestDiagnostics.groovy
----------------------------------------------------------------------
diff --git a/slider-core/src/test/groovy/org/apache/slider/client/TestDiagnostics.groovy b/slider-core/src/test/groovy/org/apache/slider/client/TestDiagnostics.groovy
index ca22cbe..3e4e913 100644
--- a/slider-core/src/test/groovy/org/apache/slider/client/TestDiagnostics.groovy
+++ b/slider-core/src/test/groovy/org/apache/slider/client/TestDiagnostics.groovy
@@ -66,6 +66,7 @@ class TestDiagnostics extends YarnZKMiniClusterTestBase {
       [
         ClientArgs.ACTION_KDIAG,
         ClientArgs.ARG_FAIL,
+        ClientArgs.ARG_KEYLEN, "128",
         ClientArgs.ARG_SYSPROP,
         define(SUN_SECURITY_KRB5_DEBUG, "true")])