You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by cn...@apache.org on 2016/02/06 01:00:55 UTC

[2/3] hadoop git commit: HADOOP-12426. Add Entry point for Kerberos health check. Contributed by Steve Loughran.

HADOOP-12426. Add Entry point for Kerberos health check. Contributed by Steve Loughran.

(cherry picked from commit cfa8513890ec1cd7f71258ff3c3083f6d89a4fc4)

Conflicts:
	hadoop-common-project/hadoop-common/src/main/conf/hadoop-env.sh


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

Branch: refs/heads/branch-2
Commit: 4db6cc44cd1f1b2692ad3fdc18798cf5f32250ec
Parents: a7232f5
Author: cnauroth <cn...@apache.org>
Authored: Fri Feb 5 15:47:51 2016 -0800
Committer: cnauroth <cn...@apache.org>
Committed: Fri Feb 5 15:52:14 2016 -0800

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES.txt |   3 +
 .../hadoop-common/src/main/conf/hadoop-env.sh   |   6 +
 .../java/org/apache/hadoop/security/KDiag.java  | 976 +++++++++++++++++++
 .../hadoop/security/UserGroupInformation.java   |  14 +-
 .../src/site/markdown/SecureMode.md             | 198 ++++
 .../org/apache/hadoop/security/TestKDiag.java   | 226 +++++
 .../apache/hadoop/security/TestKDiagNoKDC.java  | 123 +++
 .../apache/hadoop/security/secure-hdfs-site.xml |  26 +
 8 files changed, 1566 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/4db6cc44/hadoop-common-project/hadoop-common/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/CHANGES.txt b/hadoop-common-project/hadoop-common/CHANGES.txt
index 58c0d83..18a1af8 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -106,6 +106,9 @@ Release 2.8.0 - UNRELEASED
 
     HADOOP-12635. Adding Append API support for WASB. (Dushyanth via cnauroth)
 
+    HADOOP-12426. Add Entry point for Kerberos health check
+    (Steve Loughran via cnauroth)
+
   IMPROVEMENTS
 
     HADOOP-12458. Retries is typoed to spell Retires in parts of

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4db6cc44/hadoop-common-project/hadoop-common/src/main/conf/hadoop-env.sh
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/conf/hadoop-env.sh b/hadoop-common-project/hadoop-common/src/main/conf/hadoop-env.sh
index 930b10c..6469e61 100644
--- a/hadoop-common-project/hadoop-common/src/main/conf/hadoop-env.sh
+++ b/hadoop-common-project/hadoop-common/src/main/conf/hadoop-env.sh
@@ -45,7 +45,13 @@ done
 #export HADOOP_HEAPSIZE=
 #export HADOOP_NAMENODE_INIT_HEAPSIZE=""
 
+# Enable extra debugging of Hadoop's JAAS binding, used to set up
+# Kerberos security.
+# export HADOOP_JAAS_DEBUG=true
+
 # Extra Java runtime options.  Empty by default.
+# For Kerberos debugging, an extended option set logs more invormation
+# export HADOOP_OPTS="-Djava.net.preferIPv4Stack=true -Dsun.security.krb5.debug=true -Dsun.security.spnego.debug"
 export HADOOP_OPTS="$HADOOP_OPTS -Djava.net.preferIPv4Stack=true"
 
 # Command specific options appended to HADOOP_OPTS when specified

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4db6cc44/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/KDiag.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/KDiag.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/KDiag.java
new file mode 100644
index 0000000..4c2b0c4
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/KDiag.java
@@ -0,0 +1,976 @@
+/*
+ * 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.directory.server.kerberos.shared.keytab.Keytab;
+import org.apache.directory.server.kerberos.shared.keytab.KeytabEntry;
+import org.apache.directory.shared.kerberos.components.EncryptionKey;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.conf.Configured;
+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.apache.hadoop.util.StringUtils;
+import org.apache.hadoop.util.Tool;
+import org.apache.hadoop.util.ToolRunner;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.crypto.Cipher;
+import java.io.Closeable;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.PrintWriter;
+import java.lang.reflect.InvocationTargetException;
+import java.net.InetAddress;
+import java.security.NoSuchAlgorithmException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Date;
+import java.util.LinkedList;
+import java.util.List;
+
+import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.*;
+import static org.apache.hadoop.security.UserGroupInformation.*;
+import static org.apache.hadoop.security.authentication.util.KerberosUtil.*;
+import static org.apache.hadoop.util.StringUtils.popOption;
+import static org.apache.hadoop.util.StringUtils.popOptionWithArgument;
+
+/**
+ * Kerberos diagnostics
+ *
+ * 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 KDiag extends Configured implements Tool, Closeable {
+
+  private static final Logger LOG = LoggerFactory.getLogger(KDiag.class);
+  /**
+   * Location of the kerberos ticket cache as passed down via an environment
+   * variable. This is what kinit will use by default: {@value}
+   */
+  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 JAVA_SECURITY_KRB5_KDC_ADDRESS
+    = "java.security.krb5.kdc";
+  public static final String SUN_SECURITY_KRB5_DEBUG
+    = "sun.security.krb5.debug";
+  public static final String SUN_SECURITY_SPNEGO_DEBUG
+    = "sun.security.spnego.debug";
+  public static final String SUN_SECURITY_JAAS_FILE
+    = "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)";
+
+  /**
+   * String seen in {@code getDefaultRealm()} exceptions if the user has
+   * no realm: {@value}.
+   */
+  public static final String NO_DEFAULT_REALM = "Cannot locate default realm";
+
+  /**
+   * The exit code for a failure of the diagnostics: 41 == HTTP 401 == unauth.
+   */
+  public static final int KDIAG_FAILURE = 41;
+  public static final String DFS_DATA_TRANSFER_SASLPROPERTIES_RESOLVER_CLASS
+      = "dfs.data.transfer.saslproperties.resolver.class";
+  public static final String DFS_DATA_TRANSFER_PROTECTION
+      = "dfs.data.transfer.protection";
+  public static final String ETC_KRB5_CONF = "/etc/krb5.conf";
+  public static final String ETC_NTP = "/etc/ntp.conf";
+  public static final String HADOOP_JAAS_DEBUG = "HADOOP_JAAS_DEBUG";
+
+  private PrintWriter out;
+  private File keytab;
+  private String principal;
+  private long minKeyLength = 256;
+  private boolean securityRequired;
+  private boolean nofail = false;
+  private boolean nologin = false;
+  private boolean jaas = false;
+
+  /**
+   * Flag set to true if a {@link #verify(boolean, String, String, Object...)}
+   * probe failed.
+   */
+  private boolean probeHasFailed = false;
+
+  public static final String CAT_CONFIG = "CONFIG";
+  public static final String CAT_JAAS = "JAAS";
+  public static final String CAT_JVM = "JVM";
+  public static final String CAT_KERBEROS = "KERBEROS";
+  public static final String CAT_LOGIN = "LOGIN";
+  public static final String CAT_OS = "JAAS";
+  public static final String CAT_SASL = "SASL";
+  public static final String CAT_UGI = "UGI";
+
+  public static final String ARG_KEYLEN = "--keylen";
+  public static final String ARG_KEYTAB = "--keytab";
+  public static final String ARG_JAAS = "--jaas";
+  public static final String ARG_NOFAIL = "--nofail";
+  public static final String ARG_NOLOGIN = "--nologin";
+  public static final String ARG_OUTPUT = "--out";
+  public static final String ARG_PRINCIPAL = "--principal";
+  public static final String ARG_RESOURCE = "--resource";
+
+  public static final String ARG_SECURE = "--secure";
+
+  @SuppressWarnings("IOResourceOpenedButNotSafelyClosed")
+  public KDiag(Configuration conf,
+      PrintWriter out,
+      File keytab,
+      String principal,
+      long minKeyLength,
+      boolean securityRequired) {
+    super(conf);
+    this.keytab = keytab;
+    this.principal = principal;
+    this.out = out;
+    this.minKeyLength = minKeyLength;
+    this.securityRequired = securityRequired;
+  }
+
+  public KDiag() {
+  }
+
+  @Override
+  public void close() throws IOException {
+    flush();
+    if (out != null) {
+      out.close();
+    }
+  }
+
+  @Override
+  public int run(String[] argv) throws Exception {
+    List<String> args = new LinkedList<>(Arrays.asList(argv));
+    String keytabName = popOptionWithArgument(ARG_KEYTAB, args);
+    if (keytabName != null) {
+      keytab = new File(keytabName);
+    }
+    principal = popOptionWithArgument(ARG_PRINCIPAL, args);
+    String outf = popOptionWithArgument(ARG_OUTPUT, args);
+    String mkl = popOptionWithArgument(ARG_KEYLEN, args);
+    if (mkl != null) {
+      minKeyLength = Integer.parseInt(mkl);
+    }
+    securityRequired = popOption(ARG_SECURE, args);
+    nofail = popOption(ARG_NOFAIL, args);
+    jaas = popOption(ARG_JAAS, args);
+    nologin = popOption(ARG_NOLOGIN, args);
+
+    // look for list of resources
+    String resource;
+    while (null != (resource = popOptionWithArgument(ARG_RESOURCE, args))) {
+      // loading a resource
+      LOG.info("Loading resource {}", resource);
+      try (InputStream in =
+               getClass().getClassLoader().getResourceAsStream(resource)) {
+        if (verify(in != null, CAT_CONFIG, "No resource %s", resource)) {
+          Configuration.addDefaultResource(resource);
+        }
+      }
+    }
+    // look for any leftovers
+    if (!args.isEmpty()) {
+      println("Unknown arguments in command:");
+      for (String s : args) {
+        println("  \"%s\"", s);
+      }
+      println();
+      println(usage());
+      return -1;
+    }
+    if (outf != null) {
+      println("Printing output to %s", outf);
+      out = new PrintWriter(new File(outf), "UTF-8");
+    }
+    execute();
+    return probeHasFailed ? KDIAG_FAILURE : 0;
+  }
+
+  private String usage() {
+    return "KDiag: Diagnose Kerberos Problems\n"
+      + arg("-D", "key=value", "Define a configuration option")
+      + arg(ARG_JAAS, "",
+      "Require a JAAS file to be defined in " + SUN_SECURITY_JAAS_FILE)
+      + arg(ARG_KEYLEN, "<keylen>",
+      "Require a minimum size for encryption keys supported by the JVM."
+      + " Default value : "+ minKeyLength)
+      + arg(ARG_KEYTAB, "<keytab> " + ARG_PRINCIPAL + " <principal>",
+          "Login from a keytab as a specific principal")
+      + arg(ARG_NOFAIL, "", "Do not fail on the first problem")
+      + arg(ARG_NOLOGIN, "", "Do not attempt to log in")
+      + arg(ARG_OUTPUT, "<file>", "Write output to a file")
+      + arg(ARG_RESOURCE, "<resource>", "Load an XML configuration resource")
+      + arg(ARG_SECURE, "", "Require the hadoop configuration to be secure");
+  }
+
+  private String arg(String name, String params, String meaning) {
+    return String.format("  [%s%s%s] : %s",
+        name, (!params.isEmpty() ? " " : ""), params, meaning) + ".\n";
+  }
+
+  /**
+   * Execute diagnostics.
+   * <p>
+   * Things it would be nice if UGI made accessible
+   * <ol>
+   *   <li>A way to enable JAAS debug programatically</li>
+   *   <li>Access to the TGT</li>
+   * </ol>
+   * @return true if security was enabled and all probes were successful
+   * @throws KerberosDiagsFailure explicitly raised failure
+   * @throws Exception other security problems
+   */
+  @SuppressWarnings("deprecation")
+  public boolean execute() throws Exception {
+
+    title("Kerberos Diagnostics scan at %s",
+        new Date(System.currentTimeMillis()));
+
+    // check that the machine has a name
+    println("Hostname = %s",
+        InetAddress.getLocalHost().getCanonicalHostName());
+
+    println("%s = %d", ARG_KEYLEN, minKeyLength);
+    println("%s = %s", ARG_KEYTAB, keytab);
+    println("%s = %s", ARG_PRINCIPAL, principal);
+
+    // Fail fast on a JVM without JCE installed.
+    validateKeyLength();
+
+    // look at realm
+    println("JVM Kerberos Login Module = %s", getKrb5LoginModuleName());
+
+    title("Core System Properties");
+    for (String prop : new String[]{
+      "user.name",
+      "java.version",
+      "java.vendor",
+      JAVA_SECURITY_KRB5_CONF,
+      JAVA_SECURITY_KRB5_REALM,
+      JAVA_SECURITY_KRB5_KDC_ADDRESS,
+      SUN_SECURITY_KRB5_DEBUG,
+      SUN_SECURITY_SPNEGO_DEBUG,
+      SUN_SECURITY_JAAS_FILE
+    }) {
+      printSysprop(prop);
+    }
+    endln();
+
+    title("All System Properties");
+    ArrayList<String> propList = new ArrayList<>(
+        System.getProperties().stringPropertyNames());
+    Collections.sort(propList, String.CASE_INSENSITIVE_ORDER);
+    for (String s : propList) {
+      printSysprop(s);
+    }
+    endln();
+
+    title("Environment Variables");
+    for (String env : new String[]{
+      HADOOP_JAAS_DEBUG,
+      KRB5_CCNAME,
+      HADOOP_USER_NAME,
+      HADOOP_PROXY_USER,
+      HADOOP_TOKEN_FILE_LOCATION,
+      "HADOOP_SECURE_LOG",
+      "HADOOP_OPTS",
+      "HADOOP_CLIENT_OPTS",
+    }) {
+      printEnv(env);
+    }
+    endln();
+
+    title("Configuration Options");
+    for (String prop : new String[]{
+      KERBEROS_KINIT_COMMAND,
+      HADOOP_SECURITY_AUTHENTICATION,
+      HADOOP_SECURITY_AUTHORIZATION,
+      "hadoop.kerberos.min.seconds.before.relogin",    // not in 2.6
+      "hadoop.security.dns.interface",   // not in 2.6
+      "hadoop.security.dns.nameserver",  // not in 2.6
+      HADOOP_RPC_PROTECTION,
+      HADOOP_SECURITY_SASL_PROPS_RESOLVER_CLASS,
+      HADOOP_SECURITY_CRYPTO_CODEC_CLASSES_KEY_PREFIX,
+      HADOOP_SECURITY_GROUP_MAPPING,
+      "hadoop.security.impersonation.provider.class",    // not in 2.6
+      DFS_DATA_TRANSFER_PROTECTION, // HDFS
+      DFS_DATA_TRANSFER_SASLPROPERTIES_RESOLVER_CLASS // HDFS
+    }) {
+      printConfOpt(prop);
+    }
+
+    // check that authentication is enabled
+    Configuration conf = getConf();
+    if (isSimpleAuthentication(conf)) {
+      println(HADOOP_AUTHENTICATION_IS_DISABLED);
+      failif(securityRequired, CAT_CONFIG, HADOOP_AUTHENTICATION_IS_DISABLED);
+      // no security, warn
+      LOG.warn("Security is not enabled for the Hadoop cluster");
+    } else {
+      if (isSimpleAuthentication(new Configuration())) {
+        LOG.warn("The default cluster security is insecure");
+        failif(securityRequired, CAT_CONFIG, HADOOP_AUTHENTICATION_IS_DISABLED);
+      }
+    }
+
+
+    // now the big test: login, then try again
+    boolean krb5Debug = getAndSet(SUN_SECURITY_KRB5_DEBUG);
+    boolean spnegoDebug = getAndSet(SUN_SECURITY_SPNEGO_DEBUG);
+
+    try {
+      UserGroupInformation.setConfiguration(conf);
+      validateKrb5File();
+      printDefaultRealm();
+      validateSasl(HADOOP_SECURITY_SASL_PROPS_RESOLVER_CLASS);
+      if (conf.get(DFS_DATA_TRANSFER_SASLPROPERTIES_RESOLVER_CLASS) != null) {
+        validateSasl(DFS_DATA_TRANSFER_SASLPROPERTIES_RESOLVER_CLASS);
+      }
+      validateKinitExecutable();
+      validateJAAS(jaas);
+      validateNTPConf();
+
+      if (!nologin) {
+        title("Logging in");
+        if (keytab != null) {
+          dumpKeytab(keytab);
+          loginFromKeytab();
+        } else {
+          UserGroupInformation loginUser = getLoginUser();
+          dumpUGI("Log in user", loginUser);
+          validateUGI("Login user", loginUser);
+          println("Ticket based login: %b", isLoginTicketBased());
+          println("Keytab based login: %b", isLoginKeytabBased());
+        }
+      }
+
+      return true;
+    } finally {
+      // restore original system properties
+      System.setProperty(SUN_SECURITY_KRB5_DEBUG,
+        Boolean.toString(krb5Debug));
+      System.setProperty(SUN_SECURITY_SPNEGO_DEBUG,
+        Boolean.toString(spnegoDebug));
+    }
+  }
+
+  /**
+   * Is the authentication method of this configuration "simple"?
+   * @param conf configuration to check
+   * @return true if auth is simple (i.e. not kerberos)
+   */
+  protected boolean isSimpleAuthentication(Configuration conf) {
+    return SecurityUtil.getAuthenticationMethod(conf)
+        .equals(AuthenticationMethod.SIMPLE);
+  }
+
+  /**
+   * 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 bits", aesLen);
+    verify(minKeyLength <= aesLen,
+        CAT_JVM,
+        "Java Cryptography Extensions are not installed on this JVM."
+            + " Maximum supported key length %s - minimum required %d",
+        aesLen, minKeyLength);
+  }
+
+  /**
+   * Get the default realm.
+   * <p>
+   * Not having a default realm may be harmless, so is noted at info.
+   * All other invocation failures are downgraded to warn, as
+   * follow-on actions may still work.
+   * Failure to invoke the method via introspection is considered a failure,
+   * as it's a sign of JVM compatibility issues that may have other 
+   * consequences
+   */
+  protected void printDefaultRealm() {
+    try {
+      String defaultRealm = getDefaultRealm();
+      println("Default Realm = %s", defaultRealm);
+      if (defaultRealm == null) {
+        warn(CAT_KERBEROS, "Host has no default realm");
+      }
+    } catch (ClassNotFoundException
+        | IllegalAccessException
+        | NoSuchMethodException e) {
+      throw new KerberosDiagsFailure(CAT_JVM, e,
+          "Failed to invoke krb5.Config.getDefaultRealm: %s: " +e, e);
+    } catch (InvocationTargetException e) {
+      Throwable cause = e.getCause() != null ? e.getCause() : e;
+      if (cause.toString().contains(NO_DEFAULT_REALM)) {
+        // exception raised if there is no default realm. This is not
+        // always a problem, so downgrade to a message.
+        warn(CAT_KERBEROS, "Host has no default realm");
+        LOG.debug(cause.toString(), cause);
+      } else {
+        error(CAT_KERBEROS, "Kerberos.getDefaultRealm() failed: %s\n%s",
+            cause, StringUtils.stringifyException(cause));
+      }
+    }
+  }
+
+  /**
+   * Locate the {@code krb5.conf} file and dump it.
+   *
+   * No-op on windows.
+   * @throws IOException problems reading the file.
+   */
+  private void validateKrb5File() throws IOException {
+    if (!Shell.WINDOWS) {
+      title("Locating Kerberos configuration file");
+      String krbPath = ETC_KRB5_CONF;
+      String jvmKrbPath = System.getProperty(JAVA_SECURITY_KRB5_CONF);
+      if (jvmKrbPath != null && !jvmKrbPath.isEmpty()) {
+        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);
+      dump(krbFile);
+      endln();
+    }
+  }
+
+  /**
+   * 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);
+    File kt = keytabFile.getCanonicalFile();
+    verifyFileIsValid(kt, CAT_KERBEROS, "keytab");
+    List<KeytabEntry> entries = Keytab.read(kt).getEntries();
+    println("keytab entry count: %d", entries.size());
+    for (KeytabEntry entry : entries) {
+      EncryptionKey key = entry.getKey();
+      println(" %s: version=%d expires=%s encryption=%s",
+          entry.getPrincipalName(),
+          entry.getKeyVersion(),
+          entry.getTimeStamp(),
+          key.getKeyType());
+    }
+    endln();
+  }
+
+  /**
+   * Log in from a keytab, dump the UGI, validate it, then try and log in again.
+   *
+   * That second-time login catches JVM/Hadoop compatibility problems.
+   * @throws IOException Keytab loading problems
+   */
+  private void loginFromKeytab() throws IOException {
+    UserGroupInformation ugi;
+    String identity;
+    if (keytab != null) {
+      File kt = keytab.getCanonicalFile();
+      println("Using keytab %s principal %s", kt, principal);
+      identity = principal;
+
+      failif(principal == null, CAT_KERBEROS, "No principal defined");
+      ugi = loginUserFromKeytabAndReturnUGI(principal, kt.getPath());
+      dumpUGI(identity, ugi);
+      validateUGI(principal, ugi);
+
+      title("Attempting to relogin");
+      try {
+        // package scoped -hence the reason why this class must be in the
+        // hadoop.security package
+        setShouldRenewImmediatelyForTests(true);
+        // attempt a new login
+        ugi.reloginFromKeytab();
+      } catch (IllegalAccessError e) {
+        // if you've built this class into an independent JAR, package-access
+        // may fail. Downgrade
+        warn(CAT_UGI, "Failed to reset UGI -and so could not try to relogin");
+        LOG.debug("Failed to reset UGI: {}", e, e);
+      }
+    } else {
+      println("No keytab: attempting to log in is as current user");
+    }
+  }
+
+  /**
+   * Dump a UGI.
+   *
+   * @param title title of this section
+   * @param ugi UGI to dump
+   * @throws IOException
+   */
+  private void dumpUGI(String title, UserGroupInformation ugi)
+    throws IOException {
+    title(title);
+    println("UGI instance = %s", ugi);
+    println("Has kerberos credentials: %b", ugi.hasKerberosCredentials());
+    println("Authentication method: %s", ugi.getAuthenticationMethod());
+    println("Real Authentication method: %s",
+      ugi.getRealAuthenticationMethod());
+    title("Group names");
+    for (String name : ugi.getGroupNames()) {
+      println(name);
+    }
+    title("Credentials");
+    List<Text> secretKeys = ugi.getCredentials().getAllSecretKeys();
+    title("Secret keys");
+    if (!secretKeys.isEmpty()) {
+      for (Text secret: secretKeys) {
+        println("%s", secret);
+      }
+    } else {
+      println("(none)");
+    }
+
+    dumpTokens(ugi);
+  }
+
+  /**
+   * Validate the UGI: verify it is kerberized.
+   * @param messagePrefix message in exceptions
+   * @param user user to validate
+   */
+  private void validateUGI(String messagePrefix, UserGroupInformation user) {
+    if (verify(user.getAuthenticationMethod() == AuthenticationMethod.KERBEROS,
+        CAT_LOGIN, "User %s is not authenticated by Kerberos", user)) {
+      verify(user.hasKerberosCredentials(),
+          CAT_LOGIN, "%s: No kerberos credentials for %s", messagePrefix, user);
+      verify(user.getAuthenticationMethod() != null,
+          CAT_LOGIN, "%s: Null AuthenticationMethod for %s", messagePrefix,
+          user);
+    }
+  }
+
+  /**
+   * A cursory look at the {@code kinit} executable.
+   *
+   * If it is an absolute path: it must exist with a size > 0.
+   * If it is just a command, it has to be on the path. There's no check
+   * for that -but the PATH is printed out.
+   */
+  private void validateKinitExecutable() {
+    String kinit = getConf().getTrimmed(KERBEROS_KINIT_COMMAND, "");
+    if (!kinit.isEmpty()) {
+      File kinitPath = new File(kinit);
+      println("%s = %s", KERBEROS_KINIT_COMMAND, kinitPath);
+      if (kinitPath.isAbsolute()) {
+        verifyFileIsValid(kinitPath, CAT_KERBEROS, KERBEROS_KINIT_COMMAND);
+      } else {
+        println("Executable %s is relative -must be on the PATH", kinit);
+        printEnv("PATH");
+      }
+    }
+  }
+
+  /**
+   * 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 = getConf().getTrimmed(saslPropsResolverKey);
+    try {
+      Class<? extends SaslPropertiesResolver> resolverClass =
+          getConf().getClass(
+          saslPropsResolverKey,
+          SaslPropertiesResolver.class,
+          SaslPropertiesResolver.class);
+      println("Resolver is %s", resolverClass);
+    } catch (RuntimeException e) {
+      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.
+   * @param jaasRequired is JAAS required
+   */
+  private void validateJAAS(boolean jaasRequired) throws IOException {
+    String jaasFilename = System.getProperty(SUN_SECURITY_JAAS_FILE);
+    if (jaasRequired) {
+      verify(jaasFilename != null, CAT_JAAS,
+          "No JAAS file specified in " + SUN_SECURITY_JAAS_FILE);
+    }
+    if (jaasFilename != null) {
+      title("JAAS");
+      File jaasFile = new File(jaasFilename);
+      println("JAAS file is defined in %s: %s",
+          SUN_SECURITY_JAAS_FILE, jaasFile);
+      verifyFileIsValid(jaasFile, CAT_JAAS,
+          "JAAS file defined in " + SUN_SECURITY_JAAS_FILE);
+      dump(jaasFile);
+      endln();
+    }
+  }
+
+  private void validateNTPConf() throws IOException {
+    if (!Shell.WINDOWS) {
+      File ntpfile = new File(ETC_NTP);
+      if (ntpfile.exists()
+          && verifyFileIsValid(ntpfile, CAT_OS,
+          "NTP file: " + ntpfile)) {
+        title("NTP");
+        dump(ntpfile);
+        endln();
+      }
+    }
+  }
+
+
+  /**
+   * Verify that a file is valid: it is a file, non-empty and readable.
+   * @param file file
+   * @param category category for exceptions
+   * @param text text message
+   * @return true if the validation held; false if it did not <i>and</i>
+   * {@link #nofail} has disabled raising exceptions.
+   */
+  private boolean verifyFileIsValid(File file, String category, String text) {
+    return verify(file.exists(), category,
+        "%s file does not exist: %s",
+        text, file)
+     && verify(file.isFile(), category,
+        "%s path does not refer to a file: %s", text, file)
+     && verify(file.length() != 0, category,
+        "%s file is empty: %s", text, file)
+      && verify(file.canRead(), category,
+        "%s file is not readable: %s", text, file);
+  }
+
+  /**
+   * Dump all tokens of a UGI.
+   * @param ugi UGI to examine
+   */
+  public void dumpTokens(UserGroupInformation ugi) {
+    Collection<Token<? extends TokenIdentifier>> tokens
+      = ugi.getCredentials().getAllTokens();
+    title("Token Count: %d", tokens.size());
+    for (Token<? extends TokenIdentifier> token : tokens) {
+      println("Token %s", token.getKind());
+    }
+    endln();
+  }
+
+  /**
+   * Set the System property to true; return the old value for caching.
+   *
+   * @param sysprop property
+   * @return the previous value
+   */
+  private boolean getAndSet(String sysprop) {
+    boolean old = Boolean.getBoolean(sysprop);
+    System.setProperty(sysprop, "true");
+    return old;
+  }
+
+  /**
+   * Flush all active output channels, including {@Code System.err},
+   * so as to stay in sync with any JRE log messages.
+   */
+  private void flush() {
+    if (out != null) {
+      out.flush();
+    } else {
+      System.out.flush();
+    }
+    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);
+    } else {
+      System.out.println(msg);
+    }
+    flush();
+  }
+
+  /**
+   * Print a new line
+   */
+  private void println() {
+    println("");
+  }
+
+  /**
+   * Print something at the end of a section
+   */
+  private void endln() {
+    println();
+    println("-----");
+  }
+
+  /**
+   * Print a title entry.
+   *
+   * @param format format string
+   * @param args any arguments
+   */
+  private void title(String format, Object... args) {
+    println();
+    println();
+    println("== " + String.format(format, args) + " ==");
+    println();
+  }
+
+  /**
+   * 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));
+  }
+
+  /**
+   * Print a configuration option, or {@link #UNSET} if unset.
+   *
+   * @param option option to print
+   */
+  private void printConfOpt(String option) {
+    println("%s = \"%s\"", option, getConf().get(option, 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.
+   * @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)) {
+        println(line);
+      }
+    }
+  }
+
+  /**
+   * 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 category, String message, Object... args)
+    throws KerberosDiagsFailure {
+    error(category, message, args);
+    throw new KerberosDiagsFailure(category, message, args);
+  }
+
+  /**
+   * Assert that a condition must hold.
+   *
+   * If not, an exception is raised, or, if {@link #nofail} is set,
+   * an error will be logged and the method return false.
+   *
+   * @param condition condition which must hold
+   * @param category category for exception
+   * @param message string formatting message
+   * @param args any arguments for the formatting
+   * @return true if the verification succeeded, false if it failed but
+   * an exception was not raised.
+   * @throws KerberosDiagsFailure containing the formatted text
+   *         if the condition was met
+   */
+  private boolean verify(boolean condition,
+      String category,
+      String message,
+      Object... args)
+    throws KerberosDiagsFailure {
+    if (!condition) {
+      // condition not met: fail or report
+      probeHasFailed = true;
+      if (!nofail) {
+        fail(category, message, args);
+      } else {
+        error(category, message, args);
+      }
+      return false;
+    } else {
+      // condition is met
+      return true;
+    }
+  }
+
+  /**
+   * Print a message as an error
+   * @param category error category
+   * @param message format string
+   * @param args list of arguments
+   */
+  private void error(String category, String message, Object...args) {
+    println("ERROR: %s: %s", category, String.format(message, args));
+  }
+  /**
+   * Print a message as an warning
+   * @param category error category
+   * @param message format string
+   * @param args list of arguments
+   */
+  private void warn(String category, String message, Object...args) {
+    println("WARNING: %s: %s", category, String.format(message, args));
+  }
+
+  /**
+   * Conditional failure with string formatted arguments.
+   * There is no chek for the {@link #nofail} value.
+   * @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 category,
+      String message,
+      Object... args)
+      throws KerberosDiagsFailure {
+    if (condition) {
+      fail(category, message, args);
+    }
+  }
+
+  /**
+   * Inner entry point, with no logging or system exits.
+   *
+   * @param conf configuration
+   * @param argv argument list
+   * @return an exception
+   * @throws Exception
+   */
+  public static int exec(Configuration conf, String... argv) throws Exception {
+    try(KDiag kdiag = new KDiag()) {
+      return ToolRunner.run(conf, kdiag, argv);
+    }
+  }
+
+  /**
+   * Main entry point.
+   * @param argv args list
+   */
+  public static void main(String[] argv) {
+    try {
+      ExitUtil.terminate(exec(new Configuration(), argv));
+    } catch (ExitUtil.ExitException e) {
+      LOG.error(e.toString());
+      System.exit(e.status);
+    } catch (Exception e) {
+      LOG.error(e.toString(), e);
+      ExitUtil.halt(-1, e);
+    }
+  }
+
+  /**
+   * 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 {
+    private final String category;
+
+    public KerberosDiagsFailure(String category, String message) {
+      super(KDIAG_FAILURE, category + ": " + message);
+      this.category = category;
+    }
+
+    public KerberosDiagsFailure(String category,
+        String message,
+        Object... args) {
+      this(category, String.format(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/hadoop/blob/4db6cc44/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/UserGroupInformation.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/UserGroupInformation.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/UserGroupInformation.java
index 7d8a88e..b303127 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/UserGroupInformation.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/UserGroupInformation.java
@@ -821,7 +821,7 @@ public class UserGroupInformation {
       loginUser.spawnAutoRenewalThreadForUserCreds();
     } catch (LoginException le) {
       LOG.debug("failure to login", le);
-      throw new IOException("failure to login", le);
+      throw new IOException("failure to login: " + le, le);
     }
     if (LOG.isDebugEnabled()) {
       LOG.debug("UGI loginUser:"+loginUser);
@@ -993,7 +993,8 @@ public class UserGroupInformation {
       }
     } catch (LoginException le) {
       throw new IOException("Logout failure for " + user + " from keytab " +
-          keytabFile, le);
+          keytabFile + ": " + le,
+          le);
     }
 
     LOG.info("Logout successful for user " + keytabPrincipal
@@ -1083,7 +1084,7 @@ public class UserGroupInformation {
         metrics.loginFailure.add(Time.now() - start);
       }
       throw new IOException("Login failure for " + keytabPrincipal + 
-          " from keytab " + keytabFile, le);
+          " from keytab " + keytabFile + ": " + le, le);
     } 
   }
 
@@ -1131,7 +1132,8 @@ public class UserGroupInformation {
       login.login();
       setLogin(login);
     } catch (LoginException le) {
-      throw new IOException("Login failure for " + getUserName(), le);
+      throw new IOException("Login failure for " + getUserName() + ": " + le,
+          le);
     } 
   }
 
@@ -1178,7 +1180,7 @@ public class UserGroupInformation {
         metrics.loginFailure.add(Time.now() - start);
       }
       throw new IOException("Login failure for " + user + " from keytab " + 
-                            path, le);
+                            path + ": " + le, le);
     } finally {
       if(oldKeytabFile != null) keytabFile = oldKeytabFile;
       if(oldKeytabPrincipal != null) keytabPrincipal = oldKeytabPrincipal;
@@ -1698,7 +1700,7 @@ public class UserGroupInformation {
       }
       if (cause == null) {
         throw new RuntimeException("PrivilegedActionException with no " +
-                "underlying cause. UGI [" + this + "]", pae);
+                "underlying cause. UGI [" + this + "]" +": " + pae, pae);
       } else if (cause instanceof IOException) {
         throw (IOException) cause;
       } else if (cause instanceof Error) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4db6cc44/hadoop-common-project/hadoop-common/src/site/markdown/SecureMode.md
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/site/markdown/SecureMode.md b/hadoop-common-project/hadoop-common/src/site/markdown/SecureMode.md
index c8e5337..6bfe5fe 100644
--- a/hadoop-common-project/hadoop-common/src/site/markdown/SecureMode.md
+++ b/hadoop-common-project/hadoop-common/src/site/markdown/SecureMode.md
@@ -411,6 +411,204 @@ Multihoming
 
 Multihomed setups where each host has multiple hostnames in DNS (e.g. different hostnames corresponding to public and private network interfaces) may require additional configuration to get Kerberos authentication working. See [HDFS Support for Multihomed Networks](../hadoop-hdfs/HdfsMultihoming.html)
 
+Troubleshooting
+---------------
+
+Kerberos is hard to set up —and harder to debug. Common problems are
+
+1. Network and DNS configuration.
+2. Kerberos configuration on hosts (`/etc/krb5.conf`).
+3. Keytab creation and maintenance.
+4. Environment setup: JVM, user login, system clocks, etc.
+
+The fact that the error messages from the JVM are essentially meaningless
+does not aid in diagnosing and fixing such problems.
+
+Extra debugging information can be enabled for the client and for any service
+
+Set the environment variable `HADOOP_JAAS_DEBUG` to `true`.
+
+```bash
+export HADOOP_JAAS_DEBUG=true
+```
+
+Edit the `log4j.properties` file to log Hadoop's security package at `DEBUG` level. 
+
+```
+log4j.logger.org.apache.hadoop.security=DEBUG
+```
+
+Enable JVM-level debugging by setting some system properties.
+
+```
+export HADOOP_OPTS="-Djava.net.preferIPv4Stack=true -Dsun.security.krb5.debug=true -Dsun.security.spnego.debug"
+```
+
+Troubleshooting with `KDiag`
+---------------------------
+
+Hadoop has a tool to aid validating setup: `KDiag`
+
+It contains a series of probes for the JVM's configuration and the environment,
+dumps out some system files (`/etc/krb5.conf`, `/etc/ntp.conf`), prints
+out some system state and then attempts to log in to Kerberos as the current user,
+or a specific principal in a named keytab.
+ 
+The output of the command can be used for local diagnostics, or forwarded to
+whoever supports the cluster.
+
+The `KDiag` command has its own entry point; it is currently not hooked up
+to the end-user CLI. 
+
+It is invoked simply by passing its full classname to one of the `bin/hadoop`,
+`bin/hdfs` or `bin/yarn` commands. Accordingly, it will display the kerberos client
+state of the command used to invoke it.
+
+```
+hadoop org.apache.hadoop.security.KDiag 
+hdfs org.apache.hadoop.security.KDiag
+yarn org.apache.hadoop.security.KDiag
+```
+
+The command returns a status code of 0 for a successful diagnostics run.
+This does not imply that Kerberos is working —merely that the KDiag command
+did not identify any problem from its limited set of probes. In particular,
+as it does not attempt to connect to any remote service, it does not verify
+that the client is trusted by any service.
+
+If unsuccessful, exit codes are
+
+* -1: the command failed for an unknown reason
+* 41: Unauthorized (== HTTP's 401). KDiag detected a condition which causes
+Kerberos to not work. Examine the output to identify the issue.
+
+### Usage
+
+```
+KDiag: Diagnose Kerberos Problems
+  [-D key=value] : Define a configuration option.
+  [--jaas] : Require a JAAS file to be defined in java.security.auth.login.config.
+  [--keylen <keylen>] : Require a minimum size for encryption keys supported by the JVM. Default value : 256.
+  [--keytab <keytab> --principal <principal>] : Login from a keytab as a specific principal.
+  [--nofail] : Do not fail on the first problem.
+  [--nologin] : Do not attempt to log in.
+  [--out <file>] : Write output to a file.
+  [--resource <resource>] : Load an XML configuration resource.
+  [--secure] : Require the hadoop configuration to be secure.
+```
+
+#### `--jaas`: Require a JAAS file to be defined in `java.security.auth.login.config`.
+
+If `--jaas` is set, the Java system property `java.security.auth.login.config` must be
+set to a JAAS file; this file must exist, be a simple file of non-zero bytes,
+and readable by the current user. More detailed validation is not performed.
+
+JAAS files are not needed by Hadoop itself, but some services (such as Zookeeper)
+do require them for secure operation.
+
+#### `--keylen <length>`: Require a minimum size for encryption keys supported by the JVM".
+
+If the JVM does not support this length, the command will fail.
+
+The default value is to 256, as needed for the `AES256` encryption scheme.
+A JVM without the Java Cryptography Extensions installed does not support
+such a key length. Kerberos will not work unless configured to use
+an encryption scheme with a shorter key length.
+
+#### `--keytab <keytab> --principal <principal>`: Log in from a keytab.
+
+Log in from a keytab as the specific principal.
+
+1. The file must contain the specific principal, including any named host.
+That is, there is no mapping from `_HOST` to the current hostname.
+1. KDiag will log out and attempt to log back in again. This catches
+JVM compatibility problems which have existed in the past. (Hadoop's
+Kerberos support requires use of/introspection into JVM-specific classes).
+
+#### `--nofail` : Do not fail on the first problem
+
+KDiag will make a best-effort attempt to diagnose all Kerberos problems,
+rather than stop at the first one.
+
+This is somewhat limited; checks are made in the order which problems
+surface (e.g keylength is checked first), so an early failure can trigger
+many more problems. But it does produce a more detailed report.
+
+#### `--nologin`: Do not attempt to log in.
+
+Skip trying to log in. This takes precedence over the `--keytab` option,
+and also disables trying to log in to kerberos as the current kinited user.
+
+This is useful when the KDiag command is being invoked within an application,
+as it does not set up Hadoop's static security state —merely check for
+some basic Kerberos preconditions.
+
+#### `--out outfile`: Write output to file.
+
+```
+hadoop org.apache.hadoop.security.KDiag --out out.txt
+```
+
+Much of the diagnostics information comes from the JRE (to `stderr`) and
+from Log4j (to `stdout`).
+To get all the output, it is best to redirect both these output streams
+to the same file, and omit the `--out` option.
+
+```
+hadoop org.apache.hadoop.security.KDiag --keytab zk.service.keytab --principal zookeeper/devix.example.org@REALM > out.txt 2>&1
+```
+
+Even there, the output of the two streams, emitted across multiple threads, can
+be a bit confusing. It will get easier with practise. Looking at the thread
+name in the Log4j output to distinguish background threads from the main thread
+helps at the hadoop level, but doesn't assist in JVM-level logging.
+
+#### `--resource <resource>` : XML configuration resource to load.
+
+When using the `hdfs` and `yarn` commands, it is often useful to force
+load the `hdfs-site.xml` and `yarn-site.xml` resource files, to pick up any Kerberos-related
+configuration options therein.
+The `core-default` and `core-site` XML resources are always loaded.
+
+```
+hdfs org.apache.hadoop.security.KDiag --resource hbase-default.xml --resource hbase-site.xml
+yarn org.apache.hadoop.security.KDiag --resource yarn-default.xml --resource yarn-site.xml
+```
+
+For extra logging during the operation, set the logging and `HADOOP_JAAS_DEBUG` 
+environment variable to the values listed in "Troubleshooting". The JVM 
+options are automatically set in KDiag.
+
+#### `--secure`: Fail if the command is not executed on a secure cluster.
+
+That is: if the authentication mechanism of the cluster is explicitly
+or implicitly set to "simple":
+
+```xml
+<property>
+  <name>hadoop.security.authentication</name>
+  <value>simple</value>
+</property>
+```
+
+Needless to say, an application so configured cannot talk to a secure Hadoop cluster.
+
+### Example
+
+```
+hdfs org.apache.hadoop.security.KDiag \
+  --nofail \
+  --resource hbase-default.xml --resource hbase-site.xml \
+  --keylen 1024 \
+  --keytab zk.service.keytab --principal zookeeper/devix.example.org@REALM
+```
+ 
+This attempts to to perform all diagnostics without failing early, load in
+the HDFS and YARN XML resources, require a minimum key length of 1024 bytes,
+and log in as the principal `zookeeper/devix.example.org@REALM`, whose key must be in
+the keytab `zk.service.keytab`
+
+
 References
 ----------
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4db6cc44/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestKDiag.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestKDiag.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestKDiag.java
new file mode 100644
index 0000000..1ba11cc
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestKDiag.java
@@ -0,0 +1,226 @@
+/*
+ * 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.hadoop.conf.Configuration;
+import org.apache.hadoop.minikdc.MiniKdc;
+import org.apache.hadoop.test.GenericTestUtils;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+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.FileInputStream;
+import java.io.IOException;
+import java.util.Properties;
+
+import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHENTICATION;
+import static org.apache.hadoop.security.KDiag.*;
+
+public class TestKDiag extends Assert {
+  private static final Logger LOG = LoggerFactory.getLogger(TestKDiag.class);
+
+  public static final String KEYLEN = "128";
+  public static final String HDFS_SITE_XML
+      = "org/apache/hadoop/security/secure-hdfs-site.xml";
+
+  @Rule
+  public TestName methodName = new TestName();
+
+  @Rule
+  public Timeout testTimeout = new Timeout(30000);
+
+  @BeforeClass
+  public static void nameThread() {
+    Thread.currentThread().setName("JUnit");
+  }
+
+  private static MiniKdc kdc;
+  private static File workDir;
+  private static File keytab;
+  private static Properties securityProperties;
+  private static Configuration conf;
+
+  @BeforeClass
+  public static void startMiniKdc() throws Exception {
+    workDir = new File(System.getProperty("test.dir", "target"));
+    securityProperties = MiniKdc.createConf();
+    kdc = new MiniKdc(securityProperties, workDir);
+    kdc.start();
+    keytab = createKeytab("foo");
+    conf = new Configuration();
+    conf.set(HADOOP_SECURITY_AUTHENTICATION, "KERBEROS");
+  }
+
+  @AfterClass
+  public static synchronized void stopMiniKdc() {
+    if (kdc != null) {
+      kdc.stop();
+      kdc = null;
+    }
+  }
+
+  @Before
+  public void reset() {
+    UserGroupInformation.reset();
+  }
+
+  private static File createKeytab(String...principals) throws Exception {
+    File keytab = new File(workDir, "keytab");
+    kdc.createPrincipal(keytab, principals);
+    return keytab;
+  }
+
+  /**
+   * Exec KDiag and expect a failure of a given category
+   * @param category category
+   * @param args args list
+   * @throws Exception any unexpected exception
+   */
+  void kdiagFailure(String category, String ...args) throws Exception {
+    try {
+      int ex = exec(conf, args);
+      LOG.error("Expected an exception in category {}, return code {}",
+          category, ex);
+    } catch (KerberosDiagsFailure e) {
+      if (!e.getCategory().equals(category)) {
+        LOG.error("Expected an exception in category {}, got {}",
+            category, e, e);
+        throw e;
+      }
+    }
+  }
+
+  void kdiag(String... args) throws Exception {
+    KDiag.exec(conf, args);
+  }
+
+  @Test
+  public void testBasicLoginFailure() throws Throwable {
+    kdiagFailure(CAT_LOGIN, ARG_KEYLEN, KEYLEN);
+  }
+
+  @Test
+  public void testBasicLoginSkipped() throws Throwable {
+    kdiagFailure(CAT_LOGIN, ARG_KEYLEN, KEYLEN, ARG_NOLOGIN);
+  }
+
+  /**
+   * This fails as the default cluster config is checked along with
+   * the CLI
+   * @throws Throwable
+   */
+  @Test
+  public void testSecure() throws Throwable {
+    kdiagFailure(CAT_CONFIG, ARG_KEYLEN, KEYLEN, ARG_SECURE);
+  }
+
+  @Test
+  public void testNoKeytab() throws Throwable {
+    kdiagFailure(CAT_KERBEROS, ARG_KEYLEN, KEYLEN,
+        ARG_KEYTAB, "target/nofile");
+  }
+
+  @Test
+  public void testKeytabNoPrincipal() throws Throwable {
+    kdiagFailure(CAT_KERBEROS, ARG_KEYLEN, KEYLEN,
+        ARG_KEYTAB, keytab.getAbsolutePath());
+  }
+
+  @Test
+  public void testConfIsSecure() throws Throwable {
+    Assert.assertFalse(SecurityUtil.getAuthenticationMethod(conf)
+        .equals(UserGroupInformation.AuthenticationMethod.SIMPLE));
+  }
+
+  @Test
+  public void testKeytabAndPrincipal() throws Throwable {
+    kdiag(ARG_KEYLEN, KEYLEN,
+        ARG_KEYTAB, keytab.getAbsolutePath(),
+        ARG_PRINCIPAL, "foo@EXAMPLE.COM");
+  }
+
+  @Test
+  public void testFileOutput() throws Throwable {
+    File f = new File("target/kdiag.txt");
+    kdiag(ARG_KEYLEN, KEYLEN,
+        ARG_KEYTAB, keytab.getAbsolutePath(),
+        ARG_PRINCIPAL, "foo@EXAMPLE.COM",
+        ARG_OUTPUT, f.getAbsolutePath());
+    LOG.info("Output of {}", f);
+    dump(f);
+  }
+
+  @Test
+  public void testLoadResource() throws Throwable {
+    kdiag(ARG_KEYLEN, KEYLEN,
+        ARG_RESOURCE, HDFS_SITE_XML,
+        ARG_KEYTAB, keytab.getAbsolutePath(),
+        ARG_PRINCIPAL, "foo@EXAMPLE.COM");
+  }
+
+  @Test
+  public void testLoadInvalidResource() throws Throwable {
+    kdiagFailure(CAT_CONFIG,
+        ARG_KEYLEN, KEYLEN,
+        ARG_RESOURCE, "no-such-resource.xml",
+        ARG_KEYTAB, keytab.getAbsolutePath(),
+        ARG_PRINCIPAL, "foo@EXAMPLE.COM");
+  }
+
+  @Test
+  public void testRequireJAAS() throws Throwable {
+    kdiagFailure(CAT_JAAS,
+        ARG_KEYLEN, KEYLEN,
+        ARG_JAAS,
+        ARG_KEYTAB, keytab.getAbsolutePath(),
+        ARG_PRINCIPAL, "foo@EXAMPLE.COM");
+  }
+
+/*
+ commented out as once JVM gets configured, it stays configured
+  @Test(expected = IOException.class)
+  public void testKeytabUnknownPrincipal() throws Throwable {
+    kdiag(ARG_KEYLEN, KEYLEN,
+        ARG_KEYTAB, keytab.getAbsolutePath(),
+        ARG_PRINCIPAL, "bob@EXAMPLE.COM");
+  }
+*/
+
+  /**
+   * Dump any file to standard out.
+   * @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)) {
+        LOG.info(line);
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4db6cc44/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestKDiagNoKDC.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestKDiagNoKDC.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestKDiagNoKDC.java
new file mode 100644
index 0000000..9d4b87f
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestKDiagNoKDC.java
@@ -0,0 +1,123 @@
+/*
+ * 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.hadoop.conf.Configuration;
+import org.apache.hadoop.minikdc.MiniKdc;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TestName;
+import org.junit.rules.Timeout;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.util.Properties;
+
+import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHENTICATION;
+import static org.apache.hadoop.security.KDiag.ARG_KEYLEN;
+import static org.apache.hadoop.security.KDiag.ARG_KEYTAB;
+import static org.apache.hadoop.security.KDiag.ARG_NOFAIL;
+import static org.apache.hadoop.security.KDiag.ARG_NOLOGIN;
+import static org.apache.hadoop.security.KDiag.ARG_PRINCIPAL;
+import static org.apache.hadoop.security.KDiag.ARG_SECURE;
+import static org.apache.hadoop.security.KDiag.CAT_CONFIG;
+import static org.apache.hadoop.security.KDiag.CAT_KERBEROS;
+import static org.apache.hadoop.security.KDiag.CAT_LOGIN;
+import static org.apache.hadoop.security.KDiag.KerberosDiagsFailure;
+import static org.apache.hadoop.security.KDiag.exec;
+
+public class TestKDiagNoKDC extends Assert {
+  private static final Logger LOG = LoggerFactory.getLogger(TestKDiagNoKDC.class);
+
+  public static final String KEYLEN = "128";
+
+  @Rule
+  public TestName methodName = new TestName();
+
+  @Rule
+  public Timeout testTimeout = new Timeout(30000);
+
+  @BeforeClass
+  public static void nameThread() {
+    Thread.currentThread().setName("JUnit");
+  }
+
+  private static Configuration conf = new Configuration();
+
+
+  @Before
+  public void reset() {
+    UserGroupInformation.reset();
+  }
+
+  /**
+   * Exec KDiag and expect a failure of a given category
+   * @param category category
+   * @param args args list
+   * @throws Exception any unexpected exception
+   */
+  void kdiagFailure(String category, String ...args) throws Exception {
+    try {
+      int ex = exec(conf, args);
+      LOG.error("Expected an exception in category {}, return code {}",
+          category, ex);
+    } catch (KerberosDiagsFailure e) {
+      if (!e.getCategory().equals(category)) {
+        LOG.error("Expected an exception in category {}, got {}",
+            category, e, e);
+        throw e;
+      }
+    }
+  }
+
+  int kdiag(String... args) throws Exception {
+    return KDiag.exec(conf, args);
+  }
+  /**
+   * Test that the core kdiag command works when there's no KDC around.
+   * This test produces different outcomes on hosts where there is a default
+   * KDC -it needs to work on hosts without kerberos as well as those with it.
+   * @throws Throwable
+   */
+  @Test
+  public void testKDiagStandalone() throws Throwable {
+    kdiagFailure(CAT_LOGIN, ARG_KEYLEN, KEYLEN);
+  }
+
+  @Test
+  public void testKDiagNoLogin() throws Throwable {
+    kdiagFailure(CAT_LOGIN, ARG_KEYLEN, KEYLEN, ARG_NOLOGIN);
+  }
+
+  @Test
+  public void testKDiagStandaloneNofail() throws Throwable {
+    kdiag(ARG_KEYLEN, KEYLEN, ARG_NOFAIL);
+  }
+
+  @Test
+  public void testKDiagUsage() throws Throwable {
+    assertEquals(-1, kdiag("usage"));
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4db6cc44/hadoop-common-project/hadoop-common/src/test/resources/org/apache/hadoop/security/secure-hdfs-site.xml
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/resources/org/apache/hadoop/security/secure-hdfs-site.xml b/hadoop-common-project/hadoop-common/src/test/resources/org/apache/hadoop/security/secure-hdfs-site.xml
new file mode 100644
index 0000000..1a1cc78
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/test/resources/org/apache/hadoop/security/secure-hdfs-site.xml
@@ -0,0 +1,26 @@
+<?xml version="1.0"?>
+<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
+<!--
+  ~ 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.
+  -->
+
+<configuration>
+  <property>
+    <name>dfs.data.transfer.protection</name>
+    <value>org.apache.hadoop.security.SaslPropertiesResolver</value>
+  </property>
+</configuration>